You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Joe Stein <jo...@stealth.ly> on 2015/01/22 07:27:49 UTC

[DISCUSS] KIP-4 - Command line and centralized administrative operations

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

JIRA https://issues.apache.org/jira/browse/KAFKA-1694

/*******************************************
 Joe Stein
 Founder, Principal Consultant
 Big Data Open Source Security LLC
 http://www.stealth.ly
 Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
********************************************/

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Chi Hoang <ch...@groupon.com>.
For the "Sample usage" section, please consider
https://github.com/airbnb/kafkat.  We find that tool to be very easy to
use, and extremely useful for our administration tasks.

Chi

On Mon, Feb 9, 2015 at 9:03 AM, Guozhang Wang <wa...@gmail.com> wrote:

> I feel the benefits of lowering the development bar for new clients does
> not worth the complexity we need to introduce in the server side, as today
> the clients just need one more request type (metadata request) to send the
> produce / fetch to the right brokers, whereas re-routing mechanism will
> result in complicated between-brokers communication patterns that
> potentially impact Kafka performance and making debugging / trouble
> shooting much harder.
>
> An alternative way to ease the development of the clients is to use a proxy
> in front of the kafka servers, like the rest proxy we have built before,
> which we use for non-java clients primarily but also can be treated as
> handling cluster metadata discovery for clients. Comparing to the
> re-routing idea, the proxy also introduces two-hops but its layered
> architecture is simpler.
>
> Guozhang
>
>
> On Sun, Feb 8, 2015 at 8:00 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Jiangjie,
> >
> > Re routing support doesn't force clients to use it. Java and all existing
> > clients would work as now where request are intelligently routed by the
> > client, but this would lower the bar for new clients. That said I agree
> the
> > case for reroute get admin commands is much stronger than data.
> >
> > The idea of separating admin/metadata from would definitely solve some
> > problems but it would also add a lot of complexity--new ports, thread
> > pools, etc. this is an interesting idea to think over but I'm not sure if
> > it's worth it. Probably a separate effort in any case.
> >
> > -jay
> >
> > On Friday, February 6, 2015, Jiangjie Qin <jq...@linkedin.com.invalid>
> > wrote:
> >
> > > I¹m a little bit concerned about the request routers among brokers.
> > > Typically we have a dominant percentage of produce and fetch
> > > request/response. Routing them from one broker to another seems not
> > wanted.
> > > Also I think we generally have two types of requests/responses: data
> > > related and admin related. It is typically a good practice to separate
> > > data plain from control plain. That suggests we should have another
> admin
> > > port to serve those admin requests and probably have different
> > > authentication/authorization from the data port.
> > >
> > > Jiangjie (Becket) Qin
> > >
> > > On 2/6/15, 11:18 AM, "Joe Stein" <jo...@stealth.ly> wrote:
> > >
> > > >I updated the installation and sample usage for the existing patches
> on
> > > >the
> > > >KIP site
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and
> > > >+centralized+administrative+operations
> > > >
> > > >There are still a few pending items here.
> > > >
> > > >1) There was already some discussion about using the Broker that is
> the
> > > >Controller here https://issues.apache.org/jira/browse/KAFKA-1772 and
> we
> > > >should elaborate on that more in the thread or agree we are ok with
> > admin
> > > >asking for the controller to talk to and then just sending that broker
> > the
> > > >admin tasks.
> > > >
> > > >2) I like this idea https://issues.apache.org/jira/browse/KAFKA-1912
> > but
> > > >we
> > > >can refactor after KAFK-1694 committed, no? I know folks just want to
> > talk
> > > >to the broker that is the controller. It may even become useful to
> have
> > > >the
> > > >controller run on a broker that isn't even a topic broker anymore
> (small
> > > >can of worms I am opening here but it elaborates on Guozhang's hot
> spot
> > > >point.
> > > >
> > > >3) anymore feedback?
> > > >
> > > >- Joe Stein
> > > >
> > > >On Fri, Jan 23, 2015 at 3:15 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > >> A centralized admin operation protocol would be very useful.
> > > >>
> > > >> One more general comment here is that controller is originally
> > designed
> > > >>to
> > > >> only talk to other brokers through ControllerChannel, while the
> broker
> > > >> instance which carries the current controller is agnostic of its
> > > >>existence,
> > > >> and use KafkaApis to handle general Kafka requests. Having all admin
> > > >> requests redirected to the controller instance will force the broker
> > to
> > > >>be
> > > >> aware of its carried controller, and access its internal data for
> > > >>handling
> > > >> these requests. Plus with the number of clients out of Kafka's
> > control,
> > > >> this may easily cause the controller to be a hot spot in terms of
> > > >>request
> > > >> load.
> > > >>
> > > >>
> > > >> On Thu, Jan 22, 2015 at 10:09 PM, Joe Stein <jo...@stealth.ly>
> > > >>wrote:
> > > >>
> > > >> > inline
> > > >> >
> > > >> > On Thu, Jan 22, 2015 at 11:59 PM, Jay Kreps <ja...@gmail.com>
> > > >>wrote:
> > > >> >
> > > >> > > Hey Joe,
> > > >> > >
> > > >> > > This is great. A few comments on KIP-4
> > > >> > >
> > > >> > > 1. This is much needed functionality, but there are a lot of the
> > so
> > > >> let's
> > > >> > > really think these protocols through. We really want to end up
> > with
> > > >>a
> > > >> set
> > > >> > > of well thought-out, orthoganol apis. For this reason I think it
> > is
> > > >> > really
> > > >> > > important to think through the end state even if that includes
> > APIs
> > > >>we
> > > >> > > won't implement in the first phase.
> > > >> > >
> > > >> >
> > > >> > ok
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > 2. Let's please please please wait until we have switched the
> > server
> > > >> over
> > > >> > > to the new java protocol definitions. If we add upteen more ad
> hoc
> > > >> scala
> > > >> > > objects that is just generating more work for the conversion we
> > > >>know we
> > > >> > > have to do.
> > > >> > >
> > > >> >
> > > >> > ok :)
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > 3. This proposal introduces a new type of optional parameter.
> This
> > > >>is
> > > >> > > inconsistent with everything else in the protocol where we use
> -1
> > or
> > > >> some
> > > >> > > other marker value. You could argue either way but let's stick
> > with
> > > >> that
> > > >> > > for consistency. For clients that implemented the protocol in a
> > > >>better
> > > >> > way
> > > >> > > than our scala code these basic primitives are hard to change.
> > > >> > >
> > > >> >
> > > >> > yes, less confusing, ok.
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
> > > >>which
> > > >> > has
> > > >> > > brokers, topics, and partitions. I think we should rename that
> > > >>request
> > > >> > > ClusterMetadataRequest (or just MetadataRequest) and include the
> > id
> > > >>of
> > > >> > the
> > > >> > > controller. Or are there other things we could add here?
> > > >> > >
> > > >> >
> > > >> > We could add broker version to it.
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > 5. We have a tendency to try to make a lot of requests that can
> > > >>only go
> > > >> > to
> > > >> > > particular nodes. This adds a lot of burden for client
> > > >>implementations
> > > >> > (it
> > > >> > > sounds easy but each discovery can fail in many parts so it ends
> > up
> > > >> > being a
> > > >> > > full state machine to do right). I think we should consider
> making
> > > >> admin
> > > >> > > commands and ideally as many of the other apis as possible
> > > >>available on
> > > >> > all
> > > >> > > brokers and just redirect to the controller on the broker side.
> > > >>Perhaps
> > > >> > > there would be a general way to encapsulate this re-routing
> > > >>behavior.
> > > >> > >
> > > >> >
> > > >> > If we do that then we should also preserve what we have and do
> both.
> > > >>The
> > > >> > client can then decide "do I want to go to any broker and proxy"
> or
> > > >>just
> > > >> > "go to controller and run admin task". Lots of folks have seen
> > > >> controllers
> > > >> > come under distress because of their producers/consumers. There is
> > > >>ticket
> > > >> > too for controller elect and re-elect
> > > >> > https://issues.apache.org/jira/browse/KAFKA-1778 so you can force
> > it
> > > >>to
> > > >> a
> > > >> > broker that has 0 load.
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > 6. We should probably normalize the key value pairs used for
> > configs
> > > >> > rather
> > > >> > > than embedding a new formatting. So two strings rather than one
> > > >>with an
> > > >> > > internal equals sign.
> > > >> > >
> > > >> >
> > > >> > ok
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > 7. Is the postcondition of these APIs that the command has begun
> > or
> > > >> that
> > > >> > > the command has been completed? It is a lot more usable if the
> > > >>command
> > > >> > has
> > > >> > > been completed so you know that if you create a topic and then
> > > >>publish
> > > >> to
> > > >> > > it you won't get an exception about there being no such topic.
> > > >> > >
> > > >> >
> > > >> > We should define that more. There needs to be some more state
> there,
> > > >>yes.
> > > >> >
> > > >> > We should try to cover
> > > >>https://issues.apache.org/jira/browse/KAFKA-1125
> > > >> > within what we come up with.
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > 8. Describe topic and list topics duplicate a lot of stuff in
> the
> > > >> > metadata
> > > >> > > request. Is there a reason to give back topics marked for
> > deletion?
> > > >>I
> > > >> > feel
> > > >> > > like if we just make the post-condition of the delete command be
> > > >>that
> > > >> the
> > > >> > > topic is deleted that will get rid of the need for this right?
> And
> > > >>it
> > > >> > will
> > > >> > > be much more intuitive.
> > > >> > >
> > > >> >
> > > >> > I will go back and look through it.
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > 9. Should we consider batching these requests? We have generally
> > > >>tried
> > > >> to
> > > >> > > allow multiple operations to be batched. My suspicion is that
> > > >>without
> > > >> > this
> > > >> > > we will get a lot of code that does something like
> > > >> > >    for(topic: adminClient.listTopics())
> > > >> > >       adminClient.describeTopic(topic)
> > > >> > > this code will work great when you test on 5 topics but not do
> as
> > > >>well
> > > >> if
> > > >> > > you have 50k.
> > > >> > >
> > > >> >
> > > >> > So => Input is a list of topics (or none for all) and a batch
> > response
> > > >> from
> > > >> > the controller (which could be routed through another broker) of
> the
> > > >> entire
> > > >> > response? We could introduce a Batch keyword to explicitly show
> the
> > > >>usage
> > > >> > of it.
> > > >> >
> > > >> >
> > > >> > > 10. I think we should also discuss how we want to expose a
> > > >>programmatic
> > > >> > JVM
> > > >> > > client api for these operations. Currently people rely on
> > AdminUtils
> > > >> > which
> > > >> > > is totally sketchy. I think we probably need another client
> under
> > > >> > clients/
> > > >> > > that exposes administrative functionality. We will need this
> just
> > to
> > > >> > > properly test the new apis, I suspect. We should figure out that
> > > >>API.
> > > >> > >
> > > >> >
> > > >> > We were talking about that here
> > > >> > https://issues.apache.org/jira/browse/KAFKA-1774 and wrote it in
> > java
> > > >> > https://reviews.apache.org/r/29301/diff/7/?page=4#75 so we could
> do
> > > >> > something like that, sure.
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > 11. The other information that would be really useful to get
> would
> > > >>be
> > > >> > > information about partitions--how much data is in the partition,
> > > >>what
> > > >> are
> > > >> > > the segment offsets, what is the log-end offset (i.e. last
> > offset),
> > > >> what
> > > >> > is
> > > >> > > the compaction point, etc. I think that done right this would be
> > the
> > > >> > > successor to the very awkward OffsetRequest we have today.
> > > >> > >
> > > >> >
> > > >> > yes!
> > > >> >
> > > >> >
> > > >> > >
> > > >> > > -Jay
> > > >> > >
> > > >> > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
> joe.stein@stealth.ly
> > >
> > > >> > wrote:
> > > >> > >
> > > >> > > > Hi, created a KIP
> > > >> > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+an
> > > >>d+centralized+administrative+operations
> > > >> > > >
> > > >> > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> > > >> > > >
> > > >> > > > /*******************************************
> > > >> > > >  Joe Stein
> > > >> > > >  Founder, Principal Consultant
> > > >> > > >  Big Data Open Source Security LLC
> > > >> > > >  http://www.stealth.ly
> > > >> > > >  Twitter: @allthingshadoop
> > > >><http://www.twitter.com/allthingshadoop>
> > > >> > > > ********************************************/
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > > >>
> > > >>
> > > >> --
> > > >> -- Guozhang
> > > >>
> > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Guozhang Wang <wa...@gmail.com>.
I feel the benefits of lowering the development bar for new clients does
not worth the complexity we need to introduce in the server side, as today
the clients just need one more request type (metadata request) to send the
produce / fetch to the right brokers, whereas re-routing mechanism will
result in complicated between-brokers communication patterns that
potentially impact Kafka performance and making debugging / trouble
shooting much harder.

An alternative way to ease the development of the clients is to use a proxy
in front of the kafka servers, like the rest proxy we have built before,
which we use for non-java clients primarily but also can be treated as
handling cluster metadata discovery for clients. Comparing to the
re-routing idea, the proxy also introduces two-hops but its layered
architecture is simpler.

Guozhang


On Sun, Feb 8, 2015 at 8:00 AM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Jiangjie,
>
> Re routing support doesn't force clients to use it. Java and all existing
> clients would work as now where request are intelligently routed by the
> client, but this would lower the bar for new clients. That said I agree the
> case for reroute get admin commands is much stronger than data.
>
> The idea of separating admin/metadata from would definitely solve some
> problems but it would also add a lot of complexity--new ports, thread
> pools, etc. this is an interesting idea to think over but I'm not sure if
> it's worth it. Probably a separate effort in any case.
>
> -jay
>
> On Friday, February 6, 2015, Jiangjie Qin <jq...@linkedin.com.invalid>
> wrote:
>
> > I¹m a little bit concerned about the request routers among brokers.
> > Typically we have a dominant percentage of produce and fetch
> > request/response. Routing them from one broker to another seems not
> wanted.
> > Also I think we generally have two types of requests/responses: data
> > related and admin related. It is typically a good practice to separate
> > data plain from control plain. That suggests we should have another admin
> > port to serve those admin requests and probably have different
> > authentication/authorization from the data port.
> >
> > Jiangjie (Becket) Qin
> >
> > On 2/6/15, 11:18 AM, "Joe Stein" <jo...@stealth.ly> wrote:
> >
> > >I updated the installation and sample usage for the existing patches on
> > >the
> > >KIP site
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and
> > >+centralized+administrative+operations
> > >
> > >There are still a few pending items here.
> > >
> > >1) There was already some discussion about using the Broker that is the
> > >Controller here https://issues.apache.org/jira/browse/KAFKA-1772 and we
> > >should elaborate on that more in the thread or agree we are ok with
> admin
> > >asking for the controller to talk to and then just sending that broker
> the
> > >admin tasks.
> > >
> > >2) I like this idea https://issues.apache.org/jira/browse/KAFKA-1912
> but
> > >we
> > >can refactor after KAFK-1694 committed, no? I know folks just want to
> talk
> > >to the broker that is the controller. It may even become useful to have
> > >the
> > >controller run on a broker that isn't even a topic broker anymore (small
> > >can of worms I am opening here but it elaborates on Guozhang's hot spot
> > >point.
> > >
> > >3) anymore feedback?
> > >
> > >- Joe Stein
> > >
> > >On Fri, Jan 23, 2015 at 3:15 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > >> A centralized admin operation protocol would be very useful.
> > >>
> > >> One more general comment here is that controller is originally
> designed
> > >>to
> > >> only talk to other brokers through ControllerChannel, while the broker
> > >> instance which carries the current controller is agnostic of its
> > >>existence,
> > >> and use KafkaApis to handle general Kafka requests. Having all admin
> > >> requests redirected to the controller instance will force the broker
> to
> > >>be
> > >> aware of its carried controller, and access its internal data for
> > >>handling
> > >> these requests. Plus with the number of clients out of Kafka's
> control,
> > >> this may easily cause the controller to be a hot spot in terms of
> > >>request
> > >> load.
> > >>
> > >>
> > >> On Thu, Jan 22, 2015 at 10:09 PM, Joe Stein <jo...@stealth.ly>
> > >>wrote:
> > >>
> > >> > inline
> > >> >
> > >> > On Thu, Jan 22, 2015 at 11:59 PM, Jay Kreps <ja...@gmail.com>
> > >>wrote:
> > >> >
> > >> > > Hey Joe,
> > >> > >
> > >> > > This is great. A few comments on KIP-4
> > >> > >
> > >> > > 1. This is much needed functionality, but there are a lot of the
> so
> > >> let's
> > >> > > really think these protocols through. We really want to end up
> with
> > >>a
> > >> set
> > >> > > of well thought-out, orthoganol apis. For this reason I think it
> is
> > >> > really
> > >> > > important to think through the end state even if that includes
> APIs
> > >>we
> > >> > > won't implement in the first phase.
> > >> > >
> > >> >
> > >> > ok
> > >> >
> > >> >
> > >> > >
> > >> > > 2. Let's please please please wait until we have switched the
> server
> > >> over
> > >> > > to the new java protocol definitions. If we add upteen more ad hoc
> > >> scala
> > >> > > objects that is just generating more work for the conversion we
> > >>know we
> > >> > > have to do.
> > >> > >
> > >> >
> > >> > ok :)
> > >> >
> > >> >
> > >> > >
> > >> > > 3. This proposal introduces a new type of optional parameter. This
> > >>is
> > >> > > inconsistent with everything else in the protocol where we use -1
> or
> > >> some
> > >> > > other marker value. You could argue either way but let's stick
> with
> > >> that
> > >> > > for consistency. For clients that implemented the protocol in a
> > >>better
> > >> > way
> > >> > > than our scala code these basic primitives are hard to change.
> > >> > >
> > >> >
> > >> > yes, less confusing, ok.
> > >> >
> > >> >
> > >> > >
> > >> > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
> > >>which
> > >> > has
> > >> > > brokers, topics, and partitions. I think we should rename that
> > >>request
> > >> > > ClusterMetadataRequest (or just MetadataRequest) and include the
> id
> > >>of
> > >> > the
> > >> > > controller. Or are there other things we could add here?
> > >> > >
> > >> >
> > >> > We could add broker version to it.
> > >> >
> > >> >
> > >> > >
> > >> > > 5. We have a tendency to try to make a lot of requests that can
> > >>only go
> > >> > to
> > >> > > particular nodes. This adds a lot of burden for client
> > >>implementations
> > >> > (it
> > >> > > sounds easy but each discovery can fail in many parts so it ends
> up
> > >> > being a
> > >> > > full state machine to do right). I think we should consider making
> > >> admin
> > >> > > commands and ideally as many of the other apis as possible
> > >>available on
> > >> > all
> > >> > > brokers and just redirect to the controller on the broker side.
> > >>Perhaps
> > >> > > there would be a general way to encapsulate this re-routing
> > >>behavior.
> > >> > >
> > >> >
> > >> > If we do that then we should also preserve what we have and do both.
> > >>The
> > >> > client can then decide "do I want to go to any broker and proxy" or
> > >>just
> > >> > "go to controller and run admin task". Lots of folks have seen
> > >> controllers
> > >> > come under distress because of their producers/consumers. There is
> > >>ticket
> > >> > too for controller elect and re-elect
> > >> > https://issues.apache.org/jira/browse/KAFKA-1778 so you can force
> it
> > >>to
> > >> a
> > >> > broker that has 0 load.
> > >> >
> > >> >
> > >> > >
> > >> > > 6. We should probably normalize the key value pairs used for
> configs
> > >> > rather
> > >> > > than embedding a new formatting. So two strings rather than one
> > >>with an
> > >> > > internal equals sign.
> > >> > >
> > >> >
> > >> > ok
> > >> >
> > >> >
> > >> > >
> > >> > > 7. Is the postcondition of these APIs that the command has begun
> or
> > >> that
> > >> > > the command has been completed? It is a lot more usable if the
> > >>command
> > >> > has
> > >> > > been completed so you know that if you create a topic and then
> > >>publish
> > >> to
> > >> > > it you won't get an exception about there being no such topic.
> > >> > >
> > >> >
> > >> > We should define that more. There needs to be some more state there,
> > >>yes.
> > >> >
> > >> > We should try to cover
> > >>https://issues.apache.org/jira/browse/KAFKA-1125
> > >> > within what we come up with.
> > >> >
> > >> >
> > >> > >
> > >> > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > >> > metadata
> > >> > > request. Is there a reason to give back topics marked for
> deletion?
> > >>I
> > >> > feel
> > >> > > like if we just make the post-condition of the delete command be
> > >>that
> > >> the
> > >> > > topic is deleted that will get rid of the need for this right? And
> > >>it
> > >> > will
> > >> > > be much more intuitive.
> > >> > >
> > >> >
> > >> > I will go back and look through it.
> > >> >
> > >> >
> > >> > >
> > >> > > 9. Should we consider batching these requests? We have generally
> > >>tried
> > >> to
> > >> > > allow multiple operations to be batched. My suspicion is that
> > >>without
> > >> > this
> > >> > > we will get a lot of code that does something like
> > >> > >    for(topic: adminClient.listTopics())
> > >> > >       adminClient.describeTopic(topic)
> > >> > > this code will work great when you test on 5 topics but not do as
> > >>well
> > >> if
> > >> > > you have 50k.
> > >> > >
> > >> >
> > >> > So => Input is a list of topics (or none for all) and a batch
> response
> > >> from
> > >> > the controller (which could be routed through another broker) of the
> > >> entire
> > >> > response? We could introduce a Batch keyword to explicitly show the
> > >>usage
> > >> > of it.
> > >> >
> > >> >
> > >> > > 10. I think we should also discuss how we want to expose a
> > >>programmatic
> > >> > JVM
> > >> > > client api for these operations. Currently people rely on
> AdminUtils
> > >> > which
> > >> > > is totally sketchy. I think we probably need another client under
> > >> > clients/
> > >> > > that exposes administrative functionality. We will need this just
> to
> > >> > > properly test the new apis, I suspect. We should figure out that
> > >>API.
> > >> > >
> > >> >
> > >> > We were talking about that here
> > >> > https://issues.apache.org/jira/browse/KAFKA-1774 and wrote it in
> java
> > >> > https://reviews.apache.org/r/29301/diff/7/?page=4#75 so we could do
> > >> > something like that, sure.
> > >> >
> > >> >
> > >> > >
> > >> > > 11. The other information that would be really useful to get would
> > >>be
> > >> > > information about partitions--how much data is in the partition,
> > >>what
> > >> are
> > >> > > the segment offsets, what is the log-end offset (i.e. last
> offset),
> > >> what
> > >> > is
> > >> > > the compaction point, etc. I think that done right this would be
> the
> > >> > > successor to the very awkward OffsetRequest we have today.
> > >> > >
> > >> >
> > >> > yes!
> > >> >
> > >> >
> > >> > >
> > >> > > -Jay
> > >> > >
> > >> > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <joe.stein@stealth.ly
> >
> > >> > wrote:
> > >> > >
> > >> > > > Hi, created a KIP
> > >> > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+an
> > >>d+centralized+administrative+operations
> > >> > > >
> > >> > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> > >> > > >
> > >> > > > /*******************************************
> > >> > > >  Joe Stein
> > >> > > >  Founder, Principal Consultant
> > >> > > >  Big Data Open Source Security LLC
> > >> > > >  http://www.stealth.ly
> > >> > > >  Twitter: @allthingshadoop
> > >><http://www.twitter.com/allthingshadoop>
> > >> > > > ********************************************/
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >>
> > >>
> > >> --
> > >> -- Guozhang
> > >>
> >
> >
>



-- 
-- Guozhang

[DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
Hey Jiangjie,

Re routing support doesn't force clients to use it. Java and all existing
clients would work as now where request are intelligently routed by the
client, but this would lower the bar for new clients. That said I agree the
case for reroute get admin commands is much stronger than data.

The idea of separating admin/metadata from would definitely solve some
problems but it would also add a lot of complexity--new ports, thread
pools, etc. this is an interesting idea to think over but I'm not sure if
it's worth it. Probably a separate effort in any case.

-jay

On Friday, February 6, 2015, Jiangjie Qin <jq...@linkedin.com.invalid> wrote:

> I¹m a little bit concerned about the request routers among brokers.
> Typically we have a dominant percentage of produce and fetch
> request/response. Routing them from one broker to another seems not wanted.
> Also I think we generally have two types of requests/responses: data
> related and admin related. It is typically a good practice to separate
> data plain from control plain. That suggests we should have another admin
> port to serve those admin requests and probably have different
> authentication/authorization from the data port.
>
> Jiangjie (Becket) Qin
>
> On 2/6/15, 11:18 AM, "Joe Stein" <jo...@stealth.ly> wrote:
>
> >I updated the installation and sample usage for the existing patches on
> >the
> >KIP site
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and
> >+centralized+administrative+operations
> >
> >There are still a few pending items here.
> >
> >1) There was already some discussion about using the Broker that is the
> >Controller here https://issues.apache.org/jira/browse/KAFKA-1772 and we
> >should elaborate on that more in the thread or agree we are ok with admin
> >asking for the controller to talk to and then just sending that broker the
> >admin tasks.
> >
> >2) I like this idea https://issues.apache.org/jira/browse/KAFKA-1912 but
> >we
> >can refactor after KAFK-1694 committed, no? I know folks just want to talk
> >to the broker that is the controller. It may even become useful to have
> >the
> >controller run on a broker that isn't even a topic broker anymore (small
> >can of worms I am opening here but it elaborates on Guozhang's hot spot
> >point.
> >
> >3) anymore feedback?
> >
> >- Joe Stein
> >
> >On Fri, Jan 23, 2015 at 3:15 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> >> A centralized admin operation protocol would be very useful.
> >>
> >> One more general comment here is that controller is originally designed
> >>to
> >> only talk to other brokers through ControllerChannel, while the broker
> >> instance which carries the current controller is agnostic of its
> >>existence,
> >> and use KafkaApis to handle general Kafka requests. Having all admin
> >> requests redirected to the controller instance will force the broker to
> >>be
> >> aware of its carried controller, and access its internal data for
> >>handling
> >> these requests. Plus with the number of clients out of Kafka's control,
> >> this may easily cause the controller to be a hot spot in terms of
> >>request
> >> load.
> >>
> >>
> >> On Thu, Jan 22, 2015 at 10:09 PM, Joe Stein <jo...@stealth.ly>
> >>wrote:
> >>
> >> > inline
> >> >
> >> > On Thu, Jan 22, 2015 at 11:59 PM, Jay Kreps <ja...@gmail.com>
> >>wrote:
> >> >
> >> > > Hey Joe,
> >> > >
> >> > > This is great. A few comments on KIP-4
> >> > >
> >> > > 1. This is much needed functionality, but there are a lot of the so
> >> let's
> >> > > really think these protocols through. We really want to end up with
> >>a
> >> set
> >> > > of well thought-out, orthoganol apis. For this reason I think it is
> >> > really
> >> > > important to think through the end state even if that includes APIs
> >>we
> >> > > won't implement in the first phase.
> >> > >
> >> >
> >> > ok
> >> >
> >> >
> >> > >
> >> > > 2. Let's please please please wait until we have switched the server
> >> over
> >> > > to the new java protocol definitions. If we add upteen more ad hoc
> >> scala
> >> > > objects that is just generating more work for the conversion we
> >>know we
> >> > > have to do.
> >> > >
> >> >
> >> > ok :)
> >> >
> >> >
> >> > >
> >> > > 3. This proposal introduces a new type of optional parameter. This
> >>is
> >> > > inconsistent with everything else in the protocol where we use -1 or
> >> some
> >> > > other marker value. You could argue either way but let's stick with
> >> that
> >> > > for consistency. For clients that implemented the protocol in a
> >>better
> >> > way
> >> > > than our scala code these basic primitives are hard to change.
> >> > >
> >> >
> >> > yes, less confusing, ok.
> >> >
> >> >
> >> > >
> >> > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
> >>which
> >> > has
> >> > > brokers, topics, and partitions. I think we should rename that
> >>request
> >> > > ClusterMetadataRequest (or just MetadataRequest) and include the id
> >>of
> >> > the
> >> > > controller. Or are there other things we could add here?
> >> > >
> >> >
> >> > We could add broker version to it.
> >> >
> >> >
> >> > >
> >> > > 5. We have a tendency to try to make a lot of requests that can
> >>only go
> >> > to
> >> > > particular nodes. This adds a lot of burden for client
> >>implementations
> >> > (it
> >> > > sounds easy but each discovery can fail in many parts so it ends up
> >> > being a
> >> > > full state machine to do right). I think we should consider making
> >> admin
> >> > > commands and ideally as many of the other apis as possible
> >>available on
> >> > all
> >> > > brokers and just redirect to the controller on the broker side.
> >>Perhaps
> >> > > there would be a general way to encapsulate this re-routing
> >>behavior.
> >> > >
> >> >
> >> > If we do that then we should also preserve what we have and do both.
> >>The
> >> > client can then decide "do I want to go to any broker and proxy" or
> >>just
> >> > "go to controller and run admin task". Lots of folks have seen
> >> controllers
> >> > come under distress because of their producers/consumers. There is
> >>ticket
> >> > too for controller elect and re-elect
> >> > https://issues.apache.org/jira/browse/KAFKA-1778 so you can force it
> >>to
> >> a
> >> > broker that has 0 load.
> >> >
> >> >
> >> > >
> >> > > 6. We should probably normalize the key value pairs used for configs
> >> > rather
> >> > > than embedding a new formatting. So two strings rather than one
> >>with an
> >> > > internal equals sign.
> >> > >
> >> >
> >> > ok
> >> >
> >> >
> >> > >
> >> > > 7. Is the postcondition of these APIs that the command has begun or
> >> that
> >> > > the command has been completed? It is a lot more usable if the
> >>command
> >> > has
> >> > > been completed so you know that if you create a topic and then
> >>publish
> >> to
> >> > > it you won't get an exception about there being no such topic.
> >> > >
> >> >
> >> > We should define that more. There needs to be some more state there,
> >>yes.
> >> >
> >> > We should try to cover
> >>https://issues.apache.org/jira/browse/KAFKA-1125
> >> > within what we come up with.
> >> >
> >> >
> >> > >
> >> > > 8. Describe topic and list topics duplicate a lot of stuff in the
> >> > metadata
> >> > > request. Is there a reason to give back topics marked for deletion?
> >>I
> >> > feel
> >> > > like if we just make the post-condition of the delete command be
> >>that
> >> the
> >> > > topic is deleted that will get rid of the need for this right? And
> >>it
> >> > will
> >> > > be much more intuitive.
> >> > >
> >> >
> >> > I will go back and look through it.
> >> >
> >> >
> >> > >
> >> > > 9. Should we consider batching these requests? We have generally
> >>tried
> >> to
> >> > > allow multiple operations to be batched. My suspicion is that
> >>without
> >> > this
> >> > > we will get a lot of code that does something like
> >> > >    for(topic: adminClient.listTopics())
> >> > >       adminClient.describeTopic(topic)
> >> > > this code will work great when you test on 5 topics but not do as
> >>well
> >> if
> >> > > you have 50k.
> >> > >
> >> >
> >> > So => Input is a list of topics (or none for all) and a batch response
> >> from
> >> > the controller (which could be routed through another broker) of the
> >> entire
> >> > response? We could introduce a Batch keyword to explicitly show the
> >>usage
> >> > of it.
> >> >
> >> >
> >> > > 10. I think we should also discuss how we want to expose a
> >>programmatic
> >> > JVM
> >> > > client api for these operations. Currently people rely on AdminUtils
> >> > which
> >> > > is totally sketchy. I think we probably need another client under
> >> > clients/
> >> > > that exposes administrative functionality. We will need this just to
> >> > > properly test the new apis, I suspect. We should figure out that
> >>API.
> >> > >
> >> >
> >> > We were talking about that here
> >> > https://issues.apache.org/jira/browse/KAFKA-1774 and wrote it in java
> >> > https://reviews.apache.org/r/29301/diff/7/?page=4#75 so we could do
> >> > something like that, sure.
> >> >
> >> >
> >> > >
> >> > > 11. The other information that would be really useful to get would
> >>be
> >> > > information about partitions--how much data is in the partition,
> >>what
> >> are
> >> > > the segment offsets, what is the log-end offset (i.e. last offset),
> >> what
> >> > is
> >> > > the compaction point, etc. I think that done right this would be the
> >> > > successor to the very awkward OffsetRequest we have today.
> >> > >
> >> >
> >> > yes!
> >> >
> >> >
> >> > >
> >> > > -Jay
> >> > >
> >> > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly>
> >> > wrote:
> >> > >
> >> > > > Hi, created a KIP
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+an
> >>d+centralized+administrative+operations
> >> > > >
> >> > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> >> > > >
> >> > > > /*******************************************
> >> > > >  Joe Stein
> >> > > >  Founder, Principal Consultant
> >> > > >  Big Data Open Source Security LLC
> >> > > >  http://www.stealth.ly
> >> > > >  Twitter: @allthingshadoop
> >><http://www.twitter.com/allthingshadoop>
> >> > > > ********************************************/
> >> > > >
> >> > >
> >> >
> >>
> >>
> >>
> >> --
> >> -- Guozhang
> >>
>
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jiangjie Qin <jq...@linkedin.com.INVALID>.
I¹m a little bit concerned about the request routers among brokers.
Typically we have a dominant percentage of produce and fetch
request/response. Routing them from one broker to another seems not wanted.
Also I think we generally have two types of requests/responses: data
related and admin related. It is typically a good practice to separate
data plain from control plain. That suggests we should have another admin
port to serve those admin requests and probably have different
authentication/authorization from the data port.

Jiangjie (Becket) Qin

On 2/6/15, 11:18 AM, "Joe Stein" <jo...@stealth.ly> wrote:

>I updated the installation and sample usage for the existing patches on
>the
>KIP site
>https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and
>+centralized+administrative+operations
>
>There are still a few pending items here.
>
>1) There was already some discussion about using the Broker that is the
>Controller here https://issues.apache.org/jira/browse/KAFKA-1772 and we
>should elaborate on that more in the thread or agree we are ok with admin
>asking for the controller to talk to and then just sending that broker the
>admin tasks.
>
>2) I like this idea https://issues.apache.org/jira/browse/KAFKA-1912 but
>we
>can refactor after KAFK-1694 committed, no? I know folks just want to talk
>to the broker that is the controller. It may even become useful to have
>the
>controller run on a broker that isn't even a topic broker anymore (small
>can of worms I am opening here but it elaborates on Guozhang's hot spot
>point.
>
>3) anymore feedback?
>
>- Joe Stein
>
>On Fri, Jan 23, 2015 at 3:15 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
>> A centralized admin operation protocol would be very useful.
>>
>> One more general comment here is that controller is originally designed
>>to
>> only talk to other brokers through ControllerChannel, while the broker
>> instance which carries the current controller is agnostic of its
>>existence,
>> and use KafkaApis to handle general Kafka requests. Having all admin
>> requests redirected to the controller instance will force the broker to
>>be
>> aware of its carried controller, and access its internal data for
>>handling
>> these requests. Plus with the number of clients out of Kafka's control,
>> this may easily cause the controller to be a hot spot in terms of
>>request
>> load.
>>
>>
>> On Thu, Jan 22, 2015 at 10:09 PM, Joe Stein <jo...@stealth.ly>
>>wrote:
>>
>> > inline
>> >
>> > On Thu, Jan 22, 2015 at 11:59 PM, Jay Kreps <ja...@gmail.com>
>>wrote:
>> >
>> > > Hey Joe,
>> > >
>> > > This is great. A few comments on KIP-4
>> > >
>> > > 1. This is much needed functionality, but there are a lot of the so
>> let's
>> > > really think these protocols through. We really want to end up with
>>a
>> set
>> > > of well thought-out, orthoganol apis. For this reason I think it is
>> > really
>> > > important to think through the end state even if that includes APIs
>>we
>> > > won't implement in the first phase.
>> > >
>> >
>> > ok
>> >
>> >
>> > >
>> > > 2. Let's please please please wait until we have switched the server
>> over
>> > > to the new java protocol definitions. If we add upteen more ad hoc
>> scala
>> > > objects that is just generating more work for the conversion we
>>know we
>> > > have to do.
>> > >
>> >
>> > ok :)
>> >
>> >
>> > >
>> > > 3. This proposal introduces a new type of optional parameter. This
>>is
>> > > inconsistent with everything else in the protocol where we use -1 or
>> some
>> > > other marker value. You could argue either way but let's stick with
>> that
>> > > for consistency. For clients that implemented the protocol in a
>>better
>> > way
>> > > than our scala code these basic primitives are hard to change.
>> > >
>> >
>> > yes, less confusing, ok.
>> >
>> >
>> > >
>> > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
>>which
>> > has
>> > > brokers, topics, and partitions. I think we should rename that
>>request
>> > > ClusterMetadataRequest (or just MetadataRequest) and include the id
>>of
>> > the
>> > > controller. Or are there other things we could add here?
>> > >
>> >
>> > We could add broker version to it.
>> >
>> >
>> > >
>> > > 5. We have a tendency to try to make a lot of requests that can
>>only go
>> > to
>> > > particular nodes. This adds a lot of burden for client
>>implementations
>> > (it
>> > > sounds easy but each discovery can fail in many parts so it ends up
>> > being a
>> > > full state machine to do right). I think we should consider making
>> admin
>> > > commands and ideally as many of the other apis as possible
>>available on
>> > all
>> > > brokers and just redirect to the controller on the broker side.
>>Perhaps
>> > > there would be a general way to encapsulate this re-routing
>>behavior.
>> > >
>> >
>> > If we do that then we should also preserve what we have and do both.
>>The
>> > client can then decide "do I want to go to any broker and proxy" or
>>just
>> > "go to controller and run admin task". Lots of folks have seen
>> controllers
>> > come under distress because of their producers/consumers. There is
>>ticket
>> > too for controller elect and re-elect
>> > https://issues.apache.org/jira/browse/KAFKA-1778 so you can force it
>>to
>> a
>> > broker that has 0 load.
>> >
>> >
>> > >
>> > > 6. We should probably normalize the key value pairs used for configs
>> > rather
>> > > than embedding a new formatting. So two strings rather than one
>>with an
>> > > internal equals sign.
>> > >
>> >
>> > ok
>> >
>> >
>> > >
>> > > 7. Is the postcondition of these APIs that the command has begun or
>> that
>> > > the command has been completed? It is a lot more usable if the
>>command
>> > has
>> > > been completed so you know that if you create a topic and then
>>publish
>> to
>> > > it you won't get an exception about there being no such topic.
>> > >
>> >
>> > We should define that more. There needs to be some more state there,
>>yes.
>> >
>> > We should try to cover
>>https://issues.apache.org/jira/browse/KAFKA-1125
>> > within what we come up with.
>> >
>> >
>> > >
>> > > 8. Describe topic and list topics duplicate a lot of stuff in the
>> > metadata
>> > > request. Is there a reason to give back topics marked for deletion?
>>I
>> > feel
>> > > like if we just make the post-condition of the delete command be
>>that
>> the
>> > > topic is deleted that will get rid of the need for this right? And
>>it
>> > will
>> > > be much more intuitive.
>> > >
>> >
>> > I will go back and look through it.
>> >
>> >
>> > >
>> > > 9. Should we consider batching these requests? We have generally
>>tried
>> to
>> > > allow multiple operations to be batched. My suspicion is that
>>without
>> > this
>> > > we will get a lot of code that does something like
>> > >    for(topic: adminClient.listTopics())
>> > >       adminClient.describeTopic(topic)
>> > > this code will work great when you test on 5 topics but not do as
>>well
>> if
>> > > you have 50k.
>> > >
>> >
>> > So => Input is a list of topics (or none for all) and a batch response
>> from
>> > the controller (which could be routed through another broker) of the
>> entire
>> > response? We could introduce a Batch keyword to explicitly show the
>>usage
>> > of it.
>> >
>> >
>> > > 10. I think we should also discuss how we want to expose a
>>programmatic
>> > JVM
>> > > client api for these operations. Currently people rely on AdminUtils
>> > which
>> > > is totally sketchy. I think we probably need another client under
>> > clients/
>> > > that exposes administrative functionality. We will need this just to
>> > > properly test the new apis, I suspect. We should figure out that
>>API.
>> > >
>> >
>> > We were talking about that here
>> > https://issues.apache.org/jira/browse/KAFKA-1774 and wrote it in java
>> > https://reviews.apache.org/r/29301/diff/7/?page=4#75 so we could do
>> > something like that, sure.
>> >
>> >
>> > >
>> > > 11. The other information that would be really useful to get would
>>be
>> > > information about partitions--how much data is in the partition,
>>what
>> are
>> > > the segment offsets, what is the log-end offset (i.e. last offset),
>> what
>> > is
>> > > the compaction point, etc. I think that done right this would be the
>> > > successor to the very awkward OffsetRequest we have today.
>> > >
>> >
>> > yes!
>> >
>> >
>> > >
>> > > -Jay
>> > >
>> > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly>
>> > wrote:
>> > >
>> > > > Hi, created a KIP
>> > > >
>> > > >
>> > >
>> >
>> 
>>https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+an
>>d+centralized+administrative+operations
>> > > >
>> > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
>> > > >
>> > > > /*******************************************
>> > > >  Joe Stein
>> > > >  Founder, Principal Consultant
>> > > >  Big Data Open Source Security LLC
>> > > >  http://www.stealth.ly
>> > > >  Twitter: @allthingshadoop
>><http://www.twitter.com/allthingshadoop>
>> > > > ********************************************/
>> > > >
>> > >
>> >
>>
>>
>>
>> --
>> -- Guozhang
>>


Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
Hey Joe,

I think this is proposing several things:
1. A new command line utility. This isn't really fully specified here.
There is sample usage but I actually don't really understand what all the
commands will be. Also, presumably this will replace the existing shell
scripts, right? We obviously don't want to be in a state where we have
both...
2. A new set of language agnostic administrative protocols.
3. A new Java API for issuing administrative requests using the protocol. I
don't see any discussion on what this will look like.

It might be easiest to tackle these one at a time, no? If not we really do
need to get a complete description at each layer as these are pretty core
public apis.

-Jay

On Fri, Feb 6, 2015 at 11:18 AM, Joe Stein <jo...@stealth.ly> wrote:

> I updated the installation and sample usage for the existing patches on the
> KIP site
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>
> There are still a few pending items here.
>
> 1) There was already some discussion about using the Broker that is the
> Controller here https://issues.apache.org/jira/browse/KAFKA-1772 and we
> should elaborate on that more in the thread or agree we are ok with admin
> asking for the controller to talk to and then just sending that broker the
> admin tasks.
>
> 2) I like this idea https://issues.apache.org/jira/browse/KAFKA-1912 but
> we
> can refactor after KAFK-1694 committed, no? I know folks just want to talk
> to the broker that is the controller. It may even become useful to have the
> controller run on a broker that isn't even a topic broker anymore (small
> can of worms I am opening here but it elaborates on Guozhang's hot spot
> point.
>
> 3) anymore feedback?
>
> - Joe Stein
>
> On Fri, Jan 23, 2015 at 3:15 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > A centralized admin operation protocol would be very useful.
> >
> > One more general comment here is that controller is originally designed
> to
> > only talk to other brokers through ControllerChannel, while the broker
> > instance which carries the current controller is agnostic of its
> existence,
> > and use KafkaApis to handle general Kafka requests. Having all admin
> > requests redirected to the controller instance will force the broker to
> be
> > aware of its carried controller, and access its internal data for
> handling
> > these requests. Plus with the number of clients out of Kafka's control,
> > this may easily cause the controller to be a hot spot in terms of request
> > load.
> >
> >
> > On Thu, Jan 22, 2015 at 10:09 PM, Joe Stein <jo...@stealth.ly>
> wrote:
> >
> > > inline
> > >
> > > On Thu, Jan 22, 2015 at 11:59 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hey Joe,
> > > >
> > > > This is great. A few comments on KIP-4
> > > >
> > > > 1. This is much needed functionality, but there are a lot of the so
> > let's
> > > > really think these protocols through. We really want to end up with a
> > set
> > > > of well thought-out, orthoganol apis. For this reason I think it is
> > > really
> > > > important to think through the end state even if that includes APIs
> we
> > > > won't implement in the first phase.
> > > >
> > >
> > > ok
> > >
> > >
> > > >
> > > > 2. Let's please please please wait until we have switched the server
> > over
> > > > to the new java protocol definitions. If we add upteen more ad hoc
> > scala
> > > > objects that is just generating more work for the conversion we know
> we
> > > > have to do.
> > > >
> > >
> > > ok :)
> > >
> > >
> > > >
> > > > 3. This proposal introduces a new type of optional parameter. This is
> > > > inconsistent with everything else in the protocol where we use -1 or
> > some
> > > > other marker value. You could argue either way but let's stick with
> > that
> > > > for consistency. For clients that implemented the protocol in a
> better
> > > way
> > > > than our scala code these basic primitives are hard to change.
> > > >
> > >
> > > yes, less confusing, ok.
> > >
> > >
> > > >
> > > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
> which
> > > has
> > > > brokers, topics, and partitions. I think we should rename that
> request
> > > > ClusterMetadataRequest (or just MetadataRequest) and include the id
> of
> > > the
> > > > controller. Or are there other things we could add here?
> > > >
> > >
> > > We could add broker version to it.
> > >
> > >
> > > >
> > > > 5. We have a tendency to try to make a lot of requests that can only
> go
> > > to
> > > > particular nodes. This adds a lot of burden for client
> implementations
> > > (it
> > > > sounds easy but each discovery can fail in many parts so it ends up
> > > being a
> > > > full state machine to do right). I think we should consider making
> > admin
> > > > commands and ideally as many of the other apis as possible available
> on
> > > all
> > > > brokers and just redirect to the controller on the broker side.
> Perhaps
> > > > there would be a general way to encapsulate this re-routing behavior.
> > > >
> > >
> > > If we do that then we should also preserve what we have and do both.
> The
> > > client can then decide "do I want to go to any broker and proxy" or
> just
> > > "go to controller and run admin task". Lots of folks have seen
> > controllers
> > > come under distress because of their producers/consumers. There is
> ticket
> > > too for controller elect and re-elect
> > > https://issues.apache.org/jira/browse/KAFKA-1778 so you can force it
> to
> > a
> > > broker that has 0 load.
> > >
> > >
> > > >
> > > > 6. We should probably normalize the key value pairs used for configs
> > > rather
> > > > than embedding a new formatting. So two strings rather than one with
> an
> > > > internal equals sign.
> > > >
> > >
> > > ok
> > >
> > >
> > > >
> > > > 7. Is the postcondition of these APIs that the command has begun or
> > that
> > > > the command has been completed? It is a lot more usable if the
> command
> > > has
> > > > been completed so you know that if you create a topic and then
> publish
> > to
> > > > it you won't get an exception about there being no such topic.
> > > >
> > >
> > > We should define that more. There needs to be some more state there,
> yes.
> > >
> > > We should try to cover
> https://issues.apache.org/jira/browse/KAFKA-1125
> > > within what we come up with.
> > >
> > >
> > > >
> > > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > > metadata
> > > > request. Is there a reason to give back topics marked for deletion? I
> > > feel
> > > > like if we just make the post-condition of the delete command be that
> > the
> > > > topic is deleted that will get rid of the need for this right? And it
> > > will
> > > > be much more intuitive.
> > > >
> > >
> > > I will go back and look through it.
> > >
> > >
> > > >
> > > > 9. Should we consider batching these requests? We have generally
> tried
> > to
> > > > allow multiple operations to be batched. My suspicion is that without
> > > this
> > > > we will get a lot of code that does something like
> > > >    for(topic: adminClient.listTopics())
> > > >       adminClient.describeTopic(topic)
> > > > this code will work great when you test on 5 topics but not do as
> well
> > if
> > > > you have 50k.
> > > >
> > >
> > > So => Input is a list of topics (or none for all) and a batch response
> > from
> > > the controller (which could be routed through another broker) of the
> > entire
> > > response? We could introduce a Batch keyword to explicitly show the
> usage
> > > of it.
> > >
> > >
> > > > 10. I think we should also discuss how we want to expose a
> programmatic
> > > JVM
> > > > client api for these operations. Currently people rely on AdminUtils
> > > which
> > > > is totally sketchy. I think we probably need another client under
> > > clients/
> > > > that exposes administrative functionality. We will need this just to
> > > > properly test the new apis, I suspect. We should figure out that API.
> > > >
> > >
> > > We were talking about that here
> > > https://issues.apache.org/jira/browse/KAFKA-1774 and wrote it in java
> > > https://reviews.apache.org/r/29301/diff/7/?page=4#75 so we could do
> > > something like that, sure.
> > >
> > >
> > > >
> > > > 11. The other information that would be really useful to get would be
> > > > information about partitions--how much data is in the partition, what
> > are
> > > > the segment offsets, what is the log-end offset (i.e. last offset),
> > what
> > > is
> > > > the compaction point, etc. I think that done right this would be the
> > > > successor to the very awkward OffsetRequest we have today.
> > > >
> > >
> > > yes!
> > >
> > >
> > > >
> > > > -Jay
> > > >
> > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly>
> > > wrote:
> > > >
> > > > > Hi, created a KIP
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > >
> > > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> > > > >
> > > > > /*******************************************
> > > > >  Joe Stein
> > > > >  Founder, Principal Consultant
> > > > >  Big Data Open Source Security LLC
> > > > >  http://www.stealth.ly
> > > > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop
> >
> > > > > ********************************************/
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Joe Stein <jo...@stealth.ly>.
I updated the installation and sample usage for the existing patches on the
KIP site
https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations

There are still a few pending items here.

1) There was already some discussion about using the Broker that is the
Controller here https://issues.apache.org/jira/browse/KAFKA-1772 and we
should elaborate on that more in the thread or agree we are ok with admin
asking for the controller to talk to and then just sending that broker the
admin tasks.

2) I like this idea https://issues.apache.org/jira/browse/KAFKA-1912 but we
can refactor after KAFK-1694 committed, no? I know folks just want to talk
to the broker that is the controller. It may even become useful to have the
controller run on a broker that isn't even a topic broker anymore (small
can of worms I am opening here but it elaborates on Guozhang's hot spot
point.

3) anymore feedback?

- Joe Stein

On Fri, Jan 23, 2015 at 3:15 PM, Guozhang Wang <wa...@gmail.com> wrote:

> A centralized admin operation protocol would be very useful.
>
> One more general comment here is that controller is originally designed to
> only talk to other brokers through ControllerChannel, while the broker
> instance which carries the current controller is agnostic of its existence,
> and use KafkaApis to handle general Kafka requests. Having all admin
> requests redirected to the controller instance will force the broker to be
> aware of its carried controller, and access its internal data for handling
> these requests. Plus with the number of clients out of Kafka's control,
> this may easily cause the controller to be a hot spot in terms of request
> load.
>
>
> On Thu, Jan 22, 2015 at 10:09 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > inline
> >
> > On Thu, Jan 22, 2015 at 11:59 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Joe,
> > >
> > > This is great. A few comments on KIP-4
> > >
> > > 1. This is much needed functionality, but there are a lot of the so
> let's
> > > really think these protocols through. We really want to end up with a
> set
> > > of well thought-out, orthoganol apis. For this reason I think it is
> > really
> > > important to think through the end state even if that includes APIs we
> > > won't implement in the first phase.
> > >
> >
> > ok
> >
> >
> > >
> > > 2. Let's please please please wait until we have switched the server
> over
> > > to the new java protocol definitions. If we add upteen more ad hoc
> scala
> > > objects that is just generating more work for the conversion we know we
> > > have to do.
> > >
> >
> > ok :)
> >
> >
> > >
> > > 3. This proposal introduces a new type of optional parameter. This is
> > > inconsistent with everything else in the protocol where we use -1 or
> some
> > > other marker value. You could argue either way but let's stick with
> that
> > > for consistency. For clients that implemented the protocol in a better
> > way
> > > than our scala code these basic primitives are hard to change.
> > >
> >
> > yes, less confusing, ok.
> >
> >
> > >
> > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest which
> > has
> > > brokers, topics, and partitions. I think we should rename that request
> > > ClusterMetadataRequest (or just MetadataRequest) and include the id of
> > the
> > > controller. Or are there other things we could add here?
> > >
> >
> > We could add broker version to it.
> >
> >
> > >
> > > 5. We have a tendency to try to make a lot of requests that can only go
> > to
> > > particular nodes. This adds a lot of burden for client implementations
> > (it
> > > sounds easy but each discovery can fail in many parts so it ends up
> > being a
> > > full state machine to do right). I think we should consider making
> admin
> > > commands and ideally as many of the other apis as possible available on
> > all
> > > brokers and just redirect to the controller on the broker side. Perhaps
> > > there would be a general way to encapsulate this re-routing behavior.
> > >
> >
> > If we do that then we should also preserve what we have and do both. The
> > client can then decide "do I want to go to any broker and proxy" or just
> > "go to controller and run admin task". Lots of folks have seen
> controllers
> > come under distress because of their producers/consumers. There is ticket
> > too for controller elect and re-elect
> > https://issues.apache.org/jira/browse/KAFKA-1778 so you can force it to
> a
> > broker that has 0 load.
> >
> >
> > >
> > > 6. We should probably normalize the key value pairs used for configs
> > rather
> > > than embedding a new formatting. So two strings rather than one with an
> > > internal equals sign.
> > >
> >
> > ok
> >
> >
> > >
> > > 7. Is the postcondition of these APIs that the command has begun or
> that
> > > the command has been completed? It is a lot more usable if the command
> > has
> > > been completed so you know that if you create a topic and then publish
> to
> > > it you won't get an exception about there being no such topic.
> > >
> >
> > We should define that more. There needs to be some more state there, yes.
> >
> > We should try to cover https://issues.apache.org/jira/browse/KAFKA-1125
> > within what we come up with.
> >
> >
> > >
> > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > metadata
> > > request. Is there a reason to give back topics marked for deletion? I
> > feel
> > > like if we just make the post-condition of the delete command be that
> the
> > > topic is deleted that will get rid of the need for this right? And it
> > will
> > > be much more intuitive.
> > >
> >
> > I will go back and look through it.
> >
> >
> > >
> > > 9. Should we consider batching these requests? We have generally tried
> to
> > > allow multiple operations to be batched. My suspicion is that without
> > this
> > > we will get a lot of code that does something like
> > >    for(topic: adminClient.listTopics())
> > >       adminClient.describeTopic(topic)
> > > this code will work great when you test on 5 topics but not do as well
> if
> > > you have 50k.
> > >
> >
> > So => Input is a list of topics (or none for all) and a batch response
> from
> > the controller (which could be routed through another broker) of the
> entire
> > response? We could introduce a Batch keyword to explicitly show the usage
> > of it.
> >
> >
> > > 10. I think we should also discuss how we want to expose a programmatic
> > JVM
> > > client api for these operations. Currently people rely on AdminUtils
> > which
> > > is totally sketchy. I think we probably need another client under
> > clients/
> > > that exposes administrative functionality. We will need this just to
> > > properly test the new apis, I suspect. We should figure out that API.
> > >
> >
> > We were talking about that here
> > https://issues.apache.org/jira/browse/KAFKA-1774 and wrote it in java
> > https://reviews.apache.org/r/29301/diff/7/?page=4#75 so we could do
> > something like that, sure.
> >
> >
> > >
> > > 11. The other information that would be really useful to get would be
> > > information about partitions--how much data is in the partition, what
> are
> > > the segment offsets, what is the log-end offset (i.e. last offset),
> what
> > is
> > > the compaction point, etc. I think that done right this would be the
> > > successor to the very awkward OffsetRequest we have today.
> > >
> >
> > yes!
> >
> >
> > >
> > > -Jay
> > >
> > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly>
> > wrote:
> > >
> > > > Hi, created a KIP
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > >
> > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> > > >
> > > > /*******************************************
> > > >  Joe Stein
> > > >  Founder, Principal Consultant
> > > >  Big Data Open Source Security LLC
> > > >  http://www.stealth.ly
> > > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > > ********************************************/
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Guozhang Wang <wa...@gmail.com>.
A centralized admin operation protocol would be very useful.

One more general comment here is that controller is originally designed to
only talk to other brokers through ControllerChannel, while the broker
instance which carries the current controller is agnostic of its existence,
and use KafkaApis to handle general Kafka requests. Having all admin
requests redirected to the controller instance will force the broker to be
aware of its carried controller, and access its internal data for handling
these requests. Plus with the number of clients out of Kafka's control,
this may easily cause the controller to be a hot spot in terms of request
load.


On Thu, Jan 22, 2015 at 10:09 PM, Joe Stein <jo...@stealth.ly> wrote:

> inline
>
> On Thu, Jan 22, 2015 at 11:59 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Joe,
> >
> > This is great. A few comments on KIP-4
> >
> > 1. This is much needed functionality, but there are a lot of the so let's
> > really think these protocols through. We really want to end up with a set
> > of well thought-out, orthoganol apis. For this reason I think it is
> really
> > important to think through the end state even if that includes APIs we
> > won't implement in the first phase.
> >
>
> ok
>
>
> >
> > 2. Let's please please please wait until we have switched the server over
> > to the new java protocol definitions. If we add upteen more ad hoc scala
> > objects that is just generating more work for the conversion we know we
> > have to do.
> >
>
> ok :)
>
>
> >
> > 3. This proposal introduces a new type of optional parameter. This is
> > inconsistent with everything else in the protocol where we use -1 or some
> > other marker value. You could argue either way but let's stick with that
> > for consistency. For clients that implemented the protocol in a better
> way
> > than our scala code these basic primitives are hard to change.
> >
>
> yes, less confusing, ok.
>
>
> >
> > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest which
> has
> > brokers, topics, and partitions. I think we should rename that request
> > ClusterMetadataRequest (or just MetadataRequest) and include the id of
> the
> > controller. Or are there other things we could add here?
> >
>
> We could add broker version to it.
>
>
> >
> > 5. We have a tendency to try to make a lot of requests that can only go
> to
> > particular nodes. This adds a lot of burden for client implementations
> (it
> > sounds easy but each discovery can fail in many parts so it ends up
> being a
> > full state machine to do right). I think we should consider making admin
> > commands and ideally as many of the other apis as possible available on
> all
> > brokers and just redirect to the controller on the broker side. Perhaps
> > there would be a general way to encapsulate this re-routing behavior.
> >
>
> If we do that then we should also preserve what we have and do both. The
> client can then decide "do I want to go to any broker and proxy" or just
> "go to controller and run admin task". Lots of folks have seen controllers
> come under distress because of their producers/consumers. There is ticket
> too for controller elect and re-elect
> https://issues.apache.org/jira/browse/KAFKA-1778 so you can force it to a
> broker that has 0 load.
>
>
> >
> > 6. We should probably normalize the key value pairs used for configs
> rather
> > than embedding a new formatting. So two strings rather than one with an
> > internal equals sign.
> >
>
> ok
>
>
> >
> > 7. Is the postcondition of these APIs that the command has begun or that
> > the command has been completed? It is a lot more usable if the command
> has
> > been completed so you know that if you create a topic and then publish to
> > it you won't get an exception about there being no such topic.
> >
>
> We should define that more. There needs to be some more state there, yes.
>
> We should try to cover https://issues.apache.org/jira/browse/KAFKA-1125
> within what we come up with.
>
>
> >
> > 8. Describe topic and list topics duplicate a lot of stuff in the
> metadata
> > request. Is there a reason to give back topics marked for deletion? I
> feel
> > like if we just make the post-condition of the delete command be that the
> > topic is deleted that will get rid of the need for this right? And it
> will
> > be much more intuitive.
> >
>
> I will go back and look through it.
>
>
> >
> > 9. Should we consider batching these requests? We have generally tried to
> > allow multiple operations to be batched. My suspicion is that without
> this
> > we will get a lot of code that does something like
> >    for(topic: adminClient.listTopics())
> >       adminClient.describeTopic(topic)
> > this code will work great when you test on 5 topics but not do as well if
> > you have 50k.
> >
>
> So => Input is a list of topics (or none for all) and a batch response from
> the controller (which could be routed through another broker) of the entire
> response? We could introduce a Batch keyword to explicitly show the usage
> of it.
>
>
> > 10. I think we should also discuss how we want to expose a programmatic
> JVM
> > client api for these operations. Currently people rely on AdminUtils
> which
> > is totally sketchy. I think we probably need another client under
> clients/
> > that exposes administrative functionality. We will need this just to
> > properly test the new apis, I suspect. We should figure out that API.
> >
>
> We were talking about that here
> https://issues.apache.org/jira/browse/KAFKA-1774 and wrote it in java
> https://reviews.apache.org/r/29301/diff/7/?page=4#75 so we could do
> something like that, sure.
>
>
> >
> > 11. The other information that would be really useful to get would be
> > information about partitions--how much data is in the partition, what are
> > the segment offsets, what is the log-end offset (i.e. last offset), what
> is
> > the compaction point, etc. I think that done right this would be the
> > successor to the very awkward OffsetRequest we have today.
> >
>
> yes!
>
>
> >
> > -Jay
> >
> > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly>
> wrote:
> >
> > > Hi, created a KIP
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >
> > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> > >
> > > /*******************************************
> > >  Joe Stein
> > >  Founder, Principal Consultant
> > >  Big Data Open Source Security LLC
> > >  http://www.stealth.ly
> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > ********************************************/
> > >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Joe Stein <jo...@stealth.ly>.
inline

On Thu, Jan 22, 2015 at 11:59 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Joe,
>
> This is great. A few comments on KIP-4
>
> 1. This is much needed functionality, but there are a lot of the so let's
> really think these protocols through. We really want to end up with a set
> of well thought-out, orthoganol apis. For this reason I think it is really
> important to think through the end state even if that includes APIs we
> won't implement in the first phase.
>

ok


>
> 2. Let's please please please wait until we have switched the server over
> to the new java protocol definitions. If we add upteen more ad hoc scala
> objects that is just generating more work for the conversion we know we
> have to do.
>

ok :)


>
> 3. This proposal introduces a new type of optional parameter. This is
> inconsistent with everything else in the protocol where we use -1 or some
> other marker value. You could argue either way but let's stick with that
> for consistency. For clients that implemented the protocol in a better way
> than our scala code these basic primitives are hard to change.
>

yes, less confusing, ok.


>
> 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest which has
> brokers, topics, and partitions. I think we should rename that request
> ClusterMetadataRequest (or just MetadataRequest) and include the id of the
> controller. Or are there other things we could add here?
>

We could add broker version to it.


>
> 5. We have a tendency to try to make a lot of requests that can only go to
> particular nodes. This adds a lot of burden for client implementations (it
> sounds easy but each discovery can fail in many parts so it ends up being a
> full state machine to do right). I think we should consider making admin
> commands and ideally as many of the other apis as possible available on all
> brokers and just redirect to the controller on the broker side. Perhaps
> there would be a general way to encapsulate this re-routing behavior.
>

If we do that then we should also preserve what we have and do both. The
client can then decide "do I want to go to any broker and proxy" or just
"go to controller and run admin task". Lots of folks have seen controllers
come under distress because of their producers/consumers. There is ticket
too for controller elect and re-elect
https://issues.apache.org/jira/browse/KAFKA-1778 so you can force it to a
broker that has 0 load.


>
> 6. We should probably normalize the key value pairs used for configs rather
> than embedding a new formatting. So two strings rather than one with an
> internal equals sign.
>

ok


>
> 7. Is the postcondition of these APIs that the command has begun or that
> the command has been completed? It is a lot more usable if the command has
> been completed so you know that if you create a topic and then publish to
> it you won't get an exception about there being no such topic.
>

We should define that more. There needs to be some more state there, yes.

We should try to cover https://issues.apache.org/jira/browse/KAFKA-1125
within what we come up with.


>
> 8. Describe topic and list topics duplicate a lot of stuff in the metadata
> request. Is there a reason to give back topics marked for deletion? I feel
> like if we just make the post-condition of the delete command be that the
> topic is deleted that will get rid of the need for this right? And it will
> be much more intuitive.
>

I will go back and look through it.


>
> 9. Should we consider batching these requests? We have generally tried to
> allow multiple operations to be batched. My suspicion is that without this
> we will get a lot of code that does something like
>    for(topic: adminClient.listTopics())
>       adminClient.describeTopic(topic)
> this code will work great when you test on 5 topics but not do as well if
> you have 50k.
>

So => Input is a list of topics (or none for all) and a batch response from
the controller (which could be routed through another broker) of the entire
response? We could introduce a Batch keyword to explicitly show the usage
of it.


> 10. I think we should also discuss how we want to expose a programmatic JVM
> client api for these operations. Currently people rely on AdminUtils which
> is totally sketchy. I think we probably need another client under clients/
> that exposes administrative functionality. We will need this just to
> properly test the new apis, I suspect. We should figure out that API.
>

We were talking about that here
https://issues.apache.org/jira/browse/KAFKA-1774 and wrote it in java
https://reviews.apache.org/r/29301/diff/7/?page=4#75 so we could do
something like that, sure.


>
> 11. The other information that would be really useful to get would be
> information about partitions--how much data is in the partition, what are
> the segment offsets, what is the log-end offset (i.e. last offset), what is
> the compaction point, etc. I think that done right this would be the
> successor to the very awkward OffsetRequest we have today.
>

yes!


>
> -Jay
>
> On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > Hi, created a KIP
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >
> > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> >
> > /*******************************************
> >  Joe Stein
> >  Founder, Principal Consultant
> >  Big Data Open Source Security LLC
> >  http://www.stealth.ly
> >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > ********************************************/
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
Yeah I want to second this. For the protocol we should really start by
writing out the end state we want. Then we can figure out how to get there
in small, reasonable steps to avoid boiling the ocean in implementation.

-Jay

On Thu, Mar 12, 2015 at 8:27 AM, Joe Stein <jo...@stealth.ly> wrote:

> << Since we are for the first time defining a bunch of new
> request formats, I feel it is better to think through the its possible
> common use cases and try to incorporate them
>
> Agreed.... providing we are only talking about the fields and not the
> implementation of the functionality.
>
> I worry (only a little) about incorporating fields that are not used
> initially but whole heartily believe doing so will outweigh the
> pre-optimization criticism because of the requirement to version the
> protocol (as you brought up).  We can then use those fields later without
> actually implementing the functionality now.
>
> ~ Joe Stein
> - - - - - - - - - - - - - - - - -
>
>   http://www.stealth.ly
> - - - - - - - - - - - - - - - - -
>
> On Thu, Mar 12, 2015 at 11:08 AM, Guozhang Wang <wa...@gmail.com>
> wrote:
>
> > The reason I want to bring it up sooner than later is that future
> changing
> > a defined request protocol takes quite some effort: we need to bump up
> the
> > version of the request, bump up the ZK path data version, and make sure
> > server can handle old versions as well as new ones both from clients and
> > from ZK, etc. Since we are for the first time defining a bunch of new
> > request formats, I feel it is better to think through the its possible
> > common use cases and try to incorporate them, but I am also fine with
> > creating another KIP if most people feel it drags too long.
> >
> > Guozhang
> >
> > On Thu, Mar 12, 2015 at 7:34 AM, Joe Stein <jo...@stealth.ly> wrote:
> >
> > > Guozhang and Tong, I really do like this idea and where your discussion
> > > will lead as it will be very useful for folks to have. I am really
> > > concerned though that we are scope creeping this KIP.
> > >
> > > Andrii is already working on following up on ~ 14 different items of
> > > feedback in regards to the core motivations/scope of the KIP. He has
> > > uploaded a new patch already and the KIP based on those items and will
> be
> > > responding to this thread about that and for what else still requires
> > > discussion hopefully in the next few hours.
> > >
> > > I want to make sure we are focusing on the open items still requiring
> > > discussion and stabilizing what we have before trying to introducing
> more
> > > new features.
> > >
> > > Perhaps a new KIP can get added for the new features you are talking
> > about
> > > which can reference this and once this is committed that work can begin
> > for
> > > folks that are able to contribute to work on it?
> > >
> > > ~ Joe Stein
> > > - - - - - - - - - - - - - - - - -
> > >
> > >   http://www.stealth.ly
> > > - - - - - - - - - - - - - - - - -
> > >
> > > On Thu, Mar 12, 2015 at 9:51 AM, Tong Li <li...@us.ibm.com> wrote:
> > >
> > > > Guozhang,
> > > >      augmenting topic is fine, but as soon as we start doing that,
> > other
> > > > issues follow, for example, access control, who can access the topic,
> > who
> > > > can grant permissions. how the information (metadata) itself gets
> > > secured.
> > > > Should the information be saved in ZK or a datastore? Will using a
> > > metadata
> > > > file causing long term problems such as file updates/synchronization,
> > > once
> > > > we have this metadata file, more people will want to put more stuff
> in
> > > it.
> > > > how can we control the format? K-V pair not good for large data set.
> > > >     Clearly there is a need for it, I wonder if we can make this
> thing
> > > > plugable and provide a default implementation which allows us try
> > > different
> > > > solutions and also allow people to completely ignore it if they do
> not
> > > want
> > > > to deal with any of these.
> > > >
> > > > Thanks.
> > > >
> > > > Tong Li
> > > > OpenStack & Kafka Community Development
> > > > Building 501/B205
> > > > litong01@us.ibm.com
> > > >
> > > > [image: Inactive hide details for Guozhang Wang ---03/12/2015
> 09:39:50
> > > > AM---Folks, Just want to elaborate a bit more on the
> > create-topi]Guozhang
> > > > Wang ---03/12/2015 09:39:50 AM---Folks, Just want to elaborate a bit
> > more
> > > > on the create-topic metadata and batching
> > > >
> > > > From: Guozhang Wang <wa...@gmail.com>
> > > > To: "dev@kafka.apache.org" <de...@kafka.apache.org>
> > > > Date: 03/12/2015 09:39 AM
> > > > Subject: Re: [DISCUSS] KIP-4 - Command line and centralized
> > > > administrative operations
> > > > ------------------------------
> > > >
> > > >
> > > >
> > > > Folks,
> > > >
> > > > Just want to elaborate a bit more on the create-topic metadata and
> > > batching
> > > > describe-topic based on config / metadata in my previous email as we
> > work
> > > > on KAFKA-1694. The main motivation is to have some sort of topic
> > > management
> > > > mechanisms, which I think is quite important in a multi-tenant /
> cloud
> > > > architecture: today anyone can create topics in a shared Kafka
> cluster,
> > > but
> > > > there is no concept or "ownership" of topics that are created by
> > > different
> > > > users. For example, at LinkedIn we basically distinguish topic owners
> > via
> > > > some casual topic name prefix, which is a bit awkward and does not
> fly
> > as
> > > > we scale our customers. It would be great to use describe-topics such
> > as:
> > > >
> > > > Describe all topics that is created by me.
> > > >
> > > > Describe all topics whose retention time is overriden to X.
> > > >
> > > > Describe all topics whose writable group include user Y (this is
> > related
> > > to
> > > > authorization), etc..
> > > >
> > > > One possible way to achieve this is to add a metadata file in the
> > > > create-topic request, whose value will also be written ZK as we
> create
> > > the
> > > > topic; then describe-topics can choose to batch topics based on 1)
> name
> > > > regex, 2) config K-V matching, 3) metadata regex, etc.
> > > >
> > > > Thoughts?
> > > >
> > > > Guozhang
> > > >
> > > > On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Thanks for the updated wiki. A few comments below:
> > > > >
> > > > > 1. Error description in response: I think if some errorCode could
> > > > indicate
> > > > > several different error cases then we should really change it to
> > > multiple
> > > > > codes. In general the errorCode itself would be precise and
> > sufficient
> > > > for
> > > > > describing the server side errors.
> > > > >
> > > > > 2. Describe topic request: it would be great to go beyond just
> > batching
> > > > on
> > > > > topic name regex for this request. For example, a very common use
> > case
> > > of
> > > > > the topic command is to list all topics whose config A's value is
> B.
> > > With
> > > > > topic name regex then we have to first retrieve __all__ topics's
> > > > > description info and then filter at the client end, which will be a
> > > huge
> > > > > burden on ZK.
> > > > >
> > > > > 3. Config K-Vs in create topic: this is related to the previous
> > point;
> > > > > maybe we can add another metadata K-V or just a metadata string
> along
> > > > side
> > > > > with config K-V in create topic like we did for offset commit
> > request.
> > > > This
> > > > > field can be quite useful in storing information like "owner" of
> the
> > > > topic
> > > > > who issue the create command, etc, which is quite important for a
> > > > > multi-tenant setting. Then in the describe topic request we can
> also
> > > > batch
> > > > > on regex of the metadata field.
> > > > >
> > > > > 4. Today all the admin operations are async in the sense that
> command
> > > > will
> > > > > return once it is written in ZK, and that is why we need extra
> > > > verification
> > > > > like testUtil.waitForTopicCreated() / verify partition reassignment
> > > > > request, etc. With admin requests we could add a flag to enable /
> > > disable
> > > > > synchronous requests; when it is turned on, the response will not
> > > return
> > > > > until the request has been completed. And for async requests we can
> > > add a
> > > > > "token" field in the response, and then only need a general "admin
> > > > > verification request" with the given token to check if the async
> > > request
> > > > > has been completed.
> > > > >
> > > > > 5. +1 for extending Metadata request to include controller /
> > > coordinator
> > > > > information, and then we can remove the ConsumerMetadata /
> > > > ClusterMetadata
> > > > > requests.
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com>
> > > wrote:
> > > > >
> > > > >> Thanks for sending that out Joe - I don't think I will be able to
> > make
> > > > >> it today, so if notes can be sent out afterward that would be
> great.
> > > > >>
> > > > >> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> > > > >> > Thanks for sending this out Joe. Looking forward to chatting
> with
> > > > >> everyone :)
> > > > >> >
> > > > >> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <joe.stein@stealth.ly
> >
> > > > wrote:
> > > > >> > > Hey, I just sent out a google hangout invite to all pmc,
> > > committers
> > > > >> and
> > > > >> > > everyone I found working on a KIP. If I missed anyone in the
> > > invite
> > > > >> please
> > > > >> > > let me know and can update it, np.
> > > > >> > >
> > > > >> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we
> can
> > > get
> > > > >> INFRA
> > > > >> > > help to make a google account so we can manage better?
> > > > >> > >
> > > > >> > > To discuss
> > > > >> > >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > >> > > in progress and related JIRA that are interdependent and
> common
> > > > work.
> > > > >> > >
> > > > >> > > ~ Joe Stein
> > > > >> > >
> > > > >> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
> jay.kreps@gmail.com
> > >
> > > > >> wrote:
> > > > >> > >
> > > > >> > >> Let's stay on Google hangouts that will also record and make
> > the
> > > > >> sessions
> > > > >> > >> available on youtube.
> > > > >> > >>
> > > > >> > >> -Jay
> > > > >> > >>
> > > > >> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> > > > >> jholoman@cloudera.com>
> > > > >> > >> wrote:
> > > > >> > >>
> > > > >> > >> > Jay / Joe
> > > > >> > >> >
> > > > >> > >> > We're happy to send out a Webex for this purpose. We could
> > > record
> > > > >> the
> > > > >> > >> > sessions if there is interest and publish them out.
> > > > >> > >> >
> > > > >> > >> > Thanks
> > > > >> > >> >
> > > > >> > >> > Jeff
> > > > >> > >> >
> > > > >> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> > > jay.kreps@gmail.com
> > > > >
> > > > >> wrote:
> > > > >> > >> >
> > > > >> > >> > > Let's try to get the technical hang-ups sorted out,
> > though. I
> > > > >> really
> > > > >> > >> > think
> > > > >> > >> > > there is some benefit to live discussion vs writing. I am
> > > > >> hopeful that
> > > > >> > >> if
> > > > >> > >> > > we post instructions and give ourselves a few attempts we
> > can
> > > > >> get it
> > > > >> > >> > > working.
> > > > >> > >> > >
> > > > >> > >> > > Tuesday at that time would work for me...any objections?
> > > > >> > >> > >
> > > > >> > >> > > -Jay
> > > > >> > >> > >
> > > > >> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> > > > joe.stein@stealth.ly
> > > > >> >
> > > > >> > >> wrote:
> > > > >> > >> > >
> > > > >> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm
> ET /
> > > > 10am
> > > > >> PT
> > > > >> > >> ????
> > > > >> > >> > > >
> > > > >> > >> > > > I don't mind google hangout but there is always some
> > issue
> > > or
> > > > >> > >> whatever
> > > > >> > >> > so
> > > > >> > >> > > > we know the apache irc channel works. We can start
> there
> > > and
> > > > >> see how
> > > > >> > >> it
> > > > >> > >> > > > goes? We can pull transcripts too and associate to
> > tickets
> > > if
> > > > >> need be
> > > > >> > >> > > makes
> > > > >> > >> > > > it helpful for things.
> > > > >> > >> > > >
> > > > >> > >> > > > ~ Joestein
> > > > >> > >> > > >
> > > > >> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> > > > >> jay.kreps@gmail.com>
> > > > >> > >> > wrote:
> > > > >> > >> > > >
> > > > >> > >> > > > > We'd talked about doing a Google Hangout to chat
> about
> > > > this.
> > > > >> What
> > > > >> > >> > about
> > > > >> > >> > > > > generalizing that a little further...I actually think
> > it
> > > > >> would be
> > > > >> > >> > good
> > > > >> > >> > > > for
> > > > >> > >> > > > > everyone spending a reasonable chunk of their week on
> > > Kafka
> > > > >> stuff
> > > > >> > >> to
> > > > >> > >> > > > maybe
> > > > >> > >> > > > > sync up once a week. I think we could use time to
> talk
> > > > >> through
> > > > >> > >> design
> > > > >> > >> > > > > stuff, make sure we are on top of code reviews, talk
> > > > through
> > > > >> any
> > > > >> > >> > tricky
> > > > >> > >> > > > > issues, etc.
> > > > >> > >> > > > >
> > > > >> > >> > > > > We can make it publicly available so that any one can
> > > > follow
> > > > >> along
> > > > >> > >> > who
> > > > >> > >> > > > > likes.
> > > > >> > >> > > > >
> > > > >> > >> > > > > Any interest in doing this? If so I'll try to set it
> up
> > > > >> starting
> > > > >> > >> next
> > > > >> > >> > > > week.
> > > > >> > >> > > > >
> > > > >> > >> > > > > -Jay
> > > > >> > >> > > > >
> > > > >> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> > > > >> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > >> > > > >
> > > > >> > >> > > > > > Hi all,
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > I've updated KIP page, fixed / aligned document
> > > > structure.
> > > > >> Also I
> > > > >> > >> > > added
> > > > >> > >> > > > > > some
> > > > >> > >> > > > > > very initial proposal for AdminClient so we have
> > > > something
> > > > >> to
> > > > >> > >> start
> > > > >> > >> > > > from
> > > > >> > >> > > > > > while
> > > > >> > >> > > > > > discussing the KIP.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > >
> > > > >> > >> > > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > Thanks,
> > > > >> > >> > > > > > Andrii Biletskyi
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> > > > >> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > > Jay,
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > Re error messages: you are right, in most cases
> > > client
> > > > >> will
> > > > >> > >> have
> > > > >> > >> > > > enough
> > > > >> > >> > > > > > > context to show descriptive error message. My
> > concern
> > > > is
> > > > >> that
> > > > >> > >> we
> > > > >> > >> > > will
> > > > >> > >> > > > > > have
> > > > >> > >> > > > > > > to
> > > > >> > >> > > > > > > add lots of new error codes for each possible
> > error.
> > > Of
> > > > >> course,
> > > > >> > >> > we
> > > > >> > >> > > > > could
> > > > >> > >> > > > > > > reuse
> > > > >> > >> > > > > > > some of existing like UknownTopicOrPartitionCode,
> > but
> > > > we
> > > > >> will
> > > > >> > >> > also
> > > > >> > >> > > > need
> > > > >> > >> > > > > > to
> > > > >> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> > > > >> TopicConfigInvalid (both
> > > > >> > >> > for
> > > > >> > >> > > > > topic
> > > > >> > >> > > > > > > name and config, and probably user would like to
> > know
> > > > >> what
> > > > >> > >> > exactly
> > > > >> > >> > > > > > > is wrong in his config),
> InvalidReplicaAssignment,
> > > > >> > >> InternalError
> > > > >> > >> > > > (e.g.
> > > > >> > >> > > > > > > zookeeper failure) etc.
> > > > >> > >> > > > > > > And this is only for TopicCommand, we will also
> > need
> > > to
> > > > >> add
> > > > >> > >> > similar
> > > > >> > >> > > > > stuff
> > > > >> > >> > > > > > > for
> > > > >> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll
> end
> > up
> > > > >> with a
> > > > >> > >> > large
> > > > >> > >> > > > list
> > > > >> > >> > > > > > of
> > > > >> > >> > > > > > > error codes, used only in Admin protocol.
> > > > >> > >> > > > > > > Having said that, I agree my proposal is not
> > > consistent
> > > > >> with
> > > > >> > >> > other
> > > > >> > >> > > > > cases.
> > > > >> > >> > > > > > > Maybe we can find better solution or something
> > > > >> in-between.
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > Re Hangout chat: I think it is a great idea. This
> > way
> > > > we
> > > > >> can
> > > > >> > >> move
> > > > >> > >> > > on
> > > > >> > >> > > > > > > faster.
> > > > >> > >> > > > > > > Let's agree somehow on date/time so people can
> > join.
> > > > >> Will work
> > > > >> > >> > for
> > > > >> > >> > > me
> > > > >> > >> > > > > > this
> > > > >> > >> > > > > > > and
> > > > >> > >> > > > > > > next week almost anytime if agreed in advance.
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > Thanks,
> > > > >> > >> > > > > > > Andrii
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> > > > >> > >> jay.kreps@gmail.com>
> > > > >> > >> > > > > wrote:
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >> Hey Andrii,
> > > > >> > >> > > > > > >>
> > > > >> > >> > > > > > >> Generally we can do good error handling without
> > > > needing
> > > > >> custom
> > > > >> > >> > > > > > server-side
> > > > >> > >> > > > > > >> messages. I.e. generally the client has the
> > context
> > > to
> > > > >> know
> > > > >> > >> that
> > > > >> > >> > > if
> > > > >> > >> > > > it
> > > > >> > >> > > > > > got
> > > > >> > >> > > > > > >> an error that the topic doesn't exist to say
> > "Topic
> > > X
> > > > >> doesn't
> > > > >> > >> > > exist"
> > > > >> > >> > > > > > >> rather
> > > > >> > >> > > > > > >> than "error code 14" (or whatever). Maybe there
> > are
> > > > >> specific
> > > > >> > >> > cases
> > > > >> > >> > > > > where
> > > > >> > >> > > > > > >> this is hard? If we want to add server-side
> error
> > > > >> messages we
> > > > >> > >> > > really
> > > > >> > >> > > > > do
> > > > >> > >> > > > > > >> need to do this in a consistent way across the
> > > > protocol.
> > > > >> > >> > > > > > >>
> > > > >> > >> > > > > > >> I still have a bunch of open questions here from
> > my
> > > > >> previous
> > > > >> > >> > > list. I
> > > > >> > >> > > > > > will
> > > > >> > >> > > > > > >> be out for the next few days for Strata though.
> > > Maybe
> > > > >> we could
> > > > >> > >> > do
> > > > >> > >> > > a
> > > > >> > >> > > > > > Google
> > > > >> > >> > > > > > >> Hangout chat on any open issues some time
> towards
> > > the
> > > > >> end of
> > > > >> > >> > next
> > > > >> > >> > > > week
> > > > >> > >> > > > > > for
> > > > >> > >> > > > > > >> anyone interested in this ticket? I have a
> feeling
> > > > that
> > > > >> might
> > > > >> > >> > > > progress
> > > > >> > >> > > > > > >> things a little faster than email--I think we
> > could
> > > > talk
> > > > >> > >> through
> > > > >> > >> > > > those
> > > > >> > >> > > > > > >> issues I brought up fairly quickly...
> > > > >> > >> > > > > > >>
> > > > >> > >> > > > > > >> -Jay
> > > > >> > >> > > > > > >>
> > > > >> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii
> Biletskyi
> > <
> > > > >> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > > > >> > >> > > > > > >>
> > > > >> > >> > > > > > >> > Hi all,
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >> > I'm trying to address some of the issues which
> > > were
> > > > >> > >> mentioned
> > > > >> > >> > > > > earlier
> > > > >> > >> > > > > > >> about
> > > > >> > >> > > > > > >> > Admin RQ/RP format. One of those was about
> > > batching
> > > > >> > >> > operations.
> > > > >> > >> > > > What
> > > > >> > >> > > > > > if
> > > > >> > >> > > > > > >> we
> > > > >> > >> > > > > > >> > follow TopicCommand approach and let people
> > > specify
> > > > >> > >> topic-name
> > > > >> > >> > > by
> > > > >> > >> > > > > > >> regexp -
> > > > >> > >> > > > > > >> > would that cover most of the use cases?
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >> > Secondly, is what information should we
> > generally
> > > > >> provide in
> > > > >> > >> > > Admin
> > > > >> > >> > > > > > >> > responses.
> > > > >> > >> > > > > > >> > I realize that Admin commands don't imply they
> > > will
> > > > >> be used
> > > > >> > >> > only
> > > > >> > >> > > > in
> > > > >> > >> > > > > > CLI
> > > > >> > >> > > > > > >> > but,
> > > > >> > >> > > > > > >> > it seems to me, CLI is a very important client
> > of
> > > > this
> > > > >> > >> > feature.
> > > > >> > >> > > In
> > > > >> > >> > > > > > this
> > > > >> > >> > > > > > >> > case,
> > > > >> > >> > > > > > >> > seems logical, we would like to provide users
> > with
> > > > >> rich
> > > > >> > >> > > experience
> > > > >> > >> > > > > in
> > > > >> > >> > > > > > >> terms
> > > > >> > >> > > > > > >> > of
> > > > >> > >> > > > > > >> > getting results / errors of the executed
> > commands.
> > > > >> Usually
> > > > >> > >> we
> > > > >> > >> > > > supply
> > > > >> > >> > > > > > >> with
> > > > >> > >> > > > > > >> > responses only errorCode, which looks very
> > > limiting,
> > > > >> in case
> > > > >> > >> > of
> > > > >> > >> > > > CLI
> > > > >> > >> > > > > we
> > > > >> > >> > > > > > >> may
> > > > >> > >> > > > > > >> > want to print human readable error
> description.
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >> > So, taking into account previous item about
> > > > batching,
> > > > >> what
> > > > >> > >> do
> > > > >> > >> > > you
> > > > >> > >> > > > > > think
> > > > >> > >> > > > > > >> > about
> > > > >> > >> > > > > > >> > having smth like:
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >> > ('create' doesn't support regexp)
> > > > >> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions
> > > Replicas
> > > > >> > >> > > > > ReplicaAssignment
> > > > >> > >> > > > > > >> > [Config]
> > > > >> > >> > > > > > >> > CreateTopicResponse => ErrorCode
> > ErrorDescription
> > > > >> > >> > > > > > >> >   ErrorCode => int16
> > > > >> > >> > > > > > >> >   ErrorDescription => string (empty if
> > successful)
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp
> Partitions
> > > > >> > >> > > ReplicaAssignment
> > > > >> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> > > > >> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
> > > > >> ErrorDescription]
> > > > >> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
> > > > >> > >> > > > > > >> >   CommandErrorCode => int16
> > > > >> > >> > > > > > >> >   CommandErrorDescription => string (nonempty
> in
> > > > case
> > > > >> of
> > > > >> > >> fatal
> > > > >> > >> > > > > error,
> > > > >> > >> > > > > > >> e.g.
> > > > >> > >> > > > > > >> > we couldn't get topics by regexp)
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> > > > >> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> > > TopicDescription
> > > > >> > >> ErrorCode
> > > > >> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> > > > >> CommandErrorDescription
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >> > Also, any thoughts about our discussion
> > regarding
> > > > >> re-routing
> > > > >> > >> > > > > facility?
> > > > >> > >> > > > > > >> In
> > > > >> > >> > > > > > >> > my
> > > > >> > >> > > > > > >> > understanding, it is like between augmenting
> > > > >> > >> > > TopicMetadataRequest
> > > > >> > >> > > > > > >> > (to include at least controllerId) and
> > > implementing
> > > > >> new
> > > > >> > >> > generic
> > > > >> > >> > > > > > >> re-routing
> > > > >> > >> > > > > > >> > facility so sending messages to controller
> will
> > be
> > > > >> handled
> > > > >> > >> by
> > > > >> > >> > > it.
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >> > Thanks,
> > > > >> > >> > > > > > >> > Andrii Biletskyi
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii
> > Biletskyi
> > > <
> > > > >> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >> > > @Guozhang:
> > > > >> > >> > > > > > >> > > Thanks for your comments, I've answered some
> > of
> > > > >> those. The
> > > > >> > >> > > main
> > > > >> > >> > > > > > thing
> > > > >> > >> > > > > > >> is
> > > > >> > >> > > > > > >> > > having merged request for
> > > > >> create-alter-delete-describe - I
> > > > >> > >> > > have
> > > > >> > >> > > > > some
> > > > >> > >> > > > > > >> > > concerns about this approach.
> > > > >> > >> > > > > > >> > >
> > > > >> > >> > > > > > >> > > @*Jay*:
> > > > >> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest
> is
> > > also
> > > > >> one of
> > > > >> > >> > the
> > > > >> > >> > > > > > >> concerns.
> > > > >> > >> > > > > > >> > We
> > > > >> > >> > > > > > >> > > can solve it if we implement re-routing
> > > facility.
> > > > >> But I
> > > > >> > >> > agree
> > > > >> > >> > > > with
> > > > >> > >> > > > > > >> > > Guozhang - it will make clients' internals a
> > > > little
> > > > >> bit
> > > > >> > >> > easier
> > > > >> > >> > > > but
> > > > >> > >> > > > > > >> this
> > > > >> > >> > > > > > >> > > seems to be a complex logic to implement and
> > > > >> support then.
> > > > >> > >> > > > > > Especially
> > > > >> > >> > > > > > >> for
> > > > >> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing
> > > later
> > > > >> for
> > > > >> > >> these
> > > > >> > >> > > > > > >> requests).
> > > > >> > >> > > > > > >> > > Also people will tend to avoid this
> re-routing
> > > > >> facility
> > > > >> > >> and
> > > > >> > >> > > hold
> > > > >> > >> > > > > > local
> > > > >> > >> > > > > > >> > > cluster cache to ensure their high-priority
> > > > requests
> > > > >> > >> (which
> > > > >> > >> > > some
> > > > >> > >> > > > > of
> > > > >> > >> > > > > > >> the
> > > > >> > >> > > > > > >> > > admin requests are) not sent to some busy
> > broker
> > > > >> where
> > > > >> > >> they
> > > > >> > >> > > wait
> > > > >> > >> > > > > to
> > > > >> > >> > > > > > be
> > > > >> > >> > > > > > >> > > routed to the correct one.
> > > > >> > >> > > > > > >> > > As pointed out by Jun here (
> > > > >> > >> > > > > > >> > >
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >>
> > > > >> > >> > > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >>
> > > >
> > >
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > > >> > >> > > > > > >> > )
> > > > >> > >> > > > > > >> > > to solve the issue we might introduce a
> > message
> > > > >> type to
> > > > >> > >> get
> > > > >> > >> > > > > cluster
> > > > >> > >> > > > > > >> > state.
> > > > >> > >> > > > > > >> > > But I agree we can just update
> > > > >> TopicMetadataResponse to
> > > > >> > >> > > include
> > > > >> > >> > > > > > >> > > controllerId (and probably smth else).
> > > > >> > >> > > > > > >> > > What are you thougths?
> > > > >> > >> > > > > > >> > >
> > > > >> > >> > > > > > >> > > Thanks,
> > > > >> > >> > > > > > >> > > Andrii
> > > > >> > >> > > > > > >> > >
> > > > >> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang
> > Wang <
> > > > >> > >> > > > > wangguoz@gmail.com>
> > > > >> > >> > > > > > >> > wrote:
> > > > >> > >> > > > > > >> > >
> > > > >> > >> > > > > > >> > >> I think for the topics commands we can
> > actually
> > > > >> merge
> > > > >> > >> > > > > > >> > >> create/alter/delete/describe as one request
> > > type
> > > > >> since
> > > > >> > >> > their
> > > > >> > >> > > > > > formats
> > > > >> > >> > > > > > >> are
> > > > >> > >> > > > > > >> > >> very much similar, and keep list-topics and
> > > > others
> > > > >> like
> > > > >> > >> > > > > > >> > >> partition-reassignment /
> > > > preferred-leader-election
> > > > >> as
> > > > >> > >> > > separate
> > > > >> > >> > > > > > >> request
> > > > >> > >> > > > > > >> > >> types, I also left some other comments on
> the
> > > RB
> > > > (
> > > > >> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> > > > >> > >> > > > > > >> > >>
> > > > >> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps
> <
> > > > >> > >> > > > jay.kreps@gmail.com>
> > > > >> > >> > > > > > >> wrote:
> > > > >> > >> > > > > > >> > >>
> > > > >> > >> > > > > > >> > >> > Yeah I totally agree that we don't want
> to
> > > just
> > > > >> have
> > > > >> > >> one
> > > > >> > >> > > "do
> > > > >> > >> > > > > > admin
> > > > >> > >> > > > > > >> > >> stuff"
> > > > >> > >> > > > > > >> > >> > command that has the union of all
> > parameters.
> > > > >> > >> > > > > > >> > >> >
> > > > >> > >> > > > > > >> > >> > What I am saying is that command line
> tools
> > > are
> > > > >> one
> > > > >> > >> > client
> > > > >> > >> > > of
> > > > >> > >> > > > > the
> > > > >> > >> > > > > > >> > >> > administrative apis, but these will be
> used
> > > in
> > > > a
> > > > >> number
> > > > >> > >> > of
> > > > >> > >> > > > > > >> scenarios
> > > > >> > >> > > > > > >> > so
> > > > >> > >> > > > > > >> > >> > they should make logical sense even in
> the
> > > > >> absence of
> > > > >> > >> the
> > > > >> > >> > > > > command
> > > > >> > >> > > > > > >> line
> > > > >> > >> > > > > > >> > >> > tool. Hence comments like trying to
> clarify
> > > the
> > > > >> > >> > > relationship
> > > > >> > >> > > > > > >> between
> > > > >> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these
> > > kinds
> > > > >> of
> > > > >> > >> things
> > > > >> > >> > > > > really
> > > > >> > >> > > > > > >> need
> > > > >> > >> > > > > > >> > >> to be
> > > > >> > >> > > > > > >> > >> > thought through.
> > > > >> > >> > > > > > >> > >> >
> > > > >> > >> > > > > > >> > >> > Hope that makes sense.
> > > > >> > >> > > > > > >> > >> >
> > > > >> > >> > > > > > >> > >> > -Jay
> > > > >> > >> > > > > > >> > >> >
> > > > >> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
> > > > >> Biletskyi <
> > > > >> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > >> > > > > > >> > >> >
> > > > >> > >> > > > > > >> > >> > > Jay,
> > > > >> > >> > > > > > >> > >> > >
> > > > >> > >> > > > > > >> > >> > > Thanks for answering. You understood
> > > > >> correctly, most
> > > > >> > >> of
> > > > >> > >> > > my
> > > > >> > >> > > > > > >> comments
> > > > >> > >> > > > > > >> > >> were
> > > > >> > >> > > > > > >> > >> > > related to your point 1) - about "well
> > > > >> thought-out"
> > > > >> > >> > apis.
> > > > >> > >> > > > > Also,
> > > > >> > >> > > > > > >> yes,
> > > > >> > >> > > > > > >> > >> as I
> > > > >> > >> > > > > > >> > >> > > understood we would like to introduce a
> > > > single
> > > > >> > >> unified
> > > > >> > >> > > CLI
> > > > >> > >> > > > > tool
> > > > >> > >> > > > > > >> with
> > > > >> > >> > > > > > >> > >> > > centralized server-side request
> handling
> > > for
> > > > >> lots of
> > > > >> > >> > > > existing
> > > > >> > >> > > > > > >> ones
> > > > >> > >> > > > > > >> > >> (incl.
> > > > >> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> > > > >> > >> ReassignPartitions,
> > > > >> > >> > > smth
> > > > >> > >> > > > > > else
> > > > >> > >> > > > > > >> if
> > > > >> > >> > > > > > >> > >> added
> > > > >> > >> > > > > > >> > >> > > in future). In our previous discussion
> (
> > > > >> > >> > > > > > >> > >> > >
> > > > >> https://issues.apache.org/jira/browse/KAFKA-1694
> > > > )
> > > >
> > > > >> > >> > people
> > > > >> > >> > > > > said
> > > > >> > >> > > > > > >> > they'd
> > > > >> > >> > > > > > >> > >> > > rather
> > > > >> > >> > > > > > >> > >> > > have a separate message for each
> command,
> > > so,
> > > > >> yes,
> > > > >> > >> this
> > > > >> > >> > > > way I
> > > > >> > >> > > > > > >> came
> > > > >> > >> > > > > > >> > to
> > > > >> > >> > > > > > >> > >> 1-1
> > > > >> > >> > > > > > >> > >> > > mapping between commands in the tool
> and
> > > > >> protocol
> > > > >> > >> > > > additions.
> > > > >> > >> > > &g t; > > But
> > > >
> > > > >> > >> > > > > > >> I
> > > > >> > >> > > > > > >> > >> might
> > > > >> > >> > > > > > >> > >> > be
> > > > >> > >> > > > > > >> > >> > > wrong.
> > > > >> > >> > > > > > >> > >> > > At the end I just try to start
> discussion
> > > how
> > > > >> at
> > > > >> > >> least
> > > > >> > >> > > > > > generally
> > > > >> > >> > > > > > >> > this
> > > > >> > >> > > > > > >> > >> > > protocol should look like.
> > > > >> > >> > > > > > >> > >> > >
> > > > >> > >> > > > > > >> > >> > > Thanks,
> > > > >> > >> > > > > > >> > >> > > Andrii
> > > > >> > >> > > > > > >> > >> > >
> > > > >> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay
> > > Kreps <
> > > > >> > >> > > > > > jay.kreps@gmail.com
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >> > >> wrote:
> > > > >> > >> > > > > > >> > >> > >
> > > > >> > >> > > > > > >> > >> > > > Hey Andrii,
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> > >> > > > > > >> > >> > > > To answer your earlier question we
> just
> > > > >> really
> > > > >> > >> can't
> > > > >> > >> > be
> > > > >> > >> > > > > > adding
> > > > >> > >> > > > > > >> any
> > > > >> > >> > > > > > >> > >> more
> > > > >> > >> > > > > > >> > >> > > > scala protocol objects. These things
> > are
> > > > >> super hard
> > > > >> > >> > to
> > > > >> > >> > > > > > maintain
> > > > >> > >> > > > > > >> > >> because
> > > > >> > >> > > > > > >> > >> > > > they hand code the byte parsing and
> > don't
> > > > >> have good
> > > > >> > >> > > > > > versioning
> > > > >> > >> > > > > > >> > >> support.
> > > > >> > >> > > > > > >> > >> > > > Since we are already planning on
> > > converting
> > > > >> we
> > > > >> > >> > > definitely
> > > > >> > >> > > > > > don't
> > > > >> > >> > > > > > >> > >> want to
> > > > >> > >> > > > > > >> > >> > > add
> > > > >> > >> > > > > > >> > >> > > > a ton more of these--they are total
> > tech
> > > > >> debt.
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> > >> > > > > > >> > >> > > > What does it mean that the changes
> are
> > > > >> isolated
> > > > >> > >> from
> > > > >> > >> > > the
> > > > >> > >> > > > > > >> current
> > > > >> > >> > > > > > >> > >> code
> > > > >> > >> > > > > > >> > >> > > base?
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> > >> > > > > > >> > >> > > > I actually didn't understand the
> > > remaining
> > > > >> > >> comments,
> > > > >> > >> > > > which
> > > > >> > >> > > > > of
> > > > >> > >> > > > > > >> the
> > > > >> > >> > > > > > >> > >> > points
> > > > >> > >> > > > > > >> > >> > > > are you responding to?
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> > >> > > > > > >> > >> > > > Maybe one sticking point here is that
> > it
> > > > >> seems like
> > > > >> > >> > you
> > > > >> > >> > > > > want
> > > > >> > >> > > > > > to
> > > > >> > >> > > > > > >> > make
> > > > >> > >> > > > > > >> > >> > some
> > > > >> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1
> > > > mapping
> > > > >> > >> between
> > > > >> > >> > > > > > commands
> > > > >> > >> > > > > > >> you
> > > > >> > >> > > > > > >> > >> > > imagine
> > > > >> > >> > > > > > >> > >> > > > in the tool and protocol additions. I
> > > want
> > > > >> to make
> > > > >> > >> > sure
> > > > >> > >> > > > we
> > > > >> > >> > > > > > >> don't
> > > > >> > >> > > > > > >> > do
> > > > >> > >> > > > > > >> > >> > that.
> > > > >> > >> > > > > > >> > >> > > > The protocol needs to be really
> really
> > > well
> > > > >> thought
> > > > >> > >> > out
> > > > >> > >> > > > > > against
> > > > >> > >> > > > > > >> > many
> > > > >> > >> > > > > > >> > >> > use
> > > > >> > >> > > > > > >> > >> > > > cases so it should make perfect
> logical
> > > > >> sense in
> > > > >> > >> the
> > > > >> > >> > > > > absence
> > > > >> > >> > > > > > of
> > > > >> > >> > > > > > >> > >> knowing
> > > > >> > >> > > > > > >> > >> > > the
> > > > >> > >> > > > > > >> > >> > > > command line tool, right?
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> > >> > > > > > >> > >> > > > -Jay
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM,
> > Andrii
> > > > >> Biletskyi
> > > > >> > >> <
> > > > >> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> > >> > > > > > >> > >> > > > > Hey Jay,
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > > > I would like to continue this
> > > discussion
> > > > >> as it
> > > > >> > >> seem
> > > > >> > >> > > > there
> > > > >> > >> > > > > > is
> > > > >> > >> > > > > > >> no
> > > > >> > >> > > > > > >> > >> > > progress
> > > > >> > >> > > > > > >> > >> > > > > here.
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > > > First of all, could you please
> > explain
> > > > >> what did
> > > > >> > >> you
> > > > >> > >> > > > mean
> > > > >> > >> > > > > in
> > > > >> > >> > > > > > >> 2?
> > > > >> > >> > > > > > >> > How
> > > > >> > >> > > > > > >> > >> > > > exactly
> > > > >> > >> > > > > > >> > >> > > > > are we going to migrate to the new
> > java
> > > > >> protocol
> > > > >> > >> > > > > > definitions.
> > > > >> > >> > > > > > >> > And
> > > > >> > >> > > > > > >> > >> why
> > > > >> > >> > > > > > >> > >> > > > it's
> > > > >> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > > > I agree with you, this feature
> > includes
> > > > >> lots of
> > > > >> > >> > > stuff,
> > > > >> > >> > > > > but
> > > > >> > >> > > > > > >> > >> thankfully
> > > > >> > >> > > > > > >> > >> > > > > almost all changes are isolated
> from
> > > the
> > > > >> current
> > > > >> > >> > code
> > > > >> > >> > > > > base,
> > > > >> > >> > > > > > >> > >> > > > > so the main thing, I think, we need
> > to
> > > > >> agree is
> > > > >> > >> > RQ/RP
> > > > >> > >> > > > > > format.
> > > > >> > >> > > > > > >> > >> > > > > So how can we start discussion
> about
> > > the
> > > > >> concrete
> > > > >> > >> > > > > messages
> > > > >> > >> > > > > > >> > format?
> > > > >> > >> > > > > > >> > >> > > > > Can we take (
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> > >> > > > > > >> > >> > >
> > > > >> > >> > > > > > >> > >> >
> > > > >> > >> > > > > > >> > >>
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >>
> > > > >> > >> > > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >>
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > >
> > > > >> > >> > > > > > >> > >> > > > > )
> > > > >> > >> > > > > > >> > >> > > > > as starting point?
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > > > We had some doubts earlier whether
> it
> > > > worth
> > > > >> > >> > > introducing
> > > > >> > >> > > > > one
> > > > >> > >> > > > > > >> > >> generic
> > > > >> > >> > > > > > >> > >> > > Admin
> > > > >> > >> > > > > > >> > >> > > > > Request for all commands (
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> https://issues.apache.org/jira/browse/KAFKA-1694
> > > > >> > >> > > > > > >> > >> > > > > )
> > > > >> > >> > > > > > >> > >> > > > > but then everybody agreed it would
> be
> > > > >> better to
> > > > >> > >> > have
> > > > >> > >> > > > > > separate
> > > > >> > >> > > > > > >> > >> message
> > > > >> > >> > > > > > >> > >> > > for
> > > > >> > >> > > > > > >> > >> > > > > each admin command. The Request
> part
> > is
> > > > >> really
> > > > >> > >> > > dictated
> > > > >> > >> > > > > > from
> > > > >> > >> > > > > > >> the
> > > > >> > >> > > > > > >> > >> > > command
> > > > >> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments
> itself,
> > > so
> > > > >> the
> > > > >> > >> > proposed
> > > > >> > >> > > > > > version
> > > > >> > >> > > > > > >> > >> should
> > > > >> > >> > > > > > >> > >> > be
> > > > >> > >> > > > > > >> > >> > > > > fine (let's put aside for now
> remarks
> > > > about
> > > > >> > >> > Optional
> > > > >> > >> > > > > type,
> > > > >> > >> > > > > > >> > >> batching,
> > > > >> > >> > > > > > >> > >> > > > > configs normalization - I agree
> with
> > > all
> > > > of
> > > > >> > >> them).
> > > > >> > >> > > > > > >> > >> > > > > So the second part is Response. I
> see
> > > > >> there are
> > > > >> > >> two
> > > > >> > >> > > > cases
> > > > >> > >> > > > > > >> here.
> > > > >> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> > > Create/Alter/... ;
> > > > >> b)
> > > > >> > >> "Get"
> > > > >> > >> > > > > > requests -
> > > > >> > >> > > > > > >> > >> > > > > List/Describe...
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > > > a) should only hold request result
> > > > >> (regardless
> > > > >> > >> what
> > > > >> > >> > > we
> > > > >> > >> > > > > > decide
> > > > >> > >> > > > > > >> > >> about
> > > > >> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
> > > > execution).
> > > > >> > >> > > > > > >> > >> > > > > Usually we provide error code in
> > > response
> > > > >> but
> > > > >> > >> since
> > > > >> > >> > > we
> > > > >> > >> > > > > will
> > > > >> > >> > > > > > >> use
> > > > >> > >> > > > > > >> > >> this
> > > > >> > >> > > > > > >> > >> > in
> > > > >> > >> > > > > > >> > >> > > > > interactive shell we need some
> human
> > > > >> readable
> > > > >> > >> error
> > > > >> > >> > > > > > >> description
> > > > >> > >> > > > > > >> > -
> > > > >> > >> > > > > > >> > >> so
> > > > >> > >> > > > > > >> > >> > I
> > > > >> > >> > > > > > >> > >> > > > > added errorDesription field where
> you
> > > can
> > > > >> at
> > > > >> > >> least
> > > > >> > >> > > > leave
> > > > >> > >> > > > > > >> > >> > > > > exception.getMessage.
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > > > b) in addition to previous item
> > message
> > > > >> should
> > > > >> > >> hold
> > > > >> > >> > > > > command
> > > > >> > >> > > > > > >> > >> specific
> > > > >> > >> > > > > > >> > >> > > > > response data. We can discuss in
> > detail
> > > > >> each of
> > > > >> > >> > them
> > > > >> > >> > > > but
> > > > >> > >> > > > > > >> let's
> > > > >> > >> > > > > > >> > for
> > > > >> > >> > > > > > >> > >> > now
> > > > >> > >> > > > > > >> > >> > > > > agree about the overall pattern.
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > > > Thanks,
> > > > >> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM,
> Jay
> > > > Kreps
> > > > >> <
> > > > >> > >> > > > > > >> jay.kreps@gmail.com
> > > > >> > >> > > > > > >> > >
> > > > >> > >> > > > > > >> > >> > > wrote:
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > > > > Hey Joe,
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > This is great. A few comments on
> > > KIP-4
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > 1. This is much needed
> > functionality,
> > > > >> but there
> > > > >> > >> > > are a
> > > > >> > >> > > > > lot
> > > > >> > >> > > > > > >> of
> > > > >> > >> > > > > > >> > >> the so
> > > > >> > >> > > > > > >> > >> > > > let's
> > > > >> > >> > > > > > >> > >> > > > > > really think these protocols
> > through.
> > > > We
> > > > >> really
> > > > >> > >> > > want
> > > > >> > >> > > > to
> > > > >> > >> > > > > > >> end up
> > > > >> > >> > > > > > >> > >> > with a
> > > > >> > >> > > > > > >> > >> > > > set
> > > > >> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol
> > apis.
> > > > >> For this
> > > > >> > >> > > > reason I
> > > > >> > >> > > > > > >> think
> > > > >> > >> > > > > > >> > >> it is
> > > > >> > >> > > > > > >> > >> > > > > really
> > > > >> > >> > > > > > >> > >> > > > > > important to think through the
> end
> > > > state
> > > > >> even
> > > > >> > >> if
> > > > >> > >> > > that
> > > > >> > >> > > > > > >> includes
> > > > >> > >> > > > > > >> > >> APIs
> > > > >> > >> > > > > > >> > >> > > we
> > > > >> > >> > > > > > >> > >> > > > > > won't implement in the first
> phase.
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > 2. Let's please please please
> wait
> > > > until
> > > > >> we
> > > > >> > >> have
> > > > >> > >> > > > > switched
> > > > >> > >> > > > > > >> the
> > > > >> > >> > > > > > >> > >> > server
> > > > >> > >> > > > > > >> > >> > > > over
> > > > >> > >> > > > > > >> > >> > > > > > to the new java protocol
> > definitions.
> > > > If
> > > > >> we add
> > > > >> > >> > > > upteen
> > > > >> > >> > > > > > >> more ad
> > > > >> > >> > > > > > >> > >> hoc
> > > > >> > >> > > > > > >> > >> > > > scala
> > > > >> > >> > > > > > >> > >> > > > > > objects that is just generating
> > more
> > > > >> work for
> > > > >> > >> the
> > > > >> > >> > > > > > >> conversion
> > > > >> > >> > > > > > >> > we
> > > > >> > >> > > > > > >> > >> > know
> > > > >> > >> > > > > > >> > >> > > we
> > > > >> > >> > > > > > >> > >> > > > > > have to do.
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new
> > > type
> > > > of
> > > > >> > >> > optional
> > > > >> > >> > > > > > >> parameter.
> > > > >> > >> > > > > > >> > >> This
> > > > >> > >> > > > > > >> > >> > is
> > > > >> > >> > > > > > >> > >> > > > > > inconsistent with everything else
> > in
> > > > the
> > > > >> > >> protocol
> > > > >> > >> > > > where
> > > > >> > >> > > > > > we
> > > > >> > >> > > > > > >> use
> > > > >> > >> > > > > > >> > >> -1
> > > > >> > >> > > > > > >> > >> > or
> > > > >> > >> > > > > > >> > >> > > > some
> > > > >> > >> > > > > > >> > >> > > > > > other marker value. You could
> argue
> > > > >> either way
> > > > >> > >> > but
> > > > >> > >> > > > > let's
> > > > >> > >> > > > > > >> stick
> > > > >> > >> > > > > > >> > >> with
> > > > >> > >> > > > > > >> > >> > > > that
> > > > >> > >> > > > > > >> > >> > > > > > for consistency. For clients that
> > > > >> implemented
> > > > >> > >> the
> > > > >> > >> > > > > > protocol
> > > > >> > >> > > > > > >> in
> > > > >> > >> > > > > > >> > a
> > > > >> > >> > > > > > >> > >> > > better
> > > > >> > >> > > > > > >> > >> > > > > way
> > > > >> > >> > > > > > >> > >> > > > > > than our scala code these basic
> > > > >> primitives are
> > > > >> > >> > hard
> > > > >> > >> > > > to
> > > > >> > >> > > > > > >> change.
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
> > > > >> duplicate
> > > > >> > >> > > > > > >> > TopicMetadataRequest
> > > > >> > >> > > > > > >> > >> > > which
> > > > >> > >> > > > > > >> > >> > > > > has
> > > > >> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions.
> I
> > > > think
> > > > >> we
> > > > >> > >> > should
> > > > >> > >> > > > > rename
> > > > >> > >> > > > > > >> that
> > > > >> > >> > > > > > >> > >> > > request
> > > > >> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> > > > >> > >> MetadataRequest)
> > > > >> > >> > > and
> > > > >> > >> > > > > > >> include
> > > > >> > >> > > > > > >> > >> the id
> > > > >> > >> > > > > > >> > >> > > of
> > > > >> > >> > > > > > >> > >> > > > > the
> > > > >> > >> > > > > > >> > >> > > > > > controller. Or are there other
> > things
> > > > we
> > > > >> could
> > > > >> > >> > add
> > > > >> > >> > > > > here?
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to
> > make
> > > a
> > > > >> lot of
> > > > >> > >> > > > requests
> > > > >> > >> > > > > > that
> > > > >> > >> > > > > > >> > can
> > > > >> > >> > > > > > >> > >> > only
> > > > >> > >> > > > > > >> > >> > > go
> > > > >> > >> > > > > > >> > >> > > > > to
> > > > >> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot
> > of
> > > > >> burden for
> > > > >> > >> > > > client
> > > > >> > >> > > > > > >> > >> > > implementations
> > > > >> > >> > > > > > >> > >> > > > > (it
> > > > >> > >> > > > > > >> > >> > > > > > sounds easy but each discovery
> can
> > > fail
> > > > >> in many
> > > > >> > >> > > parts
> > > > >> > >> > > > > so
> > > > >> > >> > > > > > it
> > > > >> > >> > > > > > >> > >> ends up
> > > > >> > >> > > > > > >> > >> > > > > being a
> > > > >> > >> > > > > > >> > >> > > > > > full state machine to do right).
> I
> > > > think
> > > > >> we
> > > > >> > >> > should
> > > > >> > >> > > > > > consider
> > > > >> > >> > > > > > >> > >> making
> > > > >> > >> > > > > > >> > >> > > > admin
> > > > >> > >> > > > > > >> > >> > > > > > commands and ideally as many of
> the
> > > > >> other apis
> > > > >> > >> as
> > > > >> > >> > > > > > possible
> > > > >> > >> > > > > > >> > >> > available
> > > > >> > >> > > > > > >> > >> > > on
> > > > >> > >> > > > > > >> > >> > > > > all
> > > > >> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
> > > > >> controller on
> > > > >> > >> > the
> > > > >> > >> > > > > broker
> > > > >> > >> > > > > > >> > side.
> > > > >> > >> > > > > > >> > >> > > Perhaps
> > > > >> > >> > > > > > >> > >> > > > > > there would be a general way to
> > > > >> encapsulate
> > > > >> > >> this
> > > > >> > >> > > > > > re-routing
> > > > >> > >> > > > > > >> > >> > behavior.
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > 6. We should probably normalize
> the
> > > key
> > > > >> value
> > > > >> > >> > pairs
> > > > >> > >> > > > > used
> > > > >> > >> > > > > > >> for
> > > > >> > >> > > > > > >> > >> > configs
> > > > >> > >> > > > > > >> > >> > > > > rather
> > > > >> > >> > > > > > >> > >> > > > > > than embedding a new formatting.
> So
> > > two
> > > > >> strings
> > > > >> > >> > > > rather
> > > > >> > >> > > > > > than
> > > > >> > >> > > > > > >> > one
> > > > >> > >> > > > > > >> > >> > with
> > > > >> > >> > > > > > >> > >> > > an
> > > > >> > >> > > > > > >> > >> > > > > > internal equals sign.
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these
> > APIs
> > > > >> that the
> > > > >> > >> > > > command
> > > > >> > >> > > > > > has
> > > > >> > >> > > > > > >> > >> begun or
> > > > >> > >> > > > > > >> > >> > > > that
> > > > >> > >> > > > > > >> > >> > > > > > the command has been completed?
> It
> > > is a
> > > > >> lot
> > > > >> > >> more
> > > > >> > >> > > > usable
> > > > >> > >> > > > > > if
> > > > >> > >> > > > > > >> the
> > > > >> > >> > > > > > >> > >> > > command
> > > > >> > >> > > > > > >> > >> > > > > has
> > > > >> > >> > > > > > >> > >> > > > > > been completed so you know that
> if
> > > you
> > > > >> create a
> > > > >> > >> > > topic
> > > > >> > >> > > > > and
> > > > >> > >> > > > > > >> then
> > > > >> > >> > > > > > >> > >> > > publish
> > > > >> > >> > > > > > >> > >> > > > to
> > > > >> > >> > > > > > >> > >> > > > > > it you won't get an exception
> about
> > > > >> there being
> > > > >> > >> > no
> > > > >> > >> > > > such
> > > > >> > >> > > > > > >> topic.
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
> > > > >> duplicate a
> > > > >> > >> lot
> > > > >> > >> > > of
> > > > >> > >> > > > > > stuff
> > > > >> > >> > > > > > >> in
> > > > >> > >> > > > > > >> > >> the
> > > > >> > >> > > > > > >> > >> > > > > metadata
> > > > >> > >> > > > > > >> > >> > > > > > request. Is there a reason to
> give
> > > back
> > > > >> topics
> > > > >> > >> > > marked
> > > > >> > >> > > > > for
> > > > >> > >> > > > > > >> > >> > deletion? I
> > > > >> > >> > > > > > >> > >> > > > > feel
> > > > >> > >> > > > > > >> > >> > > > > > like if we just make the
> > > post-condition
> > > > >> of the
> > > > >> > >> > > delete
> > > > >> > >> > > > > > >> command
> > > > >> > >> > > > > > >> > be
> > > > >> > >> > > > > > >> > >> > that
> > > > >> > >> > > > > > >> > >> > > > the
> > > > >> > >> > > > > > >> > >> > > > > > topic is deleted that will get
> rid
> > of
> > > > >> the need
> > > > >> > >> > for
> > > > >> > >> > > > this
> > > > >> > >> > > > > > >> right?
> > > > >> > >> > > > > > >> > >> And
> > > > >> > >> > > > > > >> > >> > it
> > > > >> > >> > > > > > >> > >> > > > > will
> > > > >> > >> > > > > > >> > >> > > > > > be much more intuitive.
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > 9. Should we consider batching
> > these
> > > > >> requests?
> > > > >> > >> We
> > > > >> > >> > > > have
> > > > >> > >> > > > > > >> > generally
> > > > >> > >> > > > > > >> > >> > > tried
> > > > >> > >> > > > > > >> > >> > > > to
> > > > >> > >> > > > > > >> > >> > > > > > allow multiple operations to be
> > > > batched.
> > > > >> My
> > > > >> > >> > > suspicion
> > > > >> > >> > > > > is
> > > > >> > >> > > > > > >> that
> > > > >> > >> > > > > > >> > >> > without
> > > > >> > >> > > > > > >> > >> > > > > this
> > > > >> > >> > > > > > >> > >> > > > > > we will get a lot of code that
> does
> > > > >> something
> > > > >> > >> > like
> > > > >> > >> > > > > > >> > >> > > > > >    for(topic:
> > > adminClient.listTopics())
> > > > >> > >> > > > > > >> > >> > > > > >
> > >  adminClient.describeTopic(topic)
> > > > >> > >> > > > > > >> > >> > > > > > this code will work great when
> you
> > > test
> > > > >> on 5
> > > > >> > >> > topics
> > > > >> > >> > > > but
> > > > >> > >> > > > > > >> not do
> > > > >> > >> > > > > > >> > >> as
> > > > >> > >> > > > > > >> > >> > > well
> > > > >> > >> > > > > > >> > >> > > > if
> > > > >> > >> > > > > > >> > >> > > > > > you have 50k.
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > 10. I think we should also
> discuss
> > > how
> > > > >> we want
> > > > >> > >> to
> > > > >> > >> > > > > expose
> > > > >> > >> > > > > > a
> > > > >> > >> > > > > > >> > >> > > programmatic
> > > > >> > >> > > > > > >> > >> > > > > JVM
> > > > >> > >> > > > > > >> > >> > > > > > client api for these operations.
> > > > >> Currently
> > > > >> > >> people
> > > > >> > >> > > > rely
> > > > >> > >> > > > > on
> > > > >> > >> > > > > > >> > >> > AdminUtils
> > > > >> > >> > > > > > >> > >> > > > > which
> > > > >> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we
> > > probably
> > > > >> need
> > > > >> > >> > > another
> > > > >> > >> > > > > > client
> > > > >> > >> > > > > > >> > >> under
> > > > >> > >> > > > > > >> > >> > > > > clients/
> > > > >> > >> > > > > > >> > >> > > > > > that exposes administrative
> > > > >> functionality. We
> > > > >> > >> > will
> > > > >> > >> > > > need
> > > > >> > >> > > > > > >> this
> > > > >> > >> > > > > > >> > >> just
> > > > >> > >> > > > > > >> > >> > to
> > > > >> > >> > > > > > >> > >> > > > > > properly test the new apis, I
> > > suspect.
> > > > We
> > > > >> > >> should
> > > > >> > >> > > > figure
> > > > >> > >> > > > > > out
> > > > >> > >> > > > > > >> > that
> > > > >> > >> > > > > > >> > >> > API.
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > 11. The other information that
> > would
> > > be
> > > > >> really
> > > > >> > >> > > useful
> > > > >> > >> > > > > to
> > > > >> > >> > > > > > >> get
> > > > >> > >> > > > > > >> > >> would
> > > > >> > >> > > > > > >> > >> > be
> > > > >> > >> > > > > > >> > >> > > > > > information about partitions--how
> > > much
> > > > >> data is
> > > > >> > >> in
> > > > >> > >> > > the
> > > > >> > >> > > > > > >> > partition,
> > > > >> > >> > > > > > >> > >> > what
> > > > >> > >> > > > > > >> > >> > > > are
> > > > >> > >> > > > > > >> > >> > > > > > the segment offsets, what is the
> > > > log-end
> > > > >> offset
> > > > >> > >> > > (i.e.
> > > > >> > >> > > > > > last
> > > > >> > >> > > > > > >> > >> offset),
> > > > >> > >> > > > > > >> > >> > > > what
> > > > >> > >> > > > > > >> > >> > > > > is
> > > > >> > >> > > > > > >> > >> > > > > > the compaction point, etc. I
> think
> > > that
> > > > >> done
> > > > >> > >> > right
> > > > >> > >> > > > this
> > > > >> > >> > > > > > >> would
> > > > >> > >> > > > > > >> > be
> > > > >> > >> > > > > > >> > >> > the
> > > > >> > >> > > > > > >> > >> > > > > > successor to the very awkward
> > > > >> OffsetRequest we
> > > > >> > >> > have
> > > > >> > >> > > > > > today.
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > -Jay
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM,
> > Joe
> > > > >> Stein <
> > > > >> > >> > > > > > >> > >> joe.stein@stealth.ly>
> > > > >> > >> > > > > > >> > >> > > > > wrote:
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> > > > >> > >> > > > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> > >> > > > > > >> > >> > >
> > > > >> > >> > > > > > >> > >> >
> > > > >> > >> > > > > > >> > >>
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >>
> > > > >> > >> > > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > >> > >> > > > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > > JIRA
> > > > >> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > >> > >> > > > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >> > >> > > > > > >
> > > > >> /*******************************************
> > > > >> > >> > > > > > >> > >> > > > > > >  Joe Stein
> > > > >> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> > > > >> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security
> > LLC
> > > > >> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> > > > >> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> > > > >> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> > >> > > > > > >> > >> > > > > > >
> > > > >> ********************************************/
> > > > >> > >> > > > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >> > >> > > > > >
> > > > >> > >> > > > > > >> > >> > > > >
> > > > >> > >> > > > > > >> > >> > > >
> > > > >> > >> > > > > > >> > >> > >
> > > > >> > >> > > > > > >> > >> >
> > > > >> > >> > > > > > >> > >>
> > > > >> > >> > > > > > >> > >>
> > > > >> > >> > > > > > >> > >>
> > > > >> > >> > > > > > >> > >> --
> > > > >> > >> > > > > > >> > >> -- Guozhang
> > > > >> > >> > > > > > >> > >>
> > > > >> > >> > > > > > >> > >
> > > > >> > >> > > > > > >> > >
> > > > >> > >> > > > > > >> >
> > > > >> > >> > > > > > >>
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >> >
> > > > >> > >> > --
> > > > >> > >> > Jeff Holoman
> > > > >> > >> > Systems Engineer
> > > > >> > >> >
> > > > >> > >>
> > > > >>
> > > > >>
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Joe Stein <jo...@stealth.ly>.
<< Since we are for the first time defining a bunch of new
request formats, I feel it is better to think through the its possible
common use cases and try to incorporate them

Agreed.... providing we are only talking about the fields and not the
implementation of the functionality.

I worry (only a little) about incorporating fields that are not used
initially but whole heartily believe doing so will outweigh the
pre-optimization criticism because of the requirement to version the
protocol (as you brought up).  We can then use those fields later without
actually implementing the functionality now.

~ Joe Stein
- - - - - - - - - - - - - - - - -

  http://www.stealth.ly
- - - - - - - - - - - - - - - - -

On Thu, Mar 12, 2015 at 11:08 AM, Guozhang Wang <wa...@gmail.com> wrote:

> The reason I want to bring it up sooner than later is that future changing
> a defined request protocol takes quite some effort: we need to bump up the
> version of the request, bump up the ZK path data version, and make sure
> server can handle old versions as well as new ones both from clients and
> from ZK, etc. Since we are for the first time defining a bunch of new
> request formats, I feel it is better to think through the its possible
> common use cases and try to incorporate them, but I am also fine with
> creating another KIP if most people feel it drags too long.
>
> Guozhang
>
> On Thu, Mar 12, 2015 at 7:34 AM, Joe Stein <jo...@stealth.ly> wrote:
>
> > Guozhang and Tong, I really do like this idea and where your discussion
> > will lead as it will be very useful for folks to have. I am really
> > concerned though that we are scope creeping this KIP.
> >
> > Andrii is already working on following up on ~ 14 different items of
> > feedback in regards to the core motivations/scope of the KIP. He has
> > uploaded a new patch already and the KIP based on those items and will be
> > responding to this thread about that and for what else still requires
> > discussion hopefully in the next few hours.
> >
> > I want to make sure we are focusing on the open items still requiring
> > discussion and stabilizing what we have before trying to introducing more
> > new features.
> >
> > Perhaps a new KIP can get added for the new features you are talking
> about
> > which can reference this and once this is committed that work can begin
> for
> > folks that are able to contribute to work on it?
> >
> > ~ Joe Stein
> > - - - - - - - - - - - - - - - - -
> >
> >   http://www.stealth.ly
> > - - - - - - - - - - - - - - - - -
> >
> > On Thu, Mar 12, 2015 at 9:51 AM, Tong Li <li...@us.ibm.com> wrote:
> >
> > > Guozhang,
> > >      augmenting topic is fine, but as soon as we start doing that,
> other
> > > issues follow, for example, access control, who can access the topic,
> who
> > > can grant permissions. how the information (metadata) itself gets
> > secured.
> > > Should the information be saved in ZK or a datastore? Will using a
> > metadata
> > > file causing long term problems such as file updates/synchronization,
> > once
> > > we have this metadata file, more people will want to put more stuff in
> > it.
> > > how can we control the format? K-V pair not good for large data set.
> > >     Clearly there is a need for it, I wonder if we can make this thing
> > > plugable and provide a default implementation which allows us try
> > different
> > > solutions and also allow people to completely ignore it if they do not
> > want
> > > to deal with any of these.
> > >
> > > Thanks.
> > >
> > > Tong Li
> > > OpenStack & Kafka Community Development
> > > Building 501/B205
> > > litong01@us.ibm.com
> > >
> > > [image: Inactive hide details for Guozhang Wang ---03/12/2015 09:39:50
> > > AM---Folks, Just want to elaborate a bit more on the
> create-topi]Guozhang
> > > Wang ---03/12/2015 09:39:50 AM---Folks, Just want to elaborate a bit
> more
> > > on the create-topic metadata and batching
> > >
> > > From: Guozhang Wang <wa...@gmail.com>
> > > To: "dev@kafka.apache.org" <de...@kafka.apache.org>
> > > Date: 03/12/2015 09:39 AM
> > > Subject: Re: [DISCUSS] KIP-4 - Command line and centralized
> > > administrative operations
> > > ------------------------------
> > >
> > >
> > >
> > > Folks,
> > >
> > > Just want to elaborate a bit more on the create-topic metadata and
> > batching
> > > describe-topic based on config / metadata in my previous email as we
> work
> > > on KAFKA-1694. The main motivation is to have some sort of topic
> > management
> > > mechanisms, which I think is quite important in a multi-tenant / cloud
> > > architecture: today anyone can create topics in a shared Kafka cluster,
> > but
> > > there is no concept or "ownership" of topics that are created by
> > different
> > > users. For example, at LinkedIn we basically distinguish topic owners
> via
> > > some casual topic name prefix, which is a bit awkward and does not fly
> as
> > > we scale our customers. It would be great to use describe-topics such
> as:
> > >
> > > Describe all topics that is created by me.
> > >
> > > Describe all topics whose retention time is overriden to X.
> > >
> > > Describe all topics whose writable group include user Y (this is
> related
> > to
> > > authorization), etc..
> > >
> > > One possible way to achieve this is to add a metadata file in the
> > > create-topic request, whose value will also be written ZK as we create
> > the
> > > topic; then describe-topics can choose to batch topics based on 1) name
> > > regex, 2) config K-V matching, 3) metadata regex, etc.
> > >
> > > Thoughts?
> > >
> > > Guozhang
> > >
> > > On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Thanks for the updated wiki. A few comments below:
> > > >
> > > > 1. Error description in response: I think if some errorCode could
> > > indicate
> > > > several different error cases then we should really change it to
> > multiple
> > > > codes. In general the errorCode itself would be precise and
> sufficient
> > > for
> > > > describing the server side errors.
> > > >
> > > > 2. Describe topic request: it would be great to go beyond just
> batching
> > > on
> > > > topic name regex for this request. For example, a very common use
> case
> > of
> > > > the topic command is to list all topics whose config A's value is B.
> > With
> > > > topic name regex then we have to first retrieve __all__ topics's
> > > > description info and then filter at the client end, which will be a
> > huge
> > > > burden on ZK.
> > > >
> > > > 3. Config K-Vs in create topic: this is related to the previous
> point;
> > > > maybe we can add another metadata K-V or just a metadata string along
> > > side
> > > > with config K-V in create topic like we did for offset commit
> request.
> > > This
> > > > field can be quite useful in storing information like "owner" of the
> > > topic
> > > > who issue the create command, etc, which is quite important for a
> > > > multi-tenant setting. Then in the describe topic request we can also
> > > batch
> > > > on regex of the metadata field.
> > > >
> > > > 4. Today all the admin operations are async in the sense that command
> > > will
> > > > return once it is written in ZK, and that is why we need extra
> > > verification
> > > > like testUtil.waitForTopicCreated() / verify partition reassignment
> > > > request, etc. With admin requests we could add a flag to enable /
> > disable
> > > > synchronous requests; when it is turned on, the response will not
> > return
> > > > until the request has been completed. And for async requests we can
> > add a
> > > > "token" field in the response, and then only need a general "admin
> > > > verification request" with the given token to check if the async
> > request
> > > > has been completed.
> > > >
> > > > 5. +1 for extending Metadata request to include controller /
> > coordinator
> > > > information, and then we can remove the ConsumerMetadata /
> > > ClusterMetadata
> > > > requests.
> > > >
> > > > Guozhang
> > > >
> > > > On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > > >
> > > >> Thanks for sending that out Joe - I don't think I will be able to
> make
> > > >> it today, so if notes can be sent out afterward that would be great.
> > > >>
> > > >> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> > > >> > Thanks for sending this out Joe. Looking forward to chatting with
> > > >> everyone :)
> > > >> >
> > > >> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly>
> > > wrote:
> > > >> > > Hey, I just sent out a google hangout invite to all pmc,
> > committers
> > > >> and
> > > >> > > everyone I found working on a KIP. If I missed anyone in the
> > invite
> > > >> please
> > > >> > > let me know and can update it, np.
> > > >> > >
> > > >> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can
> > get
> > > >> INFRA
> > > >> > > help to make a google account so we can manage better?
> > > >> > >
> > > >> > > To discuss
> > > >> > >
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > >> > > in progress and related JIRA that are interdependent and common
> > > work.
> > > >> > >
> > > >> > > ~ Joe Stein
> > > >> > >
> > > >> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <jay.kreps@gmail.com
> >
> > > >> wrote:
> > > >> > >
> > > >> > >> Let's stay on Google hangouts that will also record and make
> the
> > > >> sessions
> > > >> > >> available on youtube.
> > > >> > >>
> > > >> > >> -Jay
> > > >> > >>
> > > >> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> > > >> jholoman@cloudera.com>
> > > >> > >> wrote:
> > > >> > >>
> > > >> > >> > Jay / Joe
> > > >> > >> >
> > > >> > >> > We're happy to send out a Webex for this purpose. We could
> > record
> > > >> the
> > > >> > >> > sessions if there is interest and publish them out.
> > > >> > >> >
> > > >> > >> > Thanks
> > > >> > >> >
> > > >> > >> > Jeff
> > > >> > >> >
> > > >> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> > jay.kreps@gmail.com
> > > >
> > > >> wrote:
> > > >> > >> >
> > > >> > >> > > Let's try to get the technical hang-ups sorted out,
> though. I
> > > >> really
> > > >> > >> > think
> > > >> > >> > > there is some benefit to live discussion vs writing. I am
> > > >> hopeful that
> > > >> > >> if
> > > >> > >> > > we post instructions and give ourselves a few attempts we
> can
> > > >> get it
> > > >> > >> > > working.
> > > >> > >> > >
> > > >> > >> > > Tuesday at that time would work for me...any objections?
> > > >> > >> > >
> > > >> > >> > > -Jay
> > > >> > >> > >
> > > >> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> > > joe.stein@stealth.ly
> > > >> >
> > > >> > >> wrote:
> > > >> > >> > >
> > > >> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET /
> > > 10am
> > > >> PT
> > > >> > >> ????
> > > >> > >> > > >
> > > >> > >> > > > I don't mind google hangout but there is always some
> issue
> > or
> > > >> > >> whatever
> > > >> > >> > so
> > > >> > >> > > > we know the apache irc channel works. We can start there
> > and
> > > >> see how
> > > >> > >> it
> > > >> > >> > > > goes? We can pull transcripts too and associate to
> tickets
> > if
> > > >> need be
> > > >> > >> > > makes
> > > >> > >> > > > it helpful for things.
> > > >> > >> > > >
> > > >> > >> > > > ~ Joestein
> > > >> > >> > > >
> > > >> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> > > >> jay.kreps@gmail.com>
> > > >> > >> > wrote:
> > > >> > >> > > >
> > > >> > >> > > > > We'd talked about doing a Google Hangout to chat about
> > > this.
> > > >> What
> > > >> > >> > about
> > > >> > >> > > > > generalizing that a little further...I actually think
> it
> > > >> would be
> > > >> > >> > good
> > > >> > >> > > > for
> > > >> > >> > > > > everyone spending a reasonable chunk of their week on
> > Kafka
> > > >> stuff
> > > >> > >> to
> > > >> > >> > > > maybe
> > > >> > >> > > > > sync up once a week. I think we could use time to talk
> > > >> through
> > > >> > >> design
> > > >> > >> > > > > stuff, make sure we are on top of code reviews, talk
> > > through
> > > >> any
> > > >> > >> > tricky
> > > >> > >> > > > > issues, etc.
> > > >> > >> > > > >
> > > >> > >> > > > > We can make it publicly available so that any one can
> > > follow
> > > >> along
> > > >> > >> > who
> > > >> > >> > > > > likes.
> > > >> > >> > > > >
> > > >> > >> > > > > Any interest in doing this? If so I'll try to set it up
> > > >> starting
> > > >> > >> next
> > > >> > >> > > > week.
> > > >> > >> > > > >
> > > >> > >> > > > > -Jay
> > > >> > >> > > > >
> > > >> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> > > >> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > >> > > > >
> > > >> > >> > > > > > Hi all,
> > > >> > >> > > > > >
> > > >> > >> > > > > > I've updated KIP page, fixed / aligned document
> > > structure.
> > > >> Also I
> > > >> > >> > > added
> > > >> > >> > > > > > some
> > > >> > >> > > > > > very initial proposal for AdminClient so we have
> > > something
> > > >> to
> > > >> > >> start
> > > >> > >> > > > from
> > > >> > >> > > > > > while
> > > >> > >> > > > > > discussing the KIP.
> > > >> > >> > > > > >
> > > >> > >> > > > > >
> > > >> > >> > > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > >> > >> > > > > >
> > > >> > >> > > > > > Thanks,
> > > >> > >> > > > > > Andrii Biletskyi
> > > >> > >> > > > > >
> > > >> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> > > >> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > >> > > > > >
> > > >> > >> > > > > > > Jay,
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > Re error messages: you are right, in most cases
> > client
> > > >> will
> > > >> > >> have
> > > >> > >> > > > enough
> > > >> > >> > > > > > > context to show descriptive error message. My
> concern
> > > is
> > > >> that
> > > >> > >> we
> > > >> > >> > > will
> > > >> > >> > > > > > have
> > > >> > >> > > > > > > to
> > > >> > >> > > > > > > add lots of new error codes for each possible
> error.
> > Of
> > > >> course,
> > > >> > >> > we
> > > >> > >> > > > > could
> > > >> > >> > > > > > > reuse
> > > >> > >> > > > > > > some of existing like UknownTopicOrPartitionCode,
> but
> > > we
> > > >> will
> > > >> > >> > also
> > > >> > >> > > > need
> > > >> > >> > > > > > to
> > > >> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> > > >> TopicConfigInvalid (both
> > > >> > >> > for
> > > >> > >> > > > > topic
> > > >> > >> > > > > > > name and config, and probably user would like to
> know
> > > >> what
> > > >> > >> > exactly
> > > >> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
> > > >> > >> InternalError
> > > >> > >> > > > (e.g.
> > > >> > >> > > > > > > zookeeper failure) etc.
> > > >> > >> > > > > > > And this is only for TopicCommand, we will also
> need
> > to
> > > >> add
> > > >> > >> > similar
> > > >> > >> > > > > stuff
> > > >> > >> > > > > > > for
> > > >> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end
> up
> > > >> with a
> > > >> > >> > large
> > > >> > >> > > > list
> > > >> > >> > > > > > of
> > > >> > >> > > > > > > error codes, used only in Admin protocol.
> > > >> > >> > > > > > > Having said that, I agree my proposal is not
> > consistent
> > > >> with
> > > >> > >> > other
> > > >> > >> > > > > cases.
> > > >> > >> > > > > > > Maybe we can find better solution or something
> > > >> in-between.
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > Re Hangout chat: I think it is a great idea. This
> way
> > > we
> > > >> can
> > > >> > >> move
> > > >> > >> > > on
> > > >> > >> > > > > > > faster.
> > > >> > >> > > > > > > Let's agree somehow on date/time so people can
> join.
> > > >> Will work
> > > >> > >> > for
> > > >> > >> > > me
> > > >> > >> > > > > > this
> > > >> > >> > > > > > > and
> > > >> > >> > > > > > > next week almost anytime if agreed in advance.
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > Thanks,
> > > >> > >> > > > > > > Andrii
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> > > >> > >> jay.kreps@gmail.com>
> > > >> > >> > > > > wrote:
> > > >> > >> > > > > > >
> > > >> > >> > > > > > >> Hey Andrii,
> > > >> > >> > > > > > >>
> > > >> > >> > > > > > >> Generally we can do good error handling without
> > > needing
> > > >> custom
> > > >> > >> > > > > > server-side
> > > >> > >> > > > > > >> messages. I.e. generally the client has the
> context
> > to
> > > >> know
> > > >> > >> that
> > > >> > >> > > if
> > > >> > >> > > > it
> > > >> > >> > > > > > got
> > > >> > >> > > > > > >> an error that the topic doesn't exist to say
> "Topic
> > X
> > > >> doesn't
> > > >> > >> > > exist"
> > > >> > >> > > > > > >> rather
> > > >> > >> > > > > > >> than "error code 14" (or whatever). Maybe there
> are
> > > >> specific
> > > >> > >> > cases
> > > >> > >> > > > > where
> > > >> > >> > > > > > >> this is hard? If we want to add server-side error
> > > >> messages we
> > > >> > >> > > really
> > > >> > >> > > > > do
> > > >> > >> > > > > > >> need to do this in a consistent way across the
> > > protocol.
> > > >> > >> > > > > > >>
> > > >> > >> > > > > > >> I still have a bunch of open questions here from
> my
> > > >> previous
> > > >> > >> > > list. I
> > > >> > >> > > > > > will
> > > >> > >> > > > > > >> be out for the next few days for Strata though.
> > Maybe
> > > >> we could
> > > >> > >> > do
> > > >> > >> > > a
> > > >> > >> > > > > > Google
> > > >> > >> > > > > > >> Hangout chat on any open issues some time towards
> > the
> > > >> end of
> > > >> > >> > next
> > > >> > >> > > > week
> > > >> > >> > > > > > for
> > > >> > >> > > > > > >> anyone interested in this ticket? I have a feeling
> > > that
> > > >> might
> > > >> > >> > > > progress
> > > >> > >> > > > > > >> things a little faster than email--I think we
> could
> > > talk
> > > >> > >> through
> > > >> > >> > > > those
> > > >> > >> > > > > > >> issues I brought up fairly quickly...
> > > >> > >> > > > > > >>
> > > >> > >> > > > > > >> -Jay
> > > >> > >> > > > > > >>
> > > >> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi
> <
> > > >> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > > >> > >> > > > > > >>
> > > >> > >> > > > > > >> > Hi all,
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >> > I'm trying to address some of the issues which
> > were
> > > >> > >> mentioned
> > > >> > >> > > > > earlier
> > > >> > >> > > > > > >> about
> > > >> > >> > > > > > >> > Admin RQ/RP format. One of those was about
> > batching
> > > >> > >> > operations.
> > > >> > >> > > > What
> > > >> > >> > > > > > if
> > > >> > >> > > > > > >> we
> > > >> > >> > > > > > >> > follow TopicCommand approach and let people
> > specify
> > > >> > >> topic-name
> > > >> > >> > > by
> > > >> > >> > > > > > >> regexp -
> > > >> > >> > > > > > >> > would that cover most of the use cases?
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >> > Secondly, is what information should we
> generally
> > > >> provide in
> > > >> > >> > > Admin
> > > >> > >> > > > > > >> > responses.
> > > >> > >> > > > > > >> > I realize that Admin commands don't imply they
> > will
> > > >> be used
> > > >> > >> > only
> > > >> > >> > > > in
> > > >> > >> > > > > > CLI
> > > >> > >> > > > > > >> > but,
> > > >> > >> > > > > > >> > it seems to me, CLI is a very important client
> of
> > > this
> > > >> > >> > feature.
> > > >> > >> > > In
> > > >> > >> > > > > > this
> > > >> > >> > > > > > >> > case,
> > > >> > >> > > > > > >> > seems logical, we would like to provide users
> with
> > > >> rich
> > > >> > >> > > experience
> > > >> > >> > > > > in
> > > >> > >> > > > > > >> terms
> > > >> > >> > > > > > >> > of
> > > >> > >> > > > > > >> > getting results / errors of the executed
> commands.
> > > >> Usually
> > > >> > >> we
> > > >> > >> > > > supply
> > > >> > >> > > > > > >> with
> > > >> > >> > > > > > >> > responses only errorCode, which looks very
> > limiting,
> > > >> in case
> > > >> > >> > of
> > > >> > >> > > > CLI
> > > >> > >> > > > > we
> > > >> > >> > > > > > >> may
> > > >> > >> > > > > > >> > want to print human readable error description.
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >> > So, taking into account previous item about
> > > batching,
> > > >> what
> > > >> > >> do
> > > >> > >> > > you
> > > >> > >> > > > > > think
> > > >> > >> > > > > > >> > about
> > > >> > >> > > > > > >> > having smth like:
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >> > ('create' doesn't support regexp)
> > > >> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions
> > Replicas
> > > >> > >> > > > > ReplicaAssignment
> > > >> > >> > > > > > >> > [Config]
> > > >> > >> > > > > > >> > CreateTopicResponse => ErrorCode
> ErrorDescription
> > > >> > >> > > > > > >> >   ErrorCode => int16
> > > >> > >> > > > > > >> >   ErrorDescription => string (empty if
> successful)
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> > > >> > >> > > ReplicaAssignment
> > > >> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> > > >> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
> > > >> ErrorDescription]
> > > >> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
> > > >> > >> > > > > > >> >   CommandErrorCode => int16
> > > >> > >> > > > > > >> >   CommandErrorDescription => string (nonempty in
> > > case
> > > >> of
> > > >> > >> fatal
> > > >> > >> > > > > error,
> > > >> > >> > > > > > >> e.g.
> > > >> > >> > > > > > >> > we couldn't get topics by regexp)
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> > > >> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> > TopicDescription
> > > >> > >> ErrorCode
> > > >> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> > > >> CommandErrorDescription
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >> > Also, any thoughts about our discussion
> regarding
> > > >> re-routing
> > > >> > >> > > > > facility?
> > > >> > >> > > > > > >> In
> > > >> > >> > > > > > >> > my
> > > >> > >> > > > > > >> > understanding, it is like between augmenting
> > > >> > >> > > TopicMetadataRequest
> > > >> > >> > > > > > >> > (to include at least controllerId) and
> > implementing
> > > >> new
> > > >> > >> > generic
> > > >> > >> > > > > > >> re-routing
> > > >> > >> > > > > > >> > facility so sending messages to controller will
> be
> > > >> handled
> > > >> > >> by
> > > >> > >> > > it.
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >> > Thanks,
> > > >> > >> > > > > > >> > Andrii Biletskyi
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii
> Biletskyi
> > <
> > > >> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >> > > @Guozhang:
> > > >> > >> > > > > > >> > > Thanks for your comments, I've answered some
> of
> > > >> those. The
> > > >> > >> > > main
> > > >> > >> > > > > > thing
> > > >> > >> > > > > > >> is
> > > >> > >> > > > > > >> > > having merged request for
> > > >> create-alter-delete-describe - I
> > > >> > >> > > have
> > > >> > >> > > > > some
> > > >> > >> > > > > > >> > > concerns about this approach.
> > > >> > >> > > > > > >> > >
> > > >> > >> > > > > > >> > > @*Jay*:
> > > >> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is
> > also
> > > >> one of
> > > >> > >> > the
> > > >> > >> > > > > > >> concerns.
> > > >> > >> > > > > > >> > We
> > > >> > >> > > > > > >> > > can solve it if we implement re-routing
> > facility.
> > > >> But I
> > > >> > >> > agree
> > > >> > >> > > > with
> > > >> > >> > > > > > >> > > Guozhang - it will make clients' internals a
> > > little
> > > >> bit
> > > >> > >> > easier
> > > >> > >> > > > but
> > > >> > >> > > > > > >> this
> > > >> > >> > > > > > >> > > seems to be a complex logic to implement and
> > > >> support then.
> > > >> > >> > > > > > Especially
> > > >> > >> > > > > > >> for
> > > >> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing
> > later
> > > >> for
> > > >> > >> these
> > > >> > >> > > > > > >> requests).
> > > >> > >> > > > > > >> > > Also people will tend to avoid this re-routing
> > > >> facility
> > > >> > >> and
> > > >> > >> > > hold
> > > >> > >> > > > > > local
> > > >> > >> > > > > > >> > > cluster cache to ensure their high-priority
> > > requests
> > > >> > >> (which
> > > >> > >> > > some
> > > >> > >> > > > > of
> > > >> > >> > > > > > >> the
> > > >> > >> > > > > > >> > > admin requests are) not sent to some busy
> broker
> > > >> where
> > > >> > >> they
> > > >> > >> > > wait
> > > >> > >> > > > > to
> > > >> > >> > > > > > be
> > > >> > >> > > > > > >> > > routed to the correct one.
> > > >> > >> > > > > > >> > > As pointed out by Jun here (
> > > >> > >> > > > > > >> > >
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >>
> > > >> > >> > > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >>
> > >
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > >> > >> > > > > > >> > )
> > > >> > >> > > > > > >> > > to solve the issue we might introduce a
> message
> > > >> type to
> > > >> > >> get
> > > >> > >> > > > > cluster
> > > >> > >> > > > > > >> > state.
> > > >> > >> > > > > > >> > > But I agree we can just update
> > > >> TopicMetadataResponse to
> > > >> > >> > > include
> > > >> > >> > > > > > >> > > controllerId (and probably smth else).
> > > >> > >> > > > > > >> > > What are you thougths?
> > > >> > >> > > > > > >> > >
> > > >> > >> > > > > > >> > > Thanks,
> > > >> > >> > > > > > >> > > Andrii
> > > >> > >> > > > > > >> > >
> > > >> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang
> Wang <
> > > >> > >> > > > > wangguoz@gmail.com>
> > > >> > >> > > > > > >> > wrote:
> > > >> > >> > > > > > >> > >
> > > >> > >> > > > > > >> > >> I think for the topics commands we can
> actually
> > > >> merge
> > > >> > >> > > > > > >> > >> create/alter/delete/describe as one request
> > type
> > > >> since
> > > >> > >> > their
> > > >> > >> > > > > > formats
> > > >> > >> > > > > > >> are
> > > >> > >> > > > > > >> > >> very much similar, and keep list-topics and
> > > others
> > > >> like
> > > >> > >> > > > > > >> > >> partition-reassignment /
> > > preferred-leader-election
> > > >> as
> > > >> > >> > > separate
> > > >> > >> > > > > > >> request
> > > >> > >> > > > > > >> > >> types, I also left some other comments on the
> > RB
> > > (
> > > >> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> > > >> > >> > > > > > >> > >>
> > > >> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> > > >> > >> > > > jay.kreps@gmail.com>
> > > >> > >> > > > > > >> wrote:
> > > >> > >> > > > > > >> > >>
> > > >> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to
> > just
> > > >> have
> > > >> > >> one
> > > >> > >> > > "do
> > > >> > >> > > > > > admin
> > > >> > >> > > > > > >> > >> stuff"
> > > >> > >> > > > > > >> > >> > command that has the union of all
> parameters.
> > > >> > >> > > > > > >> > >> >
> > > >> > >> > > > > > >> > >> > What I am saying is that command line tools
> > are
> > > >> one
> > > >> > >> > client
> > > >> > >> > > of
> > > >> > >> > > > > the
> > > >> > >> > > > > > >> > >> > administrative apis, but these will be used
> > in
> > > a
> > > >> number
> > > >> > >> > of
> > > >> > >> > > > > > >> scenarios
> > > >> > >> > > > > > >> > so
> > > >> > >> > > > > > >> > >> > they should make logical sense even in the
> > > >> absence of
> > > >> > >> the
> > > >> > >> > > > > command
> > > >> > >> > > > > > >> line
> > > >> > >> > > > > > >> > >> > tool. Hence comments like trying to clarify
> > the
> > > >> > >> > > relationship
> > > >> > >> > > > > > >> between
> > > >> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these
> > kinds
> > > >> of
> > > >> > >> things
> > > >> > >> > > > > really
> > > >> > >> > > > > > >> need
> > > >> > >> > > > > > >> > >> to be
> > > >> > >> > > > > > >> > >> > thought through.
> > > >> > >> > > > > > >> > >> >
> > > >> > >> > > > > > >> > >> > Hope that makes sense.
> > > >> > >> > > > > > >> > >> >
> > > >> > >> > > > > > >> > >> > -Jay
> > > >> > >> > > > > > >> > >> >
> > > >> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
> > > >> Biletskyi <
> > > >> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > > >> > >> > > > > > >> > >> >
> > > >> > >> > > > > > >> > >> > > Jay,
> > > >> > >> > > > > > >> > >> > >
> > > >> > >> > > > > > >> > >> > > Thanks for answering. You understood
> > > >> correctly, most
> > > >> > >> of
> > > >> > >> > > my
> > > >> > >> > > > > > >> comments
> > > >> > >> > > > > > >> > >> were
> > > >> > >> > > > > > >> > >> > > related to your point 1) - about "well
> > > >> thought-out"
> > > >> > >> > apis.
> > > >> > >> > > > > Also,
> > > >> > >> > > > > > >> yes,
> > > >> > >> > > > > > >> > >> as I
> > > >> > >> > > > > > >> > >> > > understood we would like to introduce a
> > > single
> > > >> > >> unified
> > > >> > >> > > CLI
> > > >> > >> > > > > tool
> > > >> > >> > > > > > >> with
> > > >> > >> > > > > > >> > >> > > centralized server-side request handling
> > for
> > > >> lots of
> > > >> > >> > > > existing
> > > >> > >> > > > > > >> ones
> > > >> > >> > > > > > >> > >> (incl.
> > > >> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> > > >> > >> ReassignPartitions,
> > > >> > >> > > smth
> > > >> > >> > > > > > else
> > > >> > >> > > > > > >> if
> > > >> > >> > > > > > >> > >> added
> > > >> > >> > > > > > >> > >> > > in future). In our previous discussion (
> > > >> > >> > > > > > >> > >> > >
> > > >> https://issues.apache.org/jira/browse/KAFKA-1694
> > > )
> > >
> > > >> > >> > people
> > > >> > >> > > > > said
> > > >> > >> > > > > > >> > they'd
> > > >> > >> > > > > > >> > >> > > rather
> > > >> > >> > > > > > >> > >> > > have a separate message for each command,
> > so,
> > > >> yes,
> > > >> > >> this
> > > >> > >> > > > way I
> > > >> > >> > > > > > >> came
> > > >> > >> > > > > > >> > to
> > > >> > >> > > > > > >> > >> 1-1
> > > >> > >> > > > > > >> > >> > > mapping between commands in the tool and
> > > >> protocol
> > > >> > >> > > > additions.
> > > >> > >> > > &g t; > > But
> > >
> > > >> > >> > > > > > >> I
> > > >> > >> > > > > > >> > >> might
> > > >> > >> > > > > > >> > >> > be
> > > >> > >> > > > > > >> > >> > > wrong.
> > > >> > >> > > > > > >> > >> > > At the end I just try to start discussion
> > how
> > > >> at
> > > >> > >> least
> > > >> > >> > > > > > generally
> > > >> > >> > > > > > >> > this
> > > >> > >> > > > > > >> > >> > > protocol should look like.
> > > >> > >> > > > > > >> > >> > >
> > > >> > >> > > > > > >> > >> > > Thanks,
> > > >> > >> > > > > > >> > >> > > Andrii
> > > >> > >> > > > > > >> > >> > >
> > > >> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay
> > Kreps <
> > > >> > >> > > > > > jay.kreps@gmail.com
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >> > >> wrote:
> > > >> > >> > > > > > >> > >> > >
> > > >> > >> > > > > > >> > >> > > > Hey Andrii,
> > > >> > >> > > > > > >> > >> > > >
> > > >> > >> > > > > > >> > >> > > > To answer your earlier question we just
> > > >> really
> > > >> > >> can't
> > > >> > >> > be
> > > >> > >> > > > > > adding
> > > >> > >> > > > > > >> any
> > > >> > >> > > > > > >> > >> more
> > > >> > >> > > > > > >> > >> > > > scala protocol objects. These things
> are
> > > >> super hard
> > > >> > >> > to
> > > >> > >> > > > > > maintain
> > > >> > >> > > > > > >> > >> because
> > > >> > >> > > > > > >> > >> > > > they hand code the byte parsing and
> don't
> > > >> have good
> > > >> > >> > > > > > versioning
> > > >> > >> > > > > > >> > >> support.
> > > >> > >> > > > > > >> > >> > > > Since we are already planning on
> > converting
> > > >> we
> > > >> > >> > > definitely
> > > >> > >> > > > > > don't
> > > >> > >> > > > > > >> > >> want to
> > > >> > >> > > > > > >> > >> > > add
> > > >> > >> > > > > > >> > >> > > > a ton more of these--they are total
> tech
> > > >> debt.
> > > >> > >> > > > > > >> > >> > > >
> > > >> > >> > > > > > >> > >> > > > What does it mean that the changes are
> > > >> isolated
> > > >> > >> from
> > > >> > >> > > the
> > > >> > >> > > > > > >> current
> > > >> > >> > > > > > >> > >> code
> > > >> > >> > > > > > >> > >> > > base?
> > > >> > >> > > > > > >> > >> > > >
> > > >> > >> > > > > > >> > >> > > > I actually didn't understand the
> > remaining
> > > >> > >> comments,
> > > >> > >> > > > which
> > > >> > >> > > > > of
> > > >> > >> > > > > > >> the
> > > >> > >> > > > > > >> > >> > points
> > > >> > >> > > > > > >> > >> > > > are you responding to?
> > > >> > >> > > > > > >> > >> > > >
> > > >> > >> > > > > > >> > >> > > > Maybe one sticking point here is that
> it
> > > >> seems like
> > > >> > >> > you
> > > >> > >> > > > > want
> > > >> > >> > > > > > to
> > > >> > >> > > > > > >> > make
> > > >> > >> > > > > > >> > >> > some
> > > >> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1
> > > mapping
> > > >> > >> between
> > > >> > >> > > > > > commands
> > > >> > >> > > > > > >> you
> > > >> > >> > > > > > >> > >> > > imagine
> > > >> > >> > > > > > >> > >> > > > in the tool and protocol additions. I
> > want
> > > >> to make
> > > >> > >> > sure
> > > >> > >> > > > we
> > > >> > >> > > > > > >> don't
> > > >> > >> > > > > > >> > do
> > > >> > >> > > > > > >> > >> > that.
> > > >> > >> > > > > > >> > >> > > > The protocol needs to be really really
> > well
> > > >> thought
> > > >> > >> > out
> > > >> > >> > > > > > against
> > > >> > >> > > > > > >> > many
> > > >> > >> > > > > > >> > >> > use
> > > >> > >> > > > > > >> > >> > > > cases so it should make perfect logical
> > > >> sense in
> > > >> > >> the
> > > >> > >> > > > > absence
> > > >> > >> > > > > > of
> > > >> > >> > > > > > >> > >> knowing
> > > >> > >> > > > > > >> > >> > > the
> > > >> > >> > > > > > >> > >> > > > command line tool, right?
> > > >> > >> > > > > > >> > >> > > >
> > > >> > >> > > > > > >> > >> > > > -Jay
> > > >> > >> > > > > > >> > >> > > >
> > > >> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM,
> Andrii
> > > >> Biletskyi
> > > >> > >> <
> > > >> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > >> > > > > > >> > >> > > >
> > > >> > >> > > > > > >> > >> > > > > Hey Jay,
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > > > I would like to continue this
> > discussion
> > > >> as it
> > > >> > >> seem
> > > >> > >> > > > there
> > > >> > >> > > > > > is
> > > >> > >> > > > > > >> no
> > > >> > >> > > > > > >> > >> > > progress
> > > >> > >> > > > > > >> > >> > > > > here.
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > > > First of all, could you please
> explain
> > > >> what did
> > > >> > >> you
> > > >> > >> > > > mean
> > > >> > >> > > > > in
> > > >> > >> > > > > > >> 2?
> > > >> > >> > > > > > >> > How
> > > >> > >> > > > > > >> > >> > > > exactly
> > > >> > >> > > > > > >> > >> > > > > are we going to migrate to the new
> java
> > > >> protocol
> > > >> > >> > > > > > definitions.
> > > >> > >> > > > > > >> > And
> > > >> > >> > > > > > >> > >> why
> > > >> > >> > > > > > >> > >> > > > it's
> > > >> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > > > I agree with you, this feature
> includes
> > > >> lots of
> > > >> > >> > > stuff,
> > > >> > >> > > > > but
> > > >> > >> > > > > > >> > >> thankfully
> > > >> > >> > > > > > >> > >> > > > > almost all changes are isolated from
> > the
> > > >> current
> > > >> > >> > code
> > > >> > >> > > > > base,
> > > >> > >> > > > > > >> > >> > > > > so the main thing, I think, we need
> to
> > > >> agree is
> > > >> > >> > RQ/RP
> > > >> > >> > > > > > format.
> > > >> > >> > > > > > >> > >> > > > > So how can we start discussion about
> > the
> > > >> concrete
> > > >> > >> > > > > messages
> > > >> > >> > > > > > >> > format?
> > > >> > >> > > > > > >> > >> > > > > Can we take (
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > >
> > > >> > >> > > > > > >> > >> > >
> > > >> > >> > > > > > >> > >> >
> > > >> > >> > > > > > >> > >>
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >>
> > > >> > >> > > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >>
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > >
> > > >> > >> > > > > > >> > >> > > > > )
> > > >> > >> > > > > > >> > >> > > > > as starting point?
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > > > We had some doubts earlier whether it
> > > worth
> > > >> > >> > > introducing
> > > >> > >> > > > > one
> > > >> > >> > > > > > >> > >> generic
> > > >> > >> > > > > > >> > >> > > Admin
> > > >> > >> > > > > > >> > >> > > > > Request for all commands (
> > > >> > >> > > > > > >> > >> > > >
> > > >> https://issues.apache.org/jira/browse/KAFKA-1694
> > > >> > >> > > > > > >> > >> > > > > )
> > > >> > >> > > > > > >> > >> > > > > but then everybody agreed it would be
> > > >> better to
> > > >> > >> > have
> > > >> > >> > > > > > separate
> > > >> > >> > > > > > >> > >> message
> > > >> > >> > > > > > >> > >> > > for
> > > >> > >> > > > > > >> > >> > > > > each admin command. The Request part
> is
> > > >> really
> > > >> > >> > > dictated
> > > >> > >> > > > > > from
> > > >> > >> > > > > > >> the
> > > >> > >> > > > > > >> > >> > > command
> > > >> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself,
> > so
> > > >> the
> > > >> > >> > proposed
> > > >> > >> > > > > > version
> > > >> > >> > > > > > >> > >> should
> > > >> > >> > > > > > >> > >> > be
> > > >> > >> > > > > > >> > >> > > > > fine (let's put aside for now remarks
> > > about
> > > >> > >> > Optional
> > > >> > >> > > > > type,
> > > >> > >> > > > > > >> > >> batching,
> > > >> > >> > > > > > >> > >> > > > > configs normalization - I agree with
> > all
> > > of
> > > >> > >> them).
> > > >> > >> > > > > > >> > >> > > > > So the second part is Response. I see
> > > >> there are
> > > >> > >> two
> > > >> > >> > > > cases
> > > >> > >> > > > > > >> here.
> > > >> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> > Create/Alter/... ;
> > > >> b)
> > > >> > >> "Get"
> > > >> > >> > > > > > requests -
> > > >> > >> > > > > > >> > >> > > > > List/Describe...
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > > > a) should only hold request result
> > > >> (regardless
> > > >> > >> what
> > > >> > >> > > we
> > > >> > >> > > > > > decide
> > > >> > >> > > > > > >> > >> about
> > > >> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
> > > execution).
> > > >> > >> > > > > > >> > >> > > > > Usually we provide error code in
> > response
> > > >> but
> > > >> > >> since
> > > >> > >> > > we
> > > >> > >> > > > > will
> > > >> > >> > > > > > >> use
> > > >> > >> > > > > > >> > >> this
> > > >> > >> > > > > > >> > >> > in
> > > >> > >> > > > > > >> > >> > > > > interactive shell we need some human
> > > >> readable
> > > >> > >> error
> > > >> > >> > > > > > >> description
> > > >> > >> > > > > > >> > -
> > > >> > >> > > > > > >> > >> so
> > > >> > >> > > > > > >> > >> > I
> > > >> > >> > > > > > >> > >> > > > > added errorDesription field where you
> > can
> > > >> at
> > > >> > >> least
> > > >> > >> > > > leave
> > > >> > >> > > > > > >> > >> > > > > exception.getMessage.
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > > > b) in addition to previous item
> message
> > > >> should
> > > >> > >> hold
> > > >> > >> > > > > command
> > > >> > >> > > > > > >> > >> specific
> > > >> > >> > > > > > >> > >> > > > > response data. We can discuss in
> detail
> > > >> each of
> > > >> > >> > them
> > > >> > >> > > > but
> > > >> > >> > > > > > >> let's
> > > >> > >> > > > > > >> > for
> > > >> > >> > > > > > >> > >> > now
> > > >> > >> > > > > > >> > >> > > > > agree about the overall pattern.
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > > > Thanks,
> > > >> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay
> > > Kreps
> > > >> <
> > > >> > >> > > > > > >> jay.kreps@gmail.com
> > > >> > >> > > > > > >> > >
> > > >> > >> > > > > > >> > >> > > wrote:
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > > > > Hey Joe,
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > This is great. A few comments on
> > KIP-4
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > 1. This is much needed
> functionality,
> > > >> but there
> > > >> > >> > > are a
> > > >> > >> > > > > lot
> > > >> > >> > > > > > >> of
> > > >> > >> > > > > > >> > >> the so
> > > >> > >> > > > > > >> > >> > > > let's
> > > >> > >> > > > > > >> > >> > > > > > really think these protocols
> through.
> > > We
> > > >> really
> > > >> > >> > > want
> > > >> > >> > > > to
> > > >> > >> > > > > > >> end up
> > > >> > >> > > > > > >> > >> > with a
> > > >> > >> > > > > > >> > >> > > > set
> > > >> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol
> apis.
> > > >> For this
> > > >> > >> > > > reason I
> > > >> > >> > > > > > >> think
> > > >> > >> > > > > > >> > >> it is
> > > >> > >> > > > > > >> > >> > > > > really
> > > >> > >> > > > > > >> > >> > > > > > important to think through the end
> > > state
> > > >> even
> > > >> > >> if
> > > >> > >> > > that
> > > >> > >> > > > > > >> includes
> > > >> > >> > > > > > >> > >> APIs
> > > >> > >> > > > > > >> > >> > > we
> > > >> > >> > > > > > >> > >> > > > > > won't implement in the first phase.
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait
> > > until
> > > >> we
> > > >> > >> have
> > > >> > >> > > > > switched
> > > >> > >> > > > > > >> the
> > > >> > >> > > > > > >> > >> > server
> > > >> > >> > > > > > >> > >> > > > over
> > > >> > >> > > > > > >> > >> > > > > > to the new java protocol
> definitions.
> > > If
> > > >> we add
> > > >> > >> > > > upteen
> > > >> > >> > > > > > >> more ad
> > > >> > >> > > > > > >> > >> hoc
> > > >> > >> > > > > > >> > >> > > > scala
> > > >> > >> > > > > > >> > >> > > > > > objects that is just generating
> more
> > > >> work for
> > > >> > >> the
> > > >> > >> > > > > > >> conversion
> > > >> > >> > > > > > >> > we
> > > >> > >> > > > > > >> > >> > know
> > > >> > >> > > > > > >> > >> > > we
> > > >> > >> > > > > > >> > >> > > > > > have to do.
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new
> > type
> > > of
> > > >> > >> > optional
> > > >> > >> > > > > > >> parameter.
> > > >> > >> > > > > > >> > >> This
> > > >> > >> > > > > > >> > >> > is
> > > >> > >> > > > > > >> > >> > > > > > inconsistent with everything else
> in
> > > the
> > > >> > >> protocol
> > > >> > >> > > > where
> > > >> > >> > > > > > we
> > > >> > >> > > > > > >> use
> > > >> > >> > > > > > >> > >> -1
> > > >> > >> > > > > > >> > >> > or
> > > >> > >> > > > > > >> > >> > > > some
> > > >> > >> > > > > > >> > >> > > > > > other marker value. You could argue
> > > >> either way
> > > >> > >> > but
> > > >> > >> > > > > let's
> > > >> > >> > > > > > >> stick
> > > >> > >> > > > > > >> > >> with
> > > >> > >> > > > > > >> > >> > > > that
> > > >> > >> > > > > > >> > >> > > > > > for consistency. For clients that
> > > >> implemented
> > > >> > >> the
> > > >> > >> > > > > > protocol
> > > >> > >> > > > > > >> in
> > > >> > >> > > > > > >> > a
> > > >> > >> > > > > > >> > >> > > better
> > > >> > >> > > > > > >> > >> > > > > way
> > > >> > >> > > > > > >> > >> > > > > > than our scala code these basic
> > > >> primitives are
> > > >> > >> > hard
> > > >> > >> > > > to
> > > >> > >> > > > > > >> change.
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
> > > >> duplicate
> > > >> > >> > > > > > >> > TopicMetadataRequest
> > > >> > >> > > > > > >> > >> > > which
> > > >> > >> > > > > > >> > >> > > > > has
> > > >> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I
> > > think
> > > >> we
> > > >> > >> > should
> > > >> > >> > > > > rename
> > > >> > >> > > > > > >> that
> > > >> > >> > > > > > >> > >> > > request
> > > >> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> > > >> > >> MetadataRequest)
> > > >> > >> > > and
> > > >> > >> > > > > > >> include
> > > >> > >> > > > > > >> > >> the id
> > > >> > >> > > > > > >> > >> > > of
> > > >> > >> > > > > > >> > >> > > > > the
> > > >> > >> > > > > > >> > >> > > > > > controller. Or are there other
> things
> > > we
> > > >> could
> > > >> > >> > add
> > > >> > >> > > > > here?
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to
> make
> > a
> > > >> lot of
> > > >> > >> > > > requests
> > > >> > >> > > > > > that
> > > >> > >> > > > > > >> > can
> > > >> > >> > > > > > >> > >> > only
> > > >> > >> > > > > > >> > >> > > go
> > > >> > >> > > > > > >> > >> > > > > to
> > > >> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot
> of
> > > >> burden for
> > > >> > >> > > > client
> > > >> > >> > > > > > >> > >> > > implementations
> > > >> > >> > > > > > >> > >> > > > > (it
> > > >> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can
> > fail
> > > >> in many
> > > >> > >> > > parts
> > > >> > >> > > > > so
> > > >> > >> > > > > > it
> > > >> > >> > > > > > >> > >> ends up
> > > >> > >> > > > > > >> > >> > > > > being a
> > > >> > >> > > > > > >> > >> > > > > > full state machine to do right). I
> > > think
> > > >> we
> > > >> > >> > should
> > > >> > >> > > > > > consider
> > > >> > >> > > > > > >> > >> making
> > > >> > >> > > > > > >> > >> > > > admin
> > > >> > >> > > > > > >> > >> > > > > > commands and ideally as many of the
> > > >> other apis
> > > >> > >> as
> > > >> > >> > > > > > possible
> > > >> > >> > > > > > >> > >> > available
> > > >> > >> > > > > > >> > >> > > on
> > > >> > >> > > > > > >> > >> > > > > all
> > > >> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
> > > >> controller on
> > > >> > >> > the
> > > >> > >> > > > > broker
> > > >> > >> > > > > > >> > side.
> > > >> > >> > > > > > >> > >> > > Perhaps
> > > >> > >> > > > > > >> > >> > > > > > there would be a general way to
> > > >> encapsulate
> > > >> > >> this
> > > >> > >> > > > > > re-routing
> > > >> > >> > > > > > >> > >> > behavior.
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > 6. We should probably normalize the
> > key
> > > >> value
> > > >> > >> > pairs
> > > >> > >> > > > > used
> > > >> > >> > > > > > >> for
> > > >> > >> > > > > > >> > >> > configs
> > > >> > >> > > > > > >> > >> > > > > rather
> > > >> > >> > > > > > >> > >> > > > > > than embedding a new formatting. So
> > two
> > > >> strings
> > > >> > >> > > > rather
> > > >> > >> > > > > > than
> > > >> > >> > > > > > >> > one
> > > >> > >> > > > > > >> > >> > with
> > > >> > >> > > > > > >> > >> > > an
> > > >> > >> > > > > > >> > >> > > > > > internal equals sign.
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these
> APIs
> > > >> that the
> > > >> > >> > > > command
> > > >> > >> > > > > > has
> > > >> > >> > > > > > >> > >> begun or
> > > >> > >> > > > > > >> > >> > > > that
> > > >> > >> > > > > > >> > >> > > > > > the command has been completed? It
> > is a
> > > >> lot
> > > >> > >> more
> > > >> > >> > > > usable
> > > >> > >> > > > > > if
> > > >> > >> > > > > > >> the
> > > >> > >> > > > > > >> > >> > > command
> > > >> > >> > > > > > >> > >> > > > > has
> > > >> > >> > > > > > >> > >> > > > > > been completed so you know that if
> > you
> > > >> create a
> > > >> > >> > > topic
> > > >> > >> > > > > and
> > > >> > >> > > > > > >> then
> > > >> > >> > > > > > >> > >> > > publish
> > > >> > >> > > > > > >> > >> > > > to
> > > >> > >> > > > > > >> > >> > > > > > it you won't get an exception about
> > > >> there being
> > > >> > >> > no
> > > >> > >> > > > such
> > > >> > >> > > > > > >> topic.
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
> > > >> duplicate a
> > > >> > >> lot
> > > >> > >> > > of
> > > >> > >> > > > > > stuff
> > > >> > >> > > > > > >> in
> > > >> > >> > > > > > >> > >> the
> > > >> > >> > > > > > >> > >> > > > > metadata
> > > >> > >> > > > > > >> > >> > > > > > request. Is there a reason to give
> > back
> > > >> topics
> > > >> > >> > > marked
> > > >> > >> > > > > for
> > > >> > >> > > > > > >> > >> > deletion? I
> > > >> > >> > > > > > >> > >> > > > > feel
> > > >> > >> > > > > > >> > >> > > > > > like if we just make the
> > post-condition
> > > >> of the
> > > >> > >> > > delete
> > > >> > >> > > > > > >> command
> > > >> > >> > > > > > >> > be
> > > >> > >> > > > > > >> > >> > that
> > > >> > >> > > > > > >> > >> > > > the
> > > >> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid
> of
> > > >> the need
> > > >> > >> > for
> > > >> > >> > > > this
> > > >> > >> > > > > > >> right?
> > > >> > >> > > > > > >> > >> And
> > > >> > >> > > > > > >> > >> > it
> > > >> > >> > > > > > >> > >> > > > > will
> > > >> > >> > > > > > >> > >> > > > > > be much more intuitive.
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > 9. Should we consider batching
> these
> > > >> requests?
> > > >> > >> We
> > > >> > >> > > > have
> > > >> > >> > > > > > >> > generally
> > > >> > >> > > > > > >> > >> > > tried
> > > >> > >> > > > > > >> > >> > > > to
> > > >> > >> > > > > > >> > >> > > > > > allow multiple operations to be
> > > batched.
> > > >> My
> > > >> > >> > > suspicion
> > > >> > >> > > > > is
> > > >> > >> > > > > > >> that
> > > >> > >> > > > > > >> > >> > without
> > > >> > >> > > > > > >> > >> > > > > this
> > > >> > >> > > > > > >> > >> > > > > > we will get a lot of code that does
> > > >> something
> > > >> > >> > like
> > > >> > >> > > > > > >> > >> > > > > >    for(topic:
> > adminClient.listTopics())
> > > >> > >> > > > > > >> > >> > > > > >
> >  adminClient.describeTopic(topic)
> > > >> > >> > > > > > >> > >> > > > > > this code will work great when you
> > test
> > > >> on 5
> > > >> > >> > topics
> > > >> > >> > > > but
> > > >> > >> > > > > > >> not do
> > > >> > >> > > > > > >> > >> as
> > > >> > >> > > > > > >> > >> > > well
> > > >> > >> > > > > > >> > >> > > > if
> > > >> > >> > > > > > >> > >> > > > > > you have 50k.
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss
> > how
> > > >> we want
> > > >> > >> to
> > > >> > >> > > > > expose
> > > >> > >> > > > > > a
> > > >> > >> > > > > > >> > >> > > programmatic
> > > >> > >> > > > > > >> > >> > > > > JVM
> > > >> > >> > > > > > >> > >> > > > > > client api for these operations.
> > > >> Currently
> > > >> > >> people
> > > >> > >> > > > rely
> > > >> > >> > > > > on
> > > >> > >> > > > > > >> > >> > AdminUtils
> > > >> > >> > > > > > >> > >> > > > > which
> > > >> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we
> > probably
> > > >> need
> > > >> > >> > > another
> > > >> > >> > > > > > client
> > > >> > >> > > > > > >> > >> under
> > > >> > >> > > > > > >> > >> > > > > clients/
> > > >> > >> > > > > > >> > >> > > > > > that exposes administrative
> > > >> functionality. We
> > > >> > >> > will
> > > >> > >> > > > need
> > > >> > >> > > > > > >> this
> > > >> > >> > > > > > >> > >> just
> > > >> > >> > > > > > >> > >> > to
> > > >> > >> > > > > > >> > >> > > > > > properly test the new apis, I
> > suspect.
> > > We
> > > >> > >> should
> > > >> > >> > > > figure
> > > >> > >> > > > > > out
> > > >> > >> > > > > > >> > that
> > > >> > >> > > > > > >> > >> > API.
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > 11. The other information that
> would
> > be
> > > >> really
> > > >> > >> > > useful
> > > >> > >> > > > > to
> > > >> > >> > > > > > >> get
> > > >> > >> > > > > > >> > >> would
> > > >> > >> > > > > > >> > >> > be
> > > >> > >> > > > > > >> > >> > > > > > information about partitions--how
> > much
> > > >> data is
> > > >> > >> in
> > > >> > >> > > the
> > > >> > >> > > > > > >> > partition,
> > > >> > >> > > > > > >> > >> > what
> > > >> > >> > > > > > >> > >> > > > are
> > > >> > >> > > > > > >> > >> > > > > > the segment offsets, what is the
> > > log-end
> > > >> offset
> > > >> > >> > > (i.e.
> > > >> > >> > > > > > last
> > > >> > >> > > > > > >> > >> offset),
> > > >> > >> > > > > > >> > >> > > > what
> > > >> > >> > > > > > >> > >> > > > > is
> > > >> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think
> > that
> > > >> done
> > > >> > >> > right
> > > >> > >> > > > this
> > > >> > >> > > > > > >> would
> > > >> > >> > > > > > >> > be
> > > >> > >> > > > > > >> > >> > the
> > > >> > >> > > > > > >> > >> > > > > > successor to the very awkward
> > > >> OffsetRequest we
> > > >> > >> > have
> > > >> > >> > > > > > today.
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > -Jay
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM,
> Joe
> > > >> Stein <
> > > >> > >> > > > > > >> > >> joe.stein@stealth.ly>
> > > >> > >> > > > > > >> > >> > > > > wrote:
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> > > >> > >> > > > > > >> > >> > > > > > >
> > > >> > >> > > > > > >> > >> > > > > > >
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > >
> > > >> > >> > > > > > >> > >> > >
> > > >> > >> > > > > > >> > >> >
> > > >> > >> > > > > > >> > >>
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >>
> > > >> > >> > > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > >> > >> > > > > > >> > >> > > > > > >
> > > >> > >> > > > > > >> > >> > > > > > > JIRA
> > > >> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > >> > >> > > > > > >> > >> > > > > > >
> > > >> > >> > > > > > >> > >> > > > > > >
> > > >> /*******************************************
> > > >> > >> > > > > > >> > >> > > > > > >  Joe Stein
> > > >> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> > > >> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security
> LLC
> > > >> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> > > >> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> > > >> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> > > >> > >> > > > > > >> > >> > > >
> > > >> > >> > > > > > >> > >> > > > > > >
> > > >> ********************************************/
> > > >> > >> > > > > > >> > >> > > > > > >
> > > >> > >> > > > > > >> > >> > > > > >
> > > >> > >> > > > > > >> > >> > > > >
> > > >> > >> > > > > > >> > >> > > >
> > > >> > >> > > > > > >> > >> > >
> > > >> > >> > > > > > >> > >> >
> > > >> > >> > > > > > >> > >>
> > > >> > >> > > > > > >> > >>
> > > >> > >> > > > > > >> > >>
> > > >> > >> > > > > > >> > >> --
> > > >> > >> > > > > > >> > >> -- Guozhang
> > > >> > >> > > > > > >> > >>
> > > >> > >> > > > > > >> > >
> > > >> > >> > > > > > >> > >
> > > >> > >> > > > > > >> >
> > > >> > >> > > > > > >>
> > > >> > >> > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >> >
> > > >> > >> >
> > > >> > >> > --
> > > >> > >> > Jeff Holoman
> > > >> > >> > Systems Engineer
> > > >> > >> >
> > > >> > >>
> > > >>
> > > >>
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Guozhang Wang <wa...@gmail.com>.
The reason I want to bring it up sooner than later is that future changing
a defined request protocol takes quite some effort: we need to bump up the
version of the request, bump up the ZK path data version, and make sure
server can handle old versions as well as new ones both from clients and
from ZK, etc. Since we are for the first time defining a bunch of new
request formats, I feel it is better to think through the its possible
common use cases and try to incorporate them, but I am also fine with
creating another KIP if most people feel it drags too long.

Guozhang

On Thu, Mar 12, 2015 at 7:34 AM, Joe Stein <jo...@stealth.ly> wrote:

> Guozhang and Tong, I really do like this idea and where your discussion
> will lead as it will be very useful for folks to have. I am really
> concerned though that we are scope creeping this KIP.
>
> Andrii is already working on following up on ~ 14 different items of
> feedback in regards to the core motivations/scope of the KIP. He has
> uploaded a new patch already and the KIP based on those items and will be
> responding to this thread about that and for what else still requires
> discussion hopefully in the next few hours.
>
> I want to make sure we are focusing on the open items still requiring
> discussion and stabilizing what we have before trying to introducing more
> new features.
>
> Perhaps a new KIP can get added for the new features you are talking about
> which can reference this and once this is committed that work can begin for
> folks that are able to contribute to work on it?
>
> ~ Joe Stein
> - - - - - - - - - - - - - - - - -
>
>   http://www.stealth.ly
> - - - - - - - - - - - - - - - - -
>
> On Thu, Mar 12, 2015 at 9:51 AM, Tong Li <li...@us.ibm.com> wrote:
>
> > Guozhang,
> >      augmenting topic is fine, but as soon as we start doing that, other
> > issues follow, for example, access control, who can access the topic, who
> > can grant permissions. how the information (metadata) itself gets
> secured.
> > Should the information be saved in ZK or a datastore? Will using a
> metadata
> > file causing long term problems such as file updates/synchronization,
> once
> > we have this metadata file, more people will want to put more stuff in
> it.
> > how can we control the format? K-V pair not good for large data set.
> >     Clearly there is a need for it, I wonder if we can make this thing
> > plugable and provide a default implementation which allows us try
> different
> > solutions and also allow people to completely ignore it if they do not
> want
> > to deal with any of these.
> >
> > Thanks.
> >
> > Tong Li
> > OpenStack & Kafka Community Development
> > Building 501/B205
> > litong01@us.ibm.com
> >
> > [image: Inactive hide details for Guozhang Wang ---03/12/2015 09:39:50
> > AM---Folks, Just want to elaborate a bit more on the create-topi]Guozhang
> > Wang ---03/12/2015 09:39:50 AM---Folks, Just want to elaborate a bit more
> > on the create-topic metadata and batching
> >
> > From: Guozhang Wang <wa...@gmail.com>
> > To: "dev@kafka.apache.org" <de...@kafka.apache.org>
> > Date: 03/12/2015 09:39 AM
> > Subject: Re: [DISCUSS] KIP-4 - Command line and centralized
> > administrative operations
> > ------------------------------
> >
> >
> >
> > Folks,
> >
> > Just want to elaborate a bit more on the create-topic metadata and
> batching
> > describe-topic based on config / metadata in my previous email as we work
> > on KAFKA-1694. The main motivation is to have some sort of topic
> management
> > mechanisms, which I think is quite important in a multi-tenant / cloud
> > architecture: today anyone can create topics in a shared Kafka cluster,
> but
> > there is no concept or "ownership" of topics that are created by
> different
> > users. For example, at LinkedIn we basically distinguish topic owners via
> > some casual topic name prefix, which is a bit awkward and does not fly as
> > we scale our customers. It would be great to use describe-topics such as:
> >
> > Describe all topics that is created by me.
> >
> > Describe all topics whose retention time is overriden to X.
> >
> > Describe all topics whose writable group include user Y (this is related
> to
> > authorization), etc..
> >
> > One possible way to achieve this is to add a metadata file in the
> > create-topic request, whose value will also be written ZK as we create
> the
> > topic; then describe-topics can choose to batch topics based on 1) name
> > regex, 2) config K-V matching, 3) metadata regex, etc.
> >
> > Thoughts?
> >
> > Guozhang
> >
> > On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Thanks for the updated wiki. A few comments below:
> > >
> > > 1. Error description in response: I think if some errorCode could
> > indicate
> > > several different error cases then we should really change it to
> multiple
> > > codes. In general the errorCode itself would be precise and sufficient
> > for
> > > describing the server side errors.
> > >
> > > 2. Describe topic request: it would be great to go beyond just batching
> > on
> > > topic name regex for this request. For example, a very common use case
> of
> > > the topic command is to list all topics whose config A's value is B.
> With
> > > topic name regex then we have to first retrieve __all__ topics's
> > > description info and then filter at the client end, which will be a
> huge
> > > burden on ZK.
> > >
> > > 3. Config K-Vs in create topic: this is related to the previous point;
> > > maybe we can add another metadata K-V or just a metadata string along
> > side
> > > with config K-V in create topic like we did for offset commit request.
> > This
> > > field can be quite useful in storing information like "owner" of the
> > topic
> > > who issue the create command, etc, which is quite important for a
> > > multi-tenant setting. Then in the describe topic request we can also
> > batch
> > > on regex of the metadata field.
> > >
> > > 4. Today all the admin operations are async in the sense that command
> > will
> > > return once it is written in ZK, and that is why we need extra
> > verification
> > > like testUtil.waitForTopicCreated() / verify partition reassignment
> > > request, etc. With admin requests we could add a flag to enable /
> disable
> > > synchronous requests; when it is turned on, the response will not
> return
> > > until the request has been completed. And for async requests we can
> add a
> > > "token" field in the response, and then only need a general "admin
> > > verification request" with the given token to check if the async
> request
> > > has been completed.
> > >
> > > 5. +1 for extending Metadata request to include controller /
> coordinator
> > > information, and then we can remove the ConsumerMetadata /
> > ClusterMetadata
> > > requests.
> > >
> > > Guozhang
> > >
> > > On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com>
> wrote:
> > >
> > >> Thanks for sending that out Joe - I don't think I will be able to make
> > >> it today, so if notes can be sent out afterward that would be great.
> > >>
> > >> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> > >> > Thanks for sending this out Joe. Looking forward to chatting with
> > >> everyone :)
> > >> >
> > >> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly>
> > wrote:
> > >> > > Hey, I just sent out a google hangout invite to all pmc,
> committers
> > >> and
> > >> > > everyone I found working on a KIP. If I missed anyone in the
> invite
> > >> please
> > >> > > let me know and can update it, np.
> > >> > >
> > >> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can
> get
> > >> INFRA
> > >> > > help to make a google account so we can manage better?
> > >> > >
> > >> > > To discuss
> > >> > >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > >> > > in progress and related JIRA that are interdependent and common
> > work.
> > >> > >
> > >> > > ~ Joe Stein
> > >> > >
> > >> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > >> Let's stay on Google hangouts that will also record and make the
> > >> sessions
> > >> > >> available on youtube.
> > >> > >>
> > >> > >> -Jay
> > >> > >>
> > >> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> > >> jholoman@cloudera.com>
> > >> > >> wrote:
> > >> > >>
> > >> > >> > Jay / Joe
> > >> > >> >
> > >> > >> > We're happy to send out a Webex for this purpose. We could
> record
> > >> the
> > >> > >> > sessions if there is interest and publish them out.
> > >> > >> >
> > >> > >> > Thanks
> > >> > >> >
> > >> > >> > Jeff
> > >> > >> >
> > >> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> jay.kreps@gmail.com
> > >
> > >> wrote:
> > >> > >> >
> > >> > >> > > Let's try to get the technical hang-ups sorted out, though. I
> > >> really
> > >> > >> > think
> > >> > >> > > there is some benefit to live discussion vs writing. I am
> > >> hopeful that
> > >> > >> if
> > >> > >> > > we post instructions and give ourselves a few attempts we can
> > >> get it
> > >> > >> > > working.
> > >> > >> > >
> > >> > >> > > Tuesday at that time would work for me...any objections?
> > >> > >> > >
> > >> > >> > > -Jay
> > >> > >> > >
> > >> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> > joe.stein@stealth.ly
> > >> >
> > >> > >> wrote:
> > >> > >> > >
> > >> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET /
> > 10am
> > >> PT
> > >> > >> ????
> > >> > >> > > >
> > >> > >> > > > I don't mind google hangout but there is always some issue
> or
> > >> > >> whatever
> > >> > >> > so
> > >> > >> > > > we know the apache irc channel works. We can start there
> and
> > >> see how
> > >> > >> it
> > >> > >> > > > goes? We can pull transcripts too and associate to tickets
> if
> > >> need be
> > >> > >> > > makes
> > >> > >> > > > it helpful for things.
> > >> > >> > > >
> > >> > >> > > > ~ Joestein
> > >> > >> > > >
> > >> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> > >> jay.kreps@gmail.com>
> > >> > >> > wrote:
> > >> > >> > > >
> > >> > >> > > > > We'd talked about doing a Google Hangout to chat about
> > this.
> > >> What
> > >> > >> > about
> > >> > >> > > > > generalizing that a little further...I actually think it
> > >> would be
> > >> > >> > good
> > >> > >> > > > for
> > >> > >> > > > > everyone spending a reasonable chunk of their week on
> Kafka
> > >> stuff
> > >> > >> to
> > >> > >> > > > maybe
> > >> > >> > > > > sync up once a week. I think we could use time to talk
> > >> through
> > >> > >> design
> > >> > >> > > > > stuff, make sure we are on top of code reviews, talk
> > through
> > >> any
> > >> > >> > tricky
> > >> > >> > > > > issues, etc.
> > >> > >> > > > >
> > >> > >> > > > > We can make it publicly available so that any one can
> > follow
> > >> along
> > >> > >> > who
> > >> > >> > > > > likes.
> > >> > >> > > > >
> > >> > >> > > > > Any interest in doing this? If so I'll try to set it up
> > >> starting
> > >> > >> next
> > >> > >> > > > week.
> > >> > >> > > > >
> > >> > >> > > > > -Jay
> > >> > >> > > > >
> > >> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> > >> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > >> > > > >
> > >> > >> > > > > > Hi all,
> > >> > >> > > > > >
> > >> > >> > > > > > I've updated KIP page, fixed / aligned document
> > structure.
> > >> Also I
> > >> > >> > > added
> > >> > >> > > > > > some
> > >> > >> > > > > > very initial proposal for AdminClient so we have
> > something
> > >> to
> > >> > >> start
> > >> > >> > > > from
> > >> > >> > > > > > while
> > >> > >> > > > > > discussing the KIP.
> > >> > >> > > > > >
> > >> > >> > > > > >
> > >> > >> > > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >> > >> > > > > >
> > >> > >> > > > > > Thanks,
> > >> > >> > > > > > Andrii Biletskyi
> > >> > >> > > > > >
> > >> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> > >> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > >> > > > > >
> > >> > >> > > > > > > Jay,
> > >> > >> > > > > > >
> > >> > >> > > > > > > Re error messages: you are right, in most cases
> client
> > >> will
> > >> > >> have
> > >> > >> > > > enough
> > >> > >> > > > > > > context to show descriptive error message. My concern
> > is
> > >> that
> > >> > >> we
> > >> > >> > > will
> > >> > >> > > > > > have
> > >> > >> > > > > > > to
> > >> > >> > > > > > > add lots of new error codes for each possible error.
> Of
> > >> course,
> > >> > >> > we
> > >> > >> > > > > could
> > >> > >> > > > > > > reuse
> > >> > >> > > > > > > some of existing like UknownTopicOrPartitionCode, but
> > we
> > >> will
> > >> > >> > also
> > >> > >> > > > need
> > >> > >> > > > > > to
> > >> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> > >> TopicConfigInvalid (both
> > >> > >> > for
> > >> > >> > > > > topic
> > >> > >> > > > > > > name and config, and probably user would like to know
> > >> what
> > >> > >> > exactly
> > >> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
> > >> > >> InternalError
> > >> > >> > > > (e.g.
> > >> > >> > > > > > > zookeeper failure) etc.
> > >> > >> > > > > > > And this is only for TopicCommand, we will also need
> to
> > >> add
> > >> > >> > similar
> > >> > >> > > > > stuff
> > >> > >> > > > > > > for
> > >> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up
> > >> with a
> > >> > >> > large
> > >> > >> > > > list
> > >> > >> > > > > > of
> > >> > >> > > > > > > error codes, used only in Admin protocol.
> > >> > >> > > > > > > Having said that, I agree my proposal is not
> consistent
> > >> with
> > >> > >> > other
> > >> > >> > > > > cases.
> > >> > >> > > > > > > Maybe we can find better solution or something
> > >> in-between.
> > >> > >> > > > > > >
> > >> > >> > > > > > > Re Hangout chat: I think it is a great idea. This way
> > we
> > >> can
> > >> > >> move
> > >> > >> > > on
> > >> > >> > > > > > > faster.
> > >> > >> > > > > > > Let's agree somehow on date/time so people can join.
> > >> Will work
> > >> > >> > for
> > >> > >> > > me
> > >> > >> > > > > > this
> > >> > >> > > > > > > and
> > >> > >> > > > > > > next week almost anytime if agreed in advance.
> > >> > >> > > > > > >
> > >> > >> > > > > > > Thanks,
> > >> > >> > > > > > > Andrii
> > >> > >> > > > > > >
> > >> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> > >> > >> jay.kreps@gmail.com>
> > >> > >> > > > > wrote:
> > >> > >> > > > > > >
> > >> > >> > > > > > >> Hey Andrii,
> > >> > >> > > > > > >>
> > >> > >> > > > > > >> Generally we can do good error handling without
> > needing
> > >> custom
> > >> > >> > > > > > server-side
> > >> > >> > > > > > >> messages. I.e. generally the client has the context
> to
> > >> know
> > >> > >> that
> > >> > >> > > if
> > >> > >> > > > it
> > >> > >> > > > > > got
> > >> > >> > > > > > >> an error that the topic doesn't exist to say "Topic
> X
> > >> doesn't
> > >> > >> > > exist"
> > >> > >> > > > > > >> rather
> > >> > >> > > > > > >> than "error code 14" (or whatever). Maybe there are
> > >> specific
> > >> > >> > cases
> > >> > >> > > > > where
> > >> > >> > > > > > >> this is hard? If we want to add server-side error
> > >> messages we
> > >> > >> > > really
> > >> > >> > > > > do
> > >> > >> > > > > > >> need to do this in a consistent way across the
> > protocol.
> > >> > >> > > > > > >>
> > >> > >> > > > > > >> I still have a bunch of open questions here from my
> > >> previous
> > >> > >> > > list. I
> > >> > >> > > > > > will
> > >> > >> > > > > > >> be out for the next few days for Strata though.
> Maybe
> > >> we could
> > >> > >> > do
> > >> > >> > > a
> > >> > >> > > > > > Google
> > >> > >> > > > > > >> Hangout chat on any open issues some time towards
> the
> > >> end of
> > >> > >> > next
> > >> > >> > > > week
> > >> > >> > > > > > for
> > >> > >> > > > > > >> anyone interested in this ticket? I have a feeling
> > that
> > >> might
> > >> > >> > > > progress
> > >> > >> > > > > > >> things a little faster than email--I think we could
> > talk
> > >> > >> through
> > >> > >> > > > those
> > >> > >> > > > > > >> issues I brought up fairly quickly...
> > >> > >> > > > > > >>
> > >> > >> > > > > > >> -Jay
> > >> > >> > > > > > >>
> > >> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> > >> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > >> > >> > > > > > >>
> > >> > >> > > > > > >> > Hi all,
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >> > I'm trying to address some of the issues which
> were
> > >> > >> mentioned
> > >> > >> > > > > earlier
> > >> > >> > > > > > >> about
> > >> > >> > > > > > >> > Admin RQ/RP format. One of those was about
> batching
> > >> > >> > operations.
> > >> > >> > > > What
> > >> > >> > > > > > if
> > >> > >> > > > > > >> we
> > >> > >> > > > > > >> > follow TopicCommand approach and let people
> specify
> > >> > >> topic-name
> > >> > >> > > by
> > >> > >> > > > > > >> regexp -
> > >> > >> > > > > > >> > would that cover most of the use cases?
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >> > Secondly, is what information should we generally
> > >> provide in
> > >> > >> > > Admin
> > >> > >> > > > > > >> > responses.
> > >> > >> > > > > > >> > I realize that Admin commands don't imply they
> will
> > >> be used
> > >> > >> > only
> > >> > >> > > > in
> > >> > >> > > > > > CLI
> > >> > >> > > > > > >> > but,
> > >> > >> > > > > > >> > it seems to me, CLI is a very important client of
> > this
> > >> > >> > feature.
> > >> > >> > > In
> > >> > >> > > > > > this
> > >> > >> > > > > > >> > case,
> > >> > >> > > > > > >> > seems logical, we would like to provide users with
> > >> rich
> > >> > >> > > experience
> > >> > >> > > > > in
> > >> > >> > > > > > >> terms
> > >> > >> > > > > > >> > of
> > >> > >> > > > > > >> > getting results / errors of the executed commands.
> > >> Usually
> > >> > >> we
> > >> > >> > > > supply
> > >> > >> > > > > > >> with
> > >> > >> > > > > > >> > responses only errorCode, which looks very
> limiting,
> > >> in case
> > >> > >> > of
> > >> > >> > > > CLI
> > >> > >> > > > > we
> > >> > >> > > > > > >> may
> > >> > >> > > > > > >> > want to print human readable error description.
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >> > So, taking into account previous item about
> > batching,
> > >> what
> > >> > >> do
> > >> > >> > > you
> > >> > >> > > > > > think
> > >> > >> > > > > > >> > about
> > >> > >> > > > > > >> > having smth like:
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >> > ('create' doesn't support regexp)
> > >> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions
> Replicas
> > >> > >> > > > > ReplicaAssignment
> > >> > >> > > > > > >> > [Config]
> > >> > >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
> > >> > >> > > > > > >> >   ErrorCode => int16
> > >> > >> > > > > > >> >   ErrorDescription => string (empty if successful)
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> > >> > >> > > ReplicaAssignment
> > >> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> > >> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
> > >> ErrorDescription]
> > >> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
> > >> > >> > > > > > >> >   CommandErrorCode => int16
> > >> > >> > > > > > >> >   CommandErrorDescription => string (nonempty in
> > case
> > >> of
> > >> > >> fatal
> > >> > >> > > > > error,
> > >> > >> > > > > > >> e.g.
> > >> > >> > > > > > >> > we couldn't get topics by regexp)
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> > >> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> TopicDescription
> > >> > >> ErrorCode
> > >> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> > >> CommandErrorDescription
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >> > Also, any thoughts about our discussion regarding
> > >> re-routing
> > >> > >> > > > > facility?
> > >> > >> > > > > > >> In
> > >> > >> > > > > > >> > my
> > >> > >> > > > > > >> > understanding, it is like between augmenting
> > >> > >> > > TopicMetadataRequest
> > >> > >> > > > > > >> > (to include at least controllerId) and
> implementing
> > >> new
> > >> > >> > generic
> > >> > >> > > > > > >> re-routing
> > >> > >> > > > > > >> > facility so sending messages to controller will be
> > >> handled
> > >> > >> by
> > >> > >> > > it.
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >> > Thanks,
> > >> > >> > > > > > >> > Andrii Biletskyi
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi
> <
> > >> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >> > > @Guozhang:
> > >> > >> > > > > > >> > > Thanks for your comments, I've answered some of
> > >> those. The
> > >> > >> > > main
> > >> > >> > > > > > thing
> > >> > >> > > > > > >> is
> > >> > >> > > > > > >> > > having merged request for
> > >> create-alter-delete-describe - I
> > >> > >> > > have
> > >> > >> > > > > some
> > >> > >> > > > > > >> > > concerns about this approach.
> > >> > >> > > > > > >> > >
> > >> > >> > > > > > >> > > @*Jay*:
> > >> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is
> also
> > >> one of
> > >> > >> > the
> > >> > >> > > > > > >> concerns.
> > >> > >> > > > > > >> > We
> > >> > >> > > > > > >> > > can solve it if we implement re-routing
> facility.
> > >> But I
> > >> > >> > agree
> > >> > >> > > > with
> > >> > >> > > > > > >> > > Guozhang - it will make clients' internals a
> > little
> > >> bit
> > >> > >> > easier
> > >> > >> > > > but
> > >> > >> > > > > > >> this
> > >> > >> > > > > > >> > > seems to be a complex logic to implement and
> > >> support then.
> > >> > >> > > > > > Especially
> > >> > >> > > > > > >> for
> > >> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing
> later
> > >> for
> > >> > >> these
> > >> > >> > > > > > >> requests).
> > >> > >> > > > > > >> > > Also people will tend to avoid this re-routing
> > >> facility
> > >> > >> and
> > >> > >> > > hold
> > >> > >> > > > > > local
> > >> > >> > > > > > >> > > cluster cache to ensure their high-priority
> > requests
> > >> > >> (which
> > >> > >> > > some
> > >> > >> > > > > of
> > >> > >> > > > > > >> the
> > >> > >> > > > > > >> > > admin requests are) not sent to some busy broker
> > >> where
> > >> > >> they
> > >> > >> > > wait
> > >> > >> > > > > to
> > >> > >> > > > > > be
> > >> > >> > > > > > >> > > routed to the correct one.
> > >> > >> > > > > > >> > > As pointed out by Jun here (
> > >> > >> > > > > > >> > >
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >>
> > >> > >> > > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >>
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > >> > >> > > > > > >> > )
> > >> > >> > > > > > >> > > to solve the issue we might introduce a message
> > >> type to
> > >> > >> get
> > >> > >> > > > > cluster
> > >> > >> > > > > > >> > state.
> > >> > >> > > > > > >> > > But I agree we can just update
> > >> TopicMetadataResponse to
> > >> > >> > > include
> > >> > >> > > > > > >> > > controllerId (and probably smth else).
> > >> > >> > > > > > >> > > What are you thougths?
> > >> > >> > > > > > >> > >
> > >> > >> > > > > > >> > > Thanks,
> > >> > >> > > > > > >> > > Andrii
> > >> > >> > > > > > >> > >
> > >> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> > >> > >> > > > > wangguoz@gmail.com>
> > >> > >> > > > > > >> > wrote:
> > >> > >> > > > > > >> > >
> > >> > >> > > > > > >> > >> I think for the topics commands we can actually
> > >> merge
> > >> > >> > > > > > >> > >> create/alter/delete/describe as one request
> type
> > >> since
> > >> > >> > their
> > >> > >> > > > > > formats
> > >> > >> > > > > > >> are
> > >> > >> > > > > > >> > >> very much similar, and keep list-topics and
> > others
> > >> like
> > >> > >> > > > > > >> > >> partition-reassignment /
> > preferred-leader-election
> > >> as
> > >> > >> > > separate
> > >> > >> > > > > > >> request
> > >> > >> > > > > > >> > >> types, I also left some other comments on the
> RB
> > (
> > >> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> > >> > >> > > > > > >> > >>
> > >> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> > >> > >> > > > jay.kreps@gmail.com>
> > >> > >> > > > > > >> wrote:
> > >> > >> > > > > > >> > >>
> > >> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to
> just
> > >> have
> > >> > >> one
> > >> > >> > > "do
> > >> > >> > > > > > admin
> > >> > >> > > > > > >> > >> stuff"
> > >> > >> > > > > > >> > >> > command that has the union of all parameters.
> > >> > >> > > > > > >> > >> >
> > >> > >> > > > > > >> > >> > What I am saying is that command line tools
> are
> > >> one
> > >> > >> > client
> > >> > >> > > of
> > >> > >> > > > > the
> > >> > >> > > > > > >> > >> > administrative apis, but these will be used
> in
> > a
> > >> number
> > >> > >> > of
> > >> > >> > > > > > >> scenarios
> > >> > >> > > > > > >> > so
> > >> > >> > > > > > >> > >> > they should make logical sense even in the
> > >> absence of
> > >> > >> the
> > >> > >> > > > > command
> > >> > >> > > > > > >> line
> > >> > >> > > > > > >> > >> > tool. Hence comments like trying to clarify
> the
> > >> > >> > > relationship
> > >> > >> > > > > > >> between
> > >> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these
> kinds
> > >> of
> > >> > >> things
> > >> > >> > > > > really
> > >> > >> > > > > > >> need
> > >> > >> > > > > > >> > >> to be
> > >> > >> > > > > > >> > >> > thought through.
> > >> > >> > > > > > >> > >> >
> > >> > >> > > > > > >> > >> > Hope that makes sense.
> > >> > >> > > > > > >> > >> >
> > >> > >> > > > > > >> > >> > -Jay
> > >> > >> > > > > > >> > >> >
> > >> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
> > >> Biletskyi <
> > >> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > >> > >> > > > > > >> > >> >
> > >> > >> > > > > > >> > >> > > Jay,
> > >> > >> > > > > > >> > >> > >
> > >> > >> > > > > > >> > >> > > Thanks for answering. You understood
> > >> correctly, most
> > >> > >> of
> > >> > >> > > my
> > >> > >> > > > > > >> comments
> > >> > >> > > > > > >> > >> were
> > >> > >> > > > > > >> > >> > > related to your point 1) - about "well
> > >> thought-out"
> > >> > >> > apis.
> > >> > >> > > > > Also,
> > >> > >> > > > > > >> yes,
> > >> > >> > > > > > >> > >> as I
> > >> > >> > > > > > >> > >> > > understood we would like to introduce a
> > single
> > >> > >> unified
> > >> > >> > > CLI
> > >> > >> > > > > tool
> > >> > >> > > > > > >> with
> > >> > >> > > > > > >> > >> > > centralized server-side request handling
> for
> > >> lots of
> > >> > >> > > > existing
> > >> > >> > > > > > >> ones
> > >> > >> > > > > > >> > >> (incl.
> > >> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> > >> > >> ReassignPartitions,
> > >> > >> > > smth
> > >> > >> > > > > > else
> > >> > >> > > > > > >> if
> > >> > >> > > > > > >> > >> added
> > >> > >> > > > > > >> > >> > > in future). In our previous discussion (
> > >> > >> > > > > > >> > >> > >
> > >> https://issues.apache.org/jira/browse/KAFKA-1694
> > )
> >
> > >> > >> > people
> > >> > >> > > > > said
> > >> > >> > > > > > >> > they'd
> > >> > >> > > > > > >> > >> > > rather
> > >> > >> > > > > > >> > >> > > have a separate message for each command,
> so,
> > >> yes,
> > >> > >> this
> > >> > >> > > > way I
> > >> > >> > > > > > >> came
> > >> > >> > > > > > >> > to
> > >> > >> > > > > > >> > >> 1-1
> > >> > >> > > > > > >> > >> > > mapping between commands in the tool and
> > >> protocol
> > >> > >> > > > additions.
> > >> > >> > > &g t; > > But
> >
> > >> > >> > > > > > >> I
> > >> > >> > > > > > >> > >> might
> > >> > >> > > > > > >> > >> > be
> > >> > >> > > > > > >> > >> > > wrong.
> > >> > >> > > > > > >> > >> > > At the end I just try to start discussion
> how
> > >> at
> > >> > >> least
> > >> > >> > > > > > generally
> > >> > >> > > > > > >> > this
> > >> > >> > > > > > >> > >> > > protocol should look like.
> > >> > >> > > > > > >> > >> > >
> > >> > >> > > > > > >> > >> > > Thanks,
> > >> > >> > > > > > >> > >> > > Andrii
> > >> > >> > > > > > >> > >> > >
> > >> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay
> Kreps <
> > >> > >> > > > > > jay.kreps@gmail.com
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >> > >> wrote:
> > >> > >> > > > > > >> > >> > >
> > >> > >> > > > > > >> > >> > > > Hey Andrii,
> > >> > >> > > > > > >> > >> > > >
> > >> > >> > > > > > >> > >> > > > To answer your earlier question we just
> > >> really
> > >> > >> can't
> > >> > >> > be
> > >> > >> > > > > > adding
> > >> > >> > > > > > >> any
> > >> > >> > > > > > >> > >> more
> > >> > >> > > > > > >> > >> > > > scala protocol objects. These things are
> > >> super hard
> > >> > >> > to
> > >> > >> > > > > > maintain
> > >> > >> > > > > > >> > >> because
> > >> > >> > > > > > >> > >> > > > they hand code the byte parsing and don't
> > >> have good
> > >> > >> > > > > > versioning
> > >> > >> > > > > > >> > >> support.
> > >> > >> > > > > > >> > >> > > > Since we are already planning on
> converting
> > >> we
> > >> > >> > > definitely
> > >> > >> > > > > > don't
> > >> > >> > > > > > >> > >> want to
> > >> > >> > > > > > >> > >> > > add
> > >> > >> > > > > > >> > >> > > > a ton more of these--they are total tech
> > >> debt.
> > >> > >> > > > > > >> > >> > > >
> > >> > >> > > > > > >> > >> > > > What does it mean that the changes are
> > >> isolated
> > >> > >> from
> > >> > >> > > the
> > >> > >> > > > > > >> current
> > >> > >> > > > > > >> > >> code
> > >> > >> > > > > > >> > >> > > base?
> > >> > >> > > > > > >> > >> > > >
> > >> > >> > > > > > >> > >> > > > I actually didn't understand the
> remaining
> > >> > >> comments,
> > >> > >> > > > which
> > >> > >> > > > > of
> > >> > >> > > > > > >> the
> > >> > >> > > > > > >> > >> > points
> > >> > >> > > > > > >> > >> > > > are you responding to?
> > >> > >> > > > > > >> > >> > > >
> > >> > >> > > > > > >> > >> > > > Maybe one sticking point here is that it
> > >> seems like
> > >> > >> > you
> > >> > >> > > > > want
> > >> > >> > > > > > to
> > >> > >> > > > > > >> > make
> > >> > >> > > > > > >> > >> > some
> > >> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1
> > mapping
> > >> > >> between
> > >> > >> > > > > > commands
> > >> > >> > > > > > >> you
> > >> > >> > > > > > >> > >> > > imagine
> > >> > >> > > > > > >> > >> > > > in the tool and protocol additions. I
> want
> > >> to make
> > >> > >> > sure
> > >> > >> > > > we
> > >> > >> > > > > > >> don't
> > >> > >> > > > > > >> > do
> > >> > >> > > > > > >> > >> > that.
> > >> > >> > > > > > >> > >> > > > The protocol needs to be really really
> well
> > >> thought
> > >> > >> > out
> > >> > >> > > > > > against
> > >> > >> > > > > > >> > many
> > >> > >> > > > > > >> > >> > use
> > >> > >> > > > > > >> > >> > > > cases so it should make perfect logical
> > >> sense in
> > >> > >> the
> > >> > >> > > > > absence
> > >> > >> > > > > > of
> > >> > >> > > > > > >> > >> knowing
> > >> > >> > > > > > >> > >> > > the
> > >> > >> > > > > > >> > >> > > > command line tool, right?
> > >> > >> > > > > > >> > >> > > >
> > >> > >> > > > > > >> > >> > > > -Jay
> > >> > >> > > > > > >> > >> > > >
> > >> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii
> > >> Biletskyi
> > >> > >> <
> > >> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > >> > > > > > >> > >> > > >
> > >> > >> > > > > > >> > >> > > > > Hey Jay,
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > > > I would like to continue this
> discussion
> > >> as it
> > >> > >> seem
> > >> > >> > > > there
> > >> > >> > > > > > is
> > >> > >> > > > > > >> no
> > >> > >> > > > > > >> > >> > > progress
> > >> > >> > > > > > >> > >> > > > > here.
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > > > First of all, could you please explain
> > >> what did
> > >> > >> you
> > >> > >> > > > mean
> > >> > >> > > > > in
> > >> > >> > > > > > >> 2?
> > >> > >> > > > > > >> > How
> > >> > >> > > > > > >> > >> > > > exactly
> > >> > >> > > > > > >> > >> > > > > are we going to migrate to the new java
> > >> protocol
> > >> > >> > > > > > definitions.
> > >> > >> > > > > > >> > And
> > >> > >> > > > > > >> > >> why
> > >> > >> > > > > > >> > >> > > > it's
> > >> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > > > I agree with you, this feature includes
> > >> lots of
> > >> > >> > > stuff,
> > >> > >> > > > > but
> > >> > >> > > > > > >> > >> thankfully
> > >> > >> > > > > > >> > >> > > > > almost all changes are isolated from
> the
> > >> current
> > >> > >> > code
> > >> > >> > > > > base,
> > >> > >> > > > > > >> > >> > > > > so the main thing, I think, we need to
> > >> agree is
> > >> > >> > RQ/RP
> > >> > >> > > > > > format.
> > >> > >> > > > > > >> > >> > > > > So how can we start discussion about
> the
> > >> concrete
> > >> > >> > > > > messages
> > >> > >> > > > > > >> > format?
> > >> > >> > > > > > >> > >> > > > > Can we take (
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > >
> > >> > >> > > > > > >> > >> > >
> > >> > >> > > > > > >> > >> >
> > >> > >> > > > > > >> > >>
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >>
> > >> > >> > > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >>
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> >
> > >> > >> > > > > > >> > >> > > > > )
> > >> > >> > > > > > >> > >> > > > > as starting point?
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > > > We had some doubts earlier whether it
> > worth
> > >> > >> > > introducing
> > >> > >> > > > > one
> > >> > >> > > > > > >> > >> generic
> > >> > >> > > > > > >> > >> > > Admin
> > >> > >> > > > > > >> > >> > > > > Request for all commands (
> > >> > >> > > > > > >> > >> > > >
> > >> https://issues.apache.org/jira/browse/KAFKA-1694
> > >> > >> > > > > > >> > >> > > > > )
> > >> > >> > > > > > >> > >> > > > > but then everybody agreed it would be
> > >> better to
> > >> > >> > have
> > >> > >> > > > > > separate
> > >> > >> > > > > > >> > >> message
> > >> > >> > > > > > >> > >> > > for
> > >> > >> > > > > > >> > >> > > > > each admin command. The Request part is
> > >> really
> > >> > >> > > dictated
> > >> > >> > > > > > from
> > >> > >> > > > > > >> the
> > >> > >> > > > > > >> > >> > > command
> > >> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself,
> so
> > >> the
> > >> > >> > proposed
> > >> > >> > > > > > version
> > >> > >> > > > > > >> > >> should
> > >> > >> > > > > > >> > >> > be
> > >> > >> > > > > > >> > >> > > > > fine (let's put aside for now remarks
> > about
> > >> > >> > Optional
> > >> > >> > > > > type,
> > >> > >> > > > > > >> > >> batching,
> > >> > >> > > > > > >> > >> > > > > configs normalization - I agree with
> all
> > of
> > >> > >> them).
> > >> > >> > > > > > >> > >> > > > > So the second part is Response. I see
> > >> there are
> > >> > >> two
> > >> > >> > > > cases
> > >> > >> > > > > > >> here.
> > >> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> Create/Alter/... ;
> > >> b)
> > >> > >> "Get"
> > >> > >> > > > > > requests -
> > >> > >> > > > > > >> > >> > > > > List/Describe...
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > > > a) should only hold request result
> > >> (regardless
> > >> > >> what
> > >> > >> > > we
> > >> > >> > > > > > decide
> > >> > >> > > > > > >> > >> about
> > >> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
> > execution).
> > >> > >> > > > > > >> > >> > > > > Usually we provide error code in
> response
> > >> but
> > >> > >> since
> > >> > >> > > we
> > >> > >> > > > > will
> > >> > >> > > > > > >> use
> > >> > >> > > > > > >> > >> this
> > >> > >> > > > > > >> > >> > in
> > >> > >> > > > > > >> > >> > > > > interactive shell we need some human
> > >> readable
> > >> > >> error
> > >> > >> > > > > > >> description
> > >> > >> > > > > > >> > -
> > >> > >> > > > > > >> > >> so
> > >> > >> > > > > > >> > >> > I
> > >> > >> > > > > > >> > >> > > > > added errorDesription field where you
> can
> > >> at
> > >> > >> least
> > >> > >> > > > leave
> > >> > >> > > > > > >> > >> > > > > exception.getMessage.
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > > > b) in addition to previous item message
> > >> should
> > >> > >> hold
> > >> > >> > > > > command
> > >> > >> > > > > > >> > >> specific
> > >> > >> > > > > > >> > >> > > > > response data. We can discuss in detail
> > >> each of
> > >> > >> > them
> > >> > >> > > > but
> > >> > >> > > > > > >> let's
> > >> > >> > > > > > >> > for
> > >> > >> > > > > > >> > >> > now
> > >> > >> > > > > > >> > >> > > > > agree about the overall pattern.
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > > > Thanks,
> > >> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay
> > Kreps
> > >> <
> > >> > >> > > > > > >> jay.kreps@gmail.com
> > >> > >> > > > > > >> > >
> > >> > >> > > > > > >> > >> > > wrote:
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > > > > Hey Joe,
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > This is great. A few comments on
> KIP-4
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > 1. This is much needed functionality,
> > >> but there
> > >> > >> > > are a
> > >> > >> > > > > lot
> > >> > >> > > > > > >> of
> > >> > >> > > > > > >> > >> the so
> > >> > >> > > > > > >> > >> > > > let's
> > >> > >> > > > > > >> > >> > > > > > really think these protocols through.
> > We
> > >> really
> > >> > >> > > want
> > >> > >> > > > to
> > >> > >> > > > > > >> end up
> > >> > >> > > > > > >> > >> > with a
> > >> > >> > > > > > >> > >> > > > set
> > >> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis.
> > >> For this
> > >> > >> > > > reason I
> > >> > >> > > > > > >> think
> > >> > >> > > > > > >> > >> it is
> > >> > >> > > > > > >> > >> > > > > really
> > >> > >> > > > > > >> > >> > > > > > important to think through the end
> > state
> > >> even
> > >> > >> if
> > >> > >> > > that
> > >> > >> > > > > > >> includes
> > >> > >> > > > > > >> > >> APIs
> > >> > >> > > > > > >> > >> > > we
> > >> > >> > > > > > >> > >> > > > > > won't implement in the first phase.
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait
> > until
> > >> we
> > >> > >> have
> > >> > >> > > > > switched
> > >> > >> > > > > > >> the
> > >> > >> > > > > > >> > >> > server
> > >> > >> > > > > > >> > >> > > > over
> > >> > >> > > > > > >> > >> > > > > > to the new java protocol definitions.
> > If
> > >> we add
> > >> > >> > > > upteen
> > >> > >> > > > > > >> more ad
> > >> > >> > > > > > >> > >> hoc
> > >> > >> > > > > > >> > >> > > > scala
> > >> > >> > > > > > >> > >> > > > > > objects that is just generating more
> > >> work for
> > >> > >> the
> > >> > >> > > > > > >> conversion
> > >> > >> > > > > > >> > we
> > >> > >> > > > > > >> > >> > know
> > >> > >> > > > > > >> > >> > > we
> > >> > >> > > > > > >> > >> > > > > > have to do.
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new
> type
> > of
> > >> > >> > optional
> > >> > >> > > > > > >> parameter.
> > >> > >> > > > > > >> > >> This
> > >> > >> > > > > > >> > >> > is
> > >> > >> > > > > > >> > >> > > > > > inconsistent with everything else in
> > the
> > >> > >> protocol
> > >> > >> > > > where
> > >> > >> > > > > > we
> > >> > >> > > > > > >> use
> > >> > >> > > > > > >> > >> -1
> > >> > >> > > > > > >> > >> > or
> > >> > >> > > > > > >> > >> > > > some
> > >> > >> > > > > > >> > >> > > > > > other marker value. You could argue
> > >> either way
> > >> > >> > but
> > >> > >> > > > > let's
> > >> > >> > > > > > >> stick
> > >> > >> > > > > > >> > >> with
> > >> > >> > > > > > >> > >> > > > that
> > >> > >> > > > > > >> > >> > > > > > for consistency. For clients that
> > >> implemented
> > >> > >> the
> > >> > >> > > > > > protocol
> > >> > >> > > > > > >> in
> > >> > >> > > > > > >> > a
> > >> > >> > > > > > >> > >> > > better
> > >> > >> > > > > > >> > >> > > > > way
> > >> > >> > > > > > >> > >> > > > > > than our scala code these basic
> > >> primitives are
> > >> > >> > hard
> > >> > >> > > > to
> > >> > >> > > > > > >> change.
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
> > >> duplicate
> > >> > >> > > > > > >> > TopicMetadataRequest
> > >> > >> > > > > > >> > >> > > which
> > >> > >> > > > > > >> > >> > > > > has
> > >> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I
> > think
> > >> we
> > >> > >> > should
> > >> > >> > > > > rename
> > >> > >> > > > > > >> that
> > >> > >> > > > > > >> > >> > > request
> > >> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> > >> > >> MetadataRequest)
> > >> > >> > > and
> > >> > >> > > > > > >> include
> > >> > >> > > > > > >> > >> the id
> > >> > >> > > > > > >> > >> > > of
> > >> > >> > > > > > >> > >> > > > > the
> > >> > >> > > > > > >> > >> > > > > > controller. Or are there other things
> > we
> > >> could
> > >> > >> > add
> > >> > >> > > > > here?
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to make
> a
> > >> lot of
> > >> > >> > > > requests
> > >> > >> > > > > > that
> > >> > >> > > > > > >> > can
> > >> > >> > > > > > >> > >> > only
> > >> > >> > > > > > >> > >> > > go
> > >> > >> > > > > > >> > >> > > > > to
> > >> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of
> > >> burden for
> > >> > >> > > > client
> > >> > >> > > > > > >> > >> > > implementations
> > >> > >> > > > > > >> > >> > > > > (it
> > >> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can
> fail
> > >> in many
> > >> > >> > > parts
> > >> > >> > > > > so
> > >> > >> > > > > > it
> > >> > >> > > > > > >> > >> ends up
> > >> > >> > > > > > >> > >> > > > > being a
> > >> > >> > > > > > >> > >> > > > > > full state machine to do right). I
> > think
> > >> we
> > >> > >> > should
> > >> > >> > > > > > consider
> > >> > >> > > > > > >> > >> making
> > >> > >> > > > > > >> > >> > > > admin
> > >> > >> > > > > > >> > >> > > > > > commands and ideally as many of the
> > >> other apis
> > >> > >> as
> > >> > >> > > > > > possible
> > >> > >> > > > > > >> > >> > available
> > >> > >> > > > > > >> > >> > > on
> > >> > >> > > > > > >> > >> > > > > all
> > >> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
> > >> controller on
> > >> > >> > the
> > >> > >> > > > > broker
> > >> > >> > > > > > >> > side.
> > >> > >> > > > > > >> > >> > > Perhaps
> > >> > >> > > > > > >> > >> > > > > > there would be a general way to
> > >> encapsulate
> > >> > >> this
> > >> > >> > > > > > re-routing
> > >> > >> > > > > > >> > >> > behavior.
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > 6. We should probably normalize the
> key
> > >> value
> > >> > >> > pairs
> > >> > >> > > > > used
> > >> > >> > > > > > >> for
> > >> > >> > > > > > >> > >> > configs
> > >> > >> > > > > > >> > >> > > > > rather
> > >> > >> > > > > > >> > >> > > > > > than embedding a new formatting. So
> two
> > >> strings
> > >> > >> > > > rather
> > >> > >> > > > > > than
> > >> > >> > > > > > >> > one
> > >> > >> > > > > > >> > >> > with
> > >> > >> > > > > > >> > >> > > an
> > >> > >> > > > > > >> > >> > > > > > internal equals sign.
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs
> > >> that the
> > >> > >> > > > command
> > >> > >> > > > > > has
> > >> > >> > > > > > >> > >> begun or
> > >> > >> > > > > > >> > >> > > > that
> > >> > >> > > > > > >> > >> > > > > > the command has been completed? It
> is a
> > >> lot
> > >> > >> more
> > >> > >> > > > usable
> > >> > >> > > > > > if
> > >> > >> > > > > > >> the
> > >> > >> > > > > > >> > >> > > command
> > >> > >> > > > > > >> > >> > > > > has
> > >> > >> > > > > > >> > >> > > > > > been completed so you know that if
> you
> > >> create a
> > >> > >> > > topic
> > >> > >> > > > > and
> > >> > >> > > > > > >> then
> > >> > >> > > > > > >> > >> > > publish
> > >> > >> > > > > > >> > >> > > > to
> > >> > >> > > > > > >> > >> > > > > > it you won't get an exception about
> > >> there being
> > >> > >> > no
> > >> > >> > > > such
> > >> > >> > > > > > >> topic.
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
> > >> duplicate a
> > >> > >> lot
> > >> > >> > > of
> > >> > >> > > > > > stuff
> > >> > >> > > > > > >> in
> > >> > >> > > > > > >> > >> the
> > >> > >> > > > > > >> > >> > > > > metadata
> > >> > >> > > > > > >> > >> > > > > > request. Is there a reason to give
> back
> > >> topics
> > >> > >> > > marked
> > >> > >> > > > > for
> > >> > >> > > > > > >> > >> > deletion? I
> > >> > >> > > > > > >> > >> > > > > feel
> > >> > >> > > > > > >> > >> > > > > > like if we just make the
> post-condition
> > >> of the
> > >> > >> > > delete
> > >> > >> > > > > > >> command
> > >> > >> > > > > > >> > be
> > >> > >> > > > > > >> > >> > that
> > >> > >> > > > > > >> > >> > > > the
> > >> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid of
> > >> the need
> > >> > >> > for
> > >> > >> > > > this
> > >> > >> > > > > > >> right?
> > >> > >> > > > > > >> > >> And
> > >> > >> > > > > > >> > >> > it
> > >> > >> > > > > > >> > >> > > > > will
> > >> > >> > > > > > >> > >> > > > > > be much more intuitive.
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > 9. Should we consider batching these
> > >> requests?
> > >> > >> We
> > >> > >> > > > have
> > >> > >> > > > > > >> > generally
> > >> > >> > > > > > >> > >> > > tried
> > >> > >> > > > > > >> > >> > > > to
> > >> > >> > > > > > >> > >> > > > > > allow multiple operations to be
> > batched.
> > >> My
> > >> > >> > > suspicion
> > >> > >> > > > > is
> > >> > >> > > > > > >> that
> > >> > >> > > > > > >> > >> > without
> > >> > >> > > > > > >> > >> > > > > this
> > >> > >> > > > > > >> > >> > > > > > we will get a lot of code that does
> > >> something
> > >> > >> > like
> > >> > >> > > > > > >> > >> > > > > >    for(topic:
> adminClient.listTopics())
> > >> > >> > > > > > >> > >> > > > > >
>  adminClient.describeTopic(topic)
> > >> > >> > > > > > >> > >> > > > > > this code will work great when you
> test
> > >> on 5
> > >> > >> > topics
> > >> > >> > > > but
> > >> > >> > > > > > >> not do
> > >> > >> > > > > > >> > >> as
> > >> > >> > > > > > >> > >> > > well
> > >> > >> > > > > > >> > >> > > > if
> > >> > >> > > > > > >> > >> > > > > > you have 50k.
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss
> how
> > >> we want
> > >> > >> to
> > >> > >> > > > > expose
> > >> > >> > > > > > a
> > >> > >> > > > > > >> > >> > > programmatic
> > >> > >> > > > > > >> > >> > > > > JVM
> > >> > >> > > > > > >> > >> > > > > > client api for these operations.
> > >> Currently
> > >> > >> people
> > >> > >> > > > rely
> > >> > >> > > > > on
> > >> > >> > > > > > >> > >> > AdminUtils
> > >> > >> > > > > > >> > >> > > > > which
> > >> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we
> probably
> > >> need
> > >> > >> > > another
> > >> > >> > > > > > client
> > >> > >> > > > > > >> > >> under
> > >> > >> > > > > > >> > >> > > > > clients/
> > >> > >> > > > > > >> > >> > > > > > that exposes administrative
> > >> functionality. We
> > >> > >> > will
> > >> > >> > > > need
> > >> > >> > > > > > >> this
> > >> > >> > > > > > >> > >> just
> > >> > >> > > > > > >> > >> > to
> > >> > >> > > > > > >> > >> > > > > > properly test the new apis, I
> suspect.
> > We
> > >> > >> should
> > >> > >> > > > figure
> > >> > >> > > > > > out
> > >> > >> > > > > > >> > that
> > >> > >> > > > > > >> > >> > API.
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > 11. The other information that would
> be
> > >> really
> > >> > >> > > useful
> > >> > >> > > > > to
> > >> > >> > > > > > >> get
> > >> > >> > > > > > >> > >> would
> > >> > >> > > > > > >> > >> > be
> > >> > >> > > > > > >> > >> > > > > > information about partitions--how
> much
> > >> data is
> > >> > >> in
> > >> > >> > > the
> > >> > >> > > > > > >> > partition,
> > >> > >> > > > > > >> > >> > what
> > >> > >> > > > > > >> > >> > > > are
> > >> > >> > > > > > >> > >> > > > > > the segment offsets, what is the
> > log-end
> > >> offset
> > >> > >> > > (i.e.
> > >> > >> > > > > > last
> > >> > >> > > > > > >> > >> offset),
> > >> > >> > > > > > >> > >> > > > what
> > >> > >> > > > > > >> > >> > > > > is
> > >> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think
> that
> > >> done
> > >> > >> > right
> > >> > >> > > > this
> > >> > >> > > > > > >> would
> > >> > >> > > > > > >> > be
> > >> > >> > > > > > >> > >> > the
> > >> > >> > > > > > >> > >> > > > > > successor to the very awkward
> > >> OffsetRequest we
> > >> > >> > have
> > >> > >> > > > > > today.
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > -Jay
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe
> > >> Stein <
> > >> > >> > > > > > >> > >> joe.stein@stealth.ly>
> > >> > >> > > > > > >> > >> > > > > wrote:
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> > >> > >> > > > > > >> > >> > > > > > >
> > >> > >> > > > > > >> > >> > > > > > >
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > >
> > >> > >> > > > > > >> > >> > >
> > >> > >> > > > > > >> > >> >
> > >> > >> > > > > > >> > >>
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >>
> > >> > >> > > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >> > >> > > > > > >> > >> > > > > > >
> > >> > >> > > > > > >> > >> > > > > > > JIRA
> > >> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > >> > >> > > > > > >> > >> > > > > > >
> > >> > >> > > > > > >> > >> > > > > > >
> > >> /*******************************************
> > >> > >> > > > > > >> > >> > > > > > >  Joe Stein
> > >> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> > >> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
> > >> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> > >> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> > >> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> > >> > >> > > > > > >> > >> > > >
> > >> > >> > > > > > >> > >> > > > > > >
> > >> ********************************************/
> > >> > >> > > > > > >> > >> > > > > > >
> > >> > >> > > > > > >> > >> > > > > >
> > >> > >> > > > > > >> > >> > > > >
> > >> > >> > > > > > >> > >> > > >
> > >> > >> > > > > > >> > >> > >
> > >> > >> > > > > > >> > >> >
> > >> > >> > > > > > >> > >>
> > >> > >> > > > > > >> > >>
> > >> > >> > > > > > >> > >>
> > >> > >> > > > > > >> > >> --
> > >> > >> > > > > > >> > >> -- Guozhang
> > >> > >> > > > > > >> > >>
> > >> > >> > > > > > >> > >
> > >> > >> > > > > > >> > >
> > >> > >> > > > > > >> >
> > >> > >> > > > > > >>
> > >> > >> > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >> >
> > >> > >> >
> > >> > >> > --
> > >> > >> > Jeff Holoman
> > >> > >> > Systems Engineer
> > >> > >> >
> > >> > >>
> > >>
> > >>
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Joe Stein <jo...@stealth.ly>.
Guozhang and Tong, I really do like this idea and where your discussion
will lead as it will be very useful for folks to have. I am really
concerned though that we are scope creeping this KIP.

Andrii is already working on following up on ~ 14 different items of
feedback in regards to the core motivations/scope of the KIP. He has
uploaded a new patch already and the KIP based on those items and will be
responding to this thread about that and for what else still requires
discussion hopefully in the next few hours.

I want to make sure we are focusing on the open items still requiring
discussion and stabilizing what we have before trying to introducing more
new features.

Perhaps a new KIP can get added for the new features you are talking about
which can reference this and once this is committed that work can begin for
folks that are able to contribute to work on it?

~ Joe Stein
- - - - - - - - - - - - - - - - -

  http://www.stealth.ly
- - - - - - - - - - - - - - - - -

On Thu, Mar 12, 2015 at 9:51 AM, Tong Li <li...@us.ibm.com> wrote:

> Guozhang,
>      augmenting topic is fine, but as soon as we start doing that, other
> issues follow, for example, access control, who can access the topic, who
> can grant permissions. how the information (metadata) itself gets secured.
> Should the information be saved in ZK or a datastore? Will using a metadata
> file causing long term problems such as file updates/synchronization, once
> we have this metadata file, more people will want to put more stuff in it.
> how can we control the format? K-V pair not good for large data set.
>     Clearly there is a need for it, I wonder if we can make this thing
> plugable and provide a default implementation which allows us try different
> solutions and also allow people to completely ignore it if they do not want
> to deal with any of these.
>
> Thanks.
>
> Tong Li
> OpenStack & Kafka Community Development
> Building 501/B205
> litong01@us.ibm.com
>
> [image: Inactive hide details for Guozhang Wang ---03/12/2015 09:39:50
> AM---Folks, Just want to elaborate a bit more on the create-topi]Guozhang
> Wang ---03/12/2015 09:39:50 AM---Folks, Just want to elaborate a bit more
> on the create-topic metadata and batching
>
> From: Guozhang Wang <wa...@gmail.com>
> To: "dev@kafka.apache.org" <de...@kafka.apache.org>
> Date: 03/12/2015 09:39 AM
> Subject: Re: [DISCUSS] KIP-4 - Command line and centralized
> administrative operations
> ------------------------------
>
>
>
> Folks,
>
> Just want to elaborate a bit more on the create-topic metadata and batching
> describe-topic based on config / metadata in my previous email as we work
> on KAFKA-1694. The main motivation is to have some sort of topic management
> mechanisms, which I think is quite important in a multi-tenant / cloud
> architecture: today anyone can create topics in a shared Kafka cluster, but
> there is no concept or "ownership" of topics that are created by different
> users. For example, at LinkedIn we basically distinguish topic owners via
> some casual topic name prefix, which is a bit awkward and does not fly as
> we scale our customers. It would be great to use describe-topics such as:
>
> Describe all topics that is created by me.
>
> Describe all topics whose retention time is overriden to X.
>
> Describe all topics whose writable group include user Y (this is related to
> authorization), etc..
>
> One possible way to achieve this is to add a metadata file in the
> create-topic request, whose value will also be written ZK as we create the
> topic; then describe-topics can choose to batch topics based on 1) name
> regex, 2) config K-V matching, 3) metadata regex, etc.
>
> Thoughts?
>
> Guozhang
>
> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Thanks for the updated wiki. A few comments below:
> >
> > 1. Error description in response: I think if some errorCode could
> indicate
> > several different error cases then we should really change it to multiple
> > codes. In general the errorCode itself would be precise and sufficient
> for
> > describing the server side errors.
> >
> > 2. Describe topic request: it would be great to go beyond just batching
> on
> > topic name regex for this request. For example, a very common use case of
> > the topic command is to list all topics whose config A's value is B. With
> > topic name regex then we have to first retrieve __all__ topics's
> > description info and then filter at the client end, which will be a huge
> > burden on ZK.
> >
> > 3. Config K-Vs in create topic: this is related to the previous point;
> > maybe we can add another metadata K-V or just a metadata string along
> side
> > with config K-V in create topic like we did for offset commit request.
> This
> > field can be quite useful in storing information like "owner" of the
> topic
> > who issue the create command, etc, which is quite important for a
> > multi-tenant setting. Then in the describe topic request we can also
> batch
> > on regex of the metadata field.
> >
> > 4. Today all the admin operations are async in the sense that command
> will
> > return once it is written in ZK, and that is why we need extra
> verification
> > like testUtil.waitForTopicCreated() / verify partition reassignment
> > request, etc. With admin requests we could add a flag to enable / disable
> > synchronous requests; when it is turned on, the response will not return
> > until the request has been completed. And for async requests we can add a
> > "token" field in the response, and then only need a general "admin
> > verification request" with the given token to check if the async request
> > has been completed.
> >
> > 5. +1 for extending Metadata request to include controller / coordinator
> > information, and then we can remove the ConsumerMetadata /
> ClusterMetadata
> > requests.
> >
> > Guozhang
> >
> > On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com> wrote:
> >
> >> Thanks for sending that out Joe - I don't think I will be able to make
> >> it today, so if notes can be sent out afterward that would be great.
> >>
> >> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> >> > Thanks for sending this out Joe. Looking forward to chatting with
> >> everyone :)
> >> >
> >> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly>
> wrote:
> >> > > Hey, I just sent out a google hangout invite to all pmc, committers
> >> and
> >> > > everyone I found working on a KIP. If I missed anyone in the invite
> >> please
> >> > > let me know and can update it, np.
> >> > >
> >> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can get
> >> INFRA
> >> > > help to make a google account so we can manage better?
> >> > >
> >> > > To discuss
> >> > >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >> > > in progress and related JIRA that are interdependent and common
> work.
> >> > >
> >> > > ~ Joe Stein
> >> > >
> >> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com>
> >> wrote:
> >> > >
> >> > >> Let's stay on Google hangouts that will also record and make the
> >> sessions
> >> > >> available on youtube.
> >> > >>
> >> > >> -Jay
> >> > >>
> >> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> >> jholoman@cloudera.com>
> >> > >> wrote:
> >> > >>
> >> > >> > Jay / Joe
> >> > >> >
> >> > >> > We're happy to send out a Webex for this purpose. We could record
> >> the
> >> > >> > sessions if there is interest and publish them out.
> >> > >> >
> >> > >> > Thanks
> >> > >> >
> >> > >> > Jeff
> >> > >> >
> >> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <jay.kreps@gmail.com
> >
> >> wrote:
> >> > >> >
> >> > >> > > Let's try to get the technical hang-ups sorted out, though. I
> >> really
> >> > >> > think
> >> > >> > > there is some benefit to live discussion vs writing. I am
> >> hopeful that
> >> > >> if
> >> > >> > > we post instructions and give ourselves a few attempts we can
> >> get it
> >> > >> > > working.
> >> > >> > >
> >> > >> > > Tuesday at that time would work for me...any objections?
> >> > >> > >
> >> > >> > > -Jay
> >> > >> > >
> >> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> joe.stein@stealth.ly
> >> >
> >> > >> wrote:
> >> > >> > >
> >> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET /
> 10am
> >> PT
> >> > >> ????
> >> > >> > > >
> >> > >> > > > I don't mind google hangout but there is always some issue or
> >> > >> whatever
> >> > >> > so
> >> > >> > > > we know the apache irc channel works. We can start there and
> >> see how
> >> > >> it
> >> > >> > > > goes? We can pull transcripts too and associate to tickets if
> >> need be
> >> > >> > > makes
> >> > >> > > > it helpful for things.
> >> > >> > > >
> >> > >> > > > ~ Joestein
> >> > >> > > >
> >> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> >> jay.kreps@gmail.com>
> >> > >> > wrote:
> >> > >> > > >
> >> > >> > > > > We'd talked about doing a Google Hangout to chat about
> this.
> >> What
> >> > >> > about
> >> > >> > > > > generalizing that a little further...I actually think it
> >> would be
> >> > >> > good
> >> > >> > > > for
> >> > >> > > > > everyone spending a reasonable chunk of their week on Kafka
> >> stuff
> >> > >> to
> >> > >> > > > maybe
> >> > >> > > > > sync up once a week. I think we could use time to talk
> >> through
> >> > >> design
> >> > >> > > > > stuff, make sure we are on top of code reviews, talk
> through
> >> any
> >> > >> > tricky
> >> > >> > > > > issues, etc.
> >> > >> > > > >
> >> > >> > > > > We can make it publicly available so that any one can
> follow
> >> along
> >> > >> > who
> >> > >> > > > > likes.
> >> > >> > > > >
> >> > >> > > > > Any interest in doing this? If so I'll try to set it up
> >> starting
> >> > >> next
> >> > >> > > > week.
> >> > >> > > > >
> >> > >> > > > > -Jay
> >> > >> > > > >
> >> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> >> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > >> > > > >
> >> > >> > > > > > Hi all,
> >> > >> > > > > >
> >> > >> > > > > > I've updated KIP page, fixed / aligned document
> structure.
> >> Also I
> >> > >> > > added
> >> > >> > > > > > some
> >> > >> > > > > > very initial proposal for AdminClient so we have
> something
> >> to
> >> > >> start
> >> > >> > > > from
> >> > >> > > > > > while
> >> > >> > > > > > discussing the KIP.
> >> > >> > > > > >
> >> > >> > > > > >
> >> > >> > > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> > >> > > > > >
> >> > >> > > > > > Thanks,
> >> > >> > > > > > Andrii Biletskyi
> >> > >> > > > > >
> >> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> >> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > >> > > > > >
> >> > >> > > > > > > Jay,
> >> > >> > > > > > >
> >> > >> > > > > > > Re error messages: you are right, in most cases client
> >> will
> >> > >> have
> >> > >> > > > enough
> >> > >> > > > > > > context to show descriptive error message. My concern
> is
> >> that
> >> > >> we
> >> > >> > > will
> >> > >> > > > > > have
> >> > >> > > > > > > to
> >> > >> > > > > > > add lots of new error codes for each possible error. Of
> >> course,
> >> > >> > we
> >> > >> > > > > could
> >> > >> > > > > > > reuse
> >> > >> > > > > > > some of existing like UknownTopicOrPartitionCode, but
> we
> >> will
> >> > >> > also
> >> > >> > > > need
> >> > >> > > > > > to
> >> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> >> TopicConfigInvalid (both
> >> > >> > for
> >> > >> > > > > topic
> >> > >> > > > > > > name and config, and probably user would like to know
> >> what
> >> > >> > exactly
> >> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
> >> > >> InternalError
> >> > >> > > > (e.g.
> >> > >> > > > > > > zookeeper failure) etc.
> >> > >> > > > > > > And this is only for TopicCommand, we will also need to
> >> add
> >> > >> > similar
> >> > >> > > > > stuff
> >> > >> > > > > > > for
> >> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up
> >> with a
> >> > >> > large
> >> > >> > > > list
> >> > >> > > > > > of
> >> > >> > > > > > > error codes, used only in Admin protocol.
> >> > >> > > > > > > Having said that, I agree my proposal is not consistent
> >> with
> >> > >> > other
> >> > >> > > > > cases.
> >> > >> > > > > > > Maybe we can find better solution or something
> >> in-between.
> >> > >> > > > > > >
> >> > >> > > > > > > Re Hangout chat: I think it is a great idea. This way
> we
> >> can
> >> > >> move
> >> > >> > > on
> >> > >> > > > > > > faster.
> >> > >> > > > > > > Let's agree somehow on date/time so people can join.
> >> Will work
> >> > >> > for
> >> > >> > > me
> >> > >> > > > > > this
> >> > >> > > > > > > and
> >> > >> > > > > > > next week almost anytime if agreed in advance.
> >> > >> > > > > > >
> >> > >> > > > > > > Thanks,
> >> > >> > > > > > > Andrii
> >> > >> > > > > > >
> >> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> >> > >> jay.kreps@gmail.com>
> >> > >> > > > > wrote:
> >> > >> > > > > > >
> >> > >> > > > > > >> Hey Andrii,
> >> > >> > > > > > >>
> >> > >> > > > > > >> Generally we can do good error handling without
> needing
> >> custom
> >> > >> > > > > > server-side
> >> > >> > > > > > >> messages. I.e. generally the client has the context to
> >> know
> >> > >> that
> >> > >> > > if
> >> > >> > > > it
> >> > >> > > > > > got
> >> > >> > > > > > >> an error that the topic doesn't exist to say "Topic X
> >> doesn't
> >> > >> > > exist"
> >> > >> > > > > > >> rather
> >> > >> > > > > > >> than "error code 14" (or whatever). Maybe there are
> >> specific
> >> > >> > cases
> >> > >> > > > > where
> >> > >> > > > > > >> this is hard? If we want to add server-side error
> >> messages we
> >> > >> > > really
> >> > >> > > > > do
> >> > >> > > > > > >> need to do this in a consistent way across the
> protocol.
> >> > >> > > > > > >>
> >> > >> > > > > > >> I still have a bunch of open questions here from my
> >> previous
> >> > >> > > list. I
> >> > >> > > > > > will
> >> > >> > > > > > >> be out for the next few days for Strata though. Maybe
> >> we could
> >> > >> > do
> >> > >> > > a
> >> > >> > > > > > Google
> >> > >> > > > > > >> Hangout chat on any open issues some time towards the
> >> end of
> >> > >> > next
> >> > >> > > > week
> >> > >> > > > > > for
> >> > >> > > > > > >> anyone interested in this ticket? I have a feeling
> that
> >> might
> >> > >> > > > progress
> >> > >> > > > > > >> things a little faster than email--I think we could
> talk
> >> > >> through
> >> > >> > > > those
> >> > >> > > > > > >> issues I brought up fairly quickly...
> >> > >> > > > > > >>
> >> > >> > > > > > >> -Jay
> >> > >> > > > > > >>
> >> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> >> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> >> > >> > > > > > >>
> >> > >> > > > > > >> > Hi all,
> >> > >> > > > > > >> >
> >> > >> > > > > > >> > I'm trying to address some of the issues which were
> >> > >> mentioned
> >> > >> > > > > earlier
> >> > >> > > > > > >> about
> >> > >> > > > > > >> > Admin RQ/RP format. One of those was about batching
> >> > >> > operations.
> >> > >> > > > What
> >> > >> > > > > > if
> >> > >> > > > > > >> we
> >> > >> > > > > > >> > follow TopicCommand approach and let people specify
> >> > >> topic-name
> >> > >> > > by
> >> > >> > > > > > >> regexp -
> >> > >> > > > > > >> > would that cover most of the use cases?
> >> > >> > > > > > >> >
> >> > >> > > > > > >> > Secondly, is what information should we generally
> >> provide in
> >> > >> > > Admin
> >> > >> > > > > > >> > responses.
> >> > >> > > > > > >> > I realize that Admin commands don't imply they will
> >> be used
> >> > >> > only
> >> > >> > > > in
> >> > >> > > > > > CLI
> >> > >> > > > > > >> > but,
> >> > >> > > > > > >> > it seems to me, CLI is a very important client of
> this
> >> > >> > feature.
> >> > >> > > In
> >> > >> > > > > > this
> >> > >> > > > > > >> > case,
> >> > >> > > > > > >> > seems logical, we would like to provide users with
> >> rich
> >> > >> > > experience
> >> > >> > > > > in
> >> > >> > > > > > >> terms
> >> > >> > > > > > >> > of
> >> > >> > > > > > >> > getting results / errors of the executed commands.
> >> Usually
> >> > >> we
> >> > >> > > > supply
> >> > >> > > > > > >> with
> >> > >> > > > > > >> > responses only errorCode, which looks very limiting,
> >> in case
> >> > >> > of
> >> > >> > > > CLI
> >> > >> > > > > we
> >> > >> > > > > > >> may
> >> > >> > > > > > >> > want to print human readable error description.
> >> > >> > > > > > >> >
> >> > >> > > > > > >> > So, taking into account previous item about
> batching,
> >> what
> >> > >> do
> >> > >> > > you
> >> > >> > > > > > think
> >> > >> > > > > > >> > about
> >> > >> > > > > > >> > having smth like:
> >> > >> > > > > > >> >
> >> > >> > > > > > >> > ('create' doesn't support regexp)
> >> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions Replicas
> >> > >> > > > > ReplicaAssignment
> >> > >> > > > > > >> > [Config]
> >> > >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
> >> > >> > > > > > >> >   ErrorCode => int16
> >> > >> > > > > > >> >   ErrorDescription => string (empty if successful)
> >> > >> > > > > > >> >
> >> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> >> > >> > > ReplicaAssignment
> >> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> >> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
> >> ErrorDescription]
> >> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
> >> > >> > > > > > >> >   CommandErrorCode => int16
> >> > >> > > > > > >> >   CommandErrorDescription => string (nonempty in
> case
> >> of
> >> > >> fatal
> >> > >> > > > > error,
> >> > >> > > > > > >> e.g.
> >> > >> > > > > > >> > we couldn't get topics by regexp)
> >> > >> > > > > > >> >
> >> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> >> > >> > > > > > >> > DescribeTopicResponse -> [TopicName TopicDescription
> >> > >> ErrorCode
> >> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> >> CommandErrorDescription
> >> > >> > > > > > >> >
> >> > >> > > > > > >> > Also, any thoughts about our discussion regarding
> >> re-routing
> >> > >> > > > > facility?
> >> > >> > > > > > >> In
> >> > >> > > > > > >> > my
> >> > >> > > > > > >> > understanding, it is like between augmenting
> >> > >> > > TopicMetadataRequest
> >> > >> > > > > > >> > (to include at least controllerId) and implementing
> >> new
> >> > >> > generic
> >> > >> > > > > > >> re-routing
> >> > >> > > > > > >> > facility so sending messages to controller will be
> >> handled
> >> > >> by
> >> > >> > > it.
> >> > >> > > > > > >> >
> >> > >> > > > > > >> > Thanks,
> >> > >> > > > > > >> > Andrii Biletskyi
> >> > >> > > > > > >> >
> >> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> >> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> >> > >> > > > > > >> >
> >> > >> > > > > > >> > > @Guozhang:
> >> > >> > > > > > >> > > Thanks for your comments, I've answered some of
> >> those. The
> >> > >> > > main
> >> > >> > > > > > thing
> >> > >> > > > > > >> is
> >> > >> > > > > > >> > > having merged request for
> >> create-alter-delete-describe - I
> >> > >> > > have
> >> > >> > > > > some
> >> > >> > > > > > >> > > concerns about this approach.
> >> > >> > > > > > >> > >
> >> > >> > > > > > >> > > @*Jay*:
> >> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is also
> >> one of
> >> > >> > the
> >> > >> > > > > > >> concerns.
> >> > >> > > > > > >> > We
> >> > >> > > > > > >> > > can solve it if we implement re-routing facility.
> >> But I
> >> > >> > agree
> >> > >> > > > with
> >> > >> > > > > > >> > > Guozhang - it will make clients' internals a
> little
> >> bit
> >> > >> > easier
> >> > >> > > > but
> >> > >> > > > > > >> this
> >> > >> > > > > > >> > > seems to be a complex logic to implement and
> >> support then.
> >> > >> > > > > > Especially
> >> > >> > > > > > >> for
> >> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing later
> >> for
> >> > >> these
> >> > >> > > > > > >> requests).
> >> > >> > > > > > >> > > Also people will tend to avoid this re-routing
> >> facility
> >> > >> and
> >> > >> > > hold
> >> > >> > > > > > local
> >> > >> > > > > > >> > > cluster cache to ensure their high-priority
> requests
> >> > >> (which
> >> > >> > > some
> >> > >> > > > > of
> >> > >> > > > > > >> the
> >> > >> > > > > > >> > > admin requests are) not sent to some busy broker
> >> where
> >> > >> they
> >> > >> > > wait
> >> > >> > > > > to
> >> > >> > > > > > be
> >> > >> > > > > > >> > > routed to the correct one.
> >> > >> > > > > > >> > > As pointed out by Jun here (
> >> > >> > > > > > >> > >
> >> > >> > > > > > >> >
> >> > >> > > > > > >>
> >> > >> > > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >>
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> >> > >> > > > > > >> > )
> >> > >> > > > > > >> > > to solve the issue we might introduce a message
> >> type to
> >> > >> get
> >> > >> > > > > cluster
> >> > >> > > > > > >> > state.
> >> > >> > > > > > >> > > But I agree we can just update
> >> TopicMetadataResponse to
> >> > >> > > include
> >> > >> > > > > > >> > > controllerId (and probably smth else).
> >> > >> > > > > > >> > > What are you thougths?
> >> > >> > > > > > >> > >
> >> > >> > > > > > >> > > Thanks,
> >> > >> > > > > > >> > > Andrii
> >> > >> > > > > > >> > >
> >> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> >> > >> > > > > wangguoz@gmail.com>
> >> > >> > > > > > >> > wrote:
> >> > >> > > > > > >> > >
> >> > >> > > > > > >> > >> I think for the topics commands we can actually
> >> merge
> >> > >> > > > > > >> > >> create/alter/delete/describe as one request type
> >> since
> >> > >> > their
> >> > >> > > > > > formats
> >> > >> > > > > > >> are
> >> > >> > > > > > >> > >> very much similar, and keep list-topics and
> others
> >> like
> >> > >> > > > > > >> > >> partition-reassignment /
> preferred-leader-election
> >> as
> >> > >> > > separate
> >> > >> > > > > > >> request
> >> > >> > > > > > >> > >> types, I also left some other comments on the RB
> (
> >> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> >> > >> > > > > > >> > >>
> >> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> >> > >> > > > jay.kreps@gmail.com>
> >> > >> > > > > > >> wrote:
> >> > >> > > > > > >> > >>
> >> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to just
> >> have
> >> > >> one
> >> > >> > > "do
> >> > >> > > > > > admin
> >> > >> > > > > > >> > >> stuff"
> >> > >> > > > > > >> > >> > command that has the union of all parameters.
> >> > >> > > > > > >> > >> >
> >> > >> > > > > > >> > >> > What I am saying is that command line tools are
> >> one
> >> > >> > client
> >> > >> > > of
> >> > >> > > > > the
> >> > >> > > > > > >> > >> > administrative apis, but these will be used in
> a
> >> number
> >> > >> > of
> >> > >> > > > > > >> scenarios
> >> > >> > > > > > >> > so
> >> > >> > > > > > >> > >> > they should make logical sense even in the
> >> absence of
> >> > >> the
> >> > >> > > > > command
> >> > >> > > > > > >> line
> >> > >> > > > > > >> > >> > tool. Hence comments like trying to clarify the
> >> > >> > > relationship
> >> > >> > > > > > >> between
> >> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds
> >> of
> >> > >> things
> >> > >> > > > > really
> >> > >> > > > > > >> need
> >> > >> > > > > > >> > >> to be
> >> > >> > > > > > >> > >> > thought through.
> >> > >> > > > > > >> > >> >
> >> > >> > > > > > >> > >> > Hope that makes sense.
> >> > >> > > > > > >> > >> >
> >> > >> > > > > > >> > >> > -Jay
> >> > >> > > > > > >> > >> >
> >> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
> >> Biletskyi <
> >> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> >> > >> > > > > > >> > >> >
> >> > >> > > > > > >> > >> > > Jay,
> >> > >> > > > > > >> > >> > >
> >> > >> > > > > > >> > >> > > Thanks for answering. You understood
> >> correctly, most
> >> > >> of
> >> > >> > > my
> >> > >> > > > > > >> comments
> >> > >> > > > > > >> > >> were
> >> > >> > > > > > >> > >> > > related to your point 1) - about "well
> >> thought-out"
> >> > >> > apis.
> >> > >> > > > > Also,
> >> > >> > > > > > >> yes,
> >> > >> > > > > > >> > >> as I
> >> > >> > > > > > >> > >> > > understood we would like to introduce a
> single
> >> > >> unified
> >> > >> > > CLI
> >> > >> > > > > tool
> >> > >> > > > > > >> with
> >> > >> > > > > > >> > >> > > centralized server-side request handling for
> >> lots of
> >> > >> > > > existing
> >> > >> > > > > > >> ones
> >> > >> > > > > > >> > >> (incl.
> >> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> >> > >> ReassignPartitions,
> >> > >> > > smth
> >> > >> > > > > > else
> >> > >> > > > > > >> if
> >> > >> > > > > > >> > >> added
> >> > >> > > > > > >> > >> > > in future). In our previous discussion (
> >> > >> > > > > > >> > >> > >
> >> https://issues.apache.org/jira/browse/KAFKA-1694
> )
>
> >> > >> > people
> >> > >> > > > > said
> >> > >> > > > > > >> > they'd
> >> > >> > > > > > >> > >> > > rather
> >> > >> > > > > > >> > >> > > have a separate message for each command, so,
> >> yes,
> >> > >> this
> >> > >> > > > way I
> >> > >> > > > > > >> came
> >> > >> > > > > > >> > to
> >> > >> > > > > > >> > >> 1-1
> >> > >> > > > > > >> > >> > > mapping between commands in the tool and
> >> protocol
> >> > >> > > > additions.
> >> > >> > > &g t; > > But
>
> >> > >> > > > > > >> I
> >> > >> > > > > > >> > >> might
> >> > >> > > > > > >> > >> > be
> >> > >> > > > > > >> > >> > > wrong.
> >> > >> > > > > > >> > >> > > At the end I just try to start discussion how
> >> at
> >> > >> least
> >> > >> > > > > > generally
> >> > >> > > > > > >> > this
> >> > >> > > > > > >> > >> > > protocol should look like.
> >> > >> > > > > > >> > >> > >
> >> > >> > > > > > >> > >> > > Thanks,
> >> > >> > > > > > >> > >> > > Andrii
> >> > >> > > > > > >> > >> > >
> >> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
> >> > >> > > > > > jay.kreps@gmail.com
> >> > >> > > > > > >> >
> >> > >> > > > > > >> > >> wrote:
> >> > >> > > > > > >> > >> > >
> >> > >> > > > > > >> > >> > > > Hey Andrii,
> >> > >> > > > > > >> > >> > > >
> >> > >> > > > > > >> > >> > > > To answer your earlier question we just
> >> really
> >> > >> can't
> >> > >> > be
> >> > >> > > > > > adding
> >> > >> > > > > > >> any
> >> > >> > > > > > >> > >> more
> >> > >> > > > > > >> > >> > > > scala protocol objects. These things are
> >> super hard
> >> > >> > to
> >> > >> > > > > > maintain
> >> > >> > > > > > >> > >> because
> >> > >> > > > > > >> > >> > > > they hand code the byte parsing and don't
> >> have good
> >> > >> > > > > > versioning
> >> > >> > > > > > >> > >> support.
> >> > >> > > > > > >> > >> > > > Since we are already planning on converting
> >> we
> >> > >> > > definitely
> >> > >> > > > > > don't
> >> > >> > > > > > >> > >> want to
> >> > >> > > > > > >> > >> > > add
> >> > >> > > > > > >> > >> > > > a ton more of these--they are total tech
> >> debt.
> >> > >> > > > > > >> > >> > > >
> >> > >> > > > > > >> > >> > > > What does it mean that the changes are
> >> isolated
> >> > >> from
> >> > >> > > the
> >> > >> > > > > > >> current
> >> > >> > > > > > >> > >> code
> >> > >> > > > > > >> > >> > > base?
> >> > >> > > > > > >> > >> > > >
> >> > >> > > > > > >> > >> > > > I actually didn't understand the remaining
> >> > >> comments,
> >> > >> > > > which
> >> > >> > > > > of
> >> > >> > > > > > >> the
> >> > >> > > > > > >> > >> > points
> >> > >> > > > > > >> > >> > > > are you responding to?
> >> > >> > > > > > >> > >> > > >
> >> > >> > > > > > >> > >> > > > Maybe one sticking point here is that it
> >> seems like
> >> > >> > you
> >> > >> > > > > want
> >> > >> > > > > > to
> >> > >> > > > > > >> > make
> >> > >> > > > > > >> > >> > some
> >> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1
> mapping
> >> > >> between
> >> > >> > > > > > commands
> >> > >> > > > > > >> you
> >> > >> > > > > > >> > >> > > imagine
> >> > >> > > > > > >> > >> > > > in the tool and protocol additions. I want
> >> to make
> >> > >> > sure
> >> > >> > > > we
> >> > >> > > > > > >> don't
> >> > >> > > > > > >> > do
> >> > >> > > > > > >> > >> > that.
> >> > >> > > > > > >> > >> > > > The protocol needs to be really really well
> >> thought
> >> > >> > out
> >> > >> > > > > > against
> >> > >> > > > > > >> > many
> >> > >> > > > > > >> > >> > use
> >> > >> > > > > > >> > >> > > > cases so it should make perfect logical
> >> sense in
> >> > >> the
> >> > >> > > > > absence
> >> > >> > > > > > of
> >> > >> > > > > > >> > >> knowing
> >> > >> > > > > > >> > >> > > the
> >> > >> > > > > > >> > >> > > > command line tool, right?
> >> > >> > > > > > >> > >> > > >
> >> > >> > > > > > >> > >> > > > -Jay
> >> > >> > > > > > >> > >> > > >
> >> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii
> >> Biletskyi
> >> > >> <
> >> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> >> > >> > > > > > >> > >> > > >
> >> > >> > > > > > >> > >> > > > > Hey Jay,
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > > > I would like to continue this discussion
> >> as it
> >> > >> seem
> >> > >> > > > there
> >> > >> > > > > > is
> >> > >> > > > > > >> no
> >> > >> > > > > > >> > >> > > progress
> >> > >> > > > > > >> > >> > > > > here.
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > > > First of all, could you please explain
> >> what did
> >> > >> you
> >> > >> > > > mean
> >> > >> > > > > in
> >> > >> > > > > > >> 2?
> >> > >> > > > > > >> > How
> >> > >> > > > > > >> > >> > > > exactly
> >> > >> > > > > > >> > >> > > > > are we going to migrate to the new java
> >> protocol
> >> > >> > > > > > definitions.
> >> > >> > > > > > >> > And
> >> > >> > > > > > >> > >> why
> >> > >> > > > > > >> > >> > > > it's
> >> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > > > I agree with you, this feature includes
> >> lots of
> >> > >> > > stuff,
> >> > >> > > > > but
> >> > >> > > > > > >> > >> thankfully
> >> > >> > > > > > >> > >> > > > > almost all changes are isolated from the
> >> current
> >> > >> > code
> >> > >> > > > > base,
> >> > >> > > > > > >> > >> > > > > so the main thing, I think, we need to
> >> agree is
> >> > >> > RQ/RP
> >> > >> > > > > > format.
> >> > >> > > > > > >> > >> > > > > So how can we start discussion about the
> >> concrete
> >> > >> > > > > messages
> >> > >> > > > > > >> > format?
> >> > >> > > > > > >> > >> > > > > Can we take (
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > >
> >> > >> > > > > > >> > >> > >
> >> > >> > > > > > >> > >> >
> >> > >> > > > > > >> > >>
> >> > >> > > > > > >> >
> >> > >> > > > > > >>
> >> > >> > > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
>
> >> > >> > > > > > >> > >> > > > > )
> >> > >> > > > > > >> > >> > > > > as starting point?
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > > > We had some doubts earlier whether it
> worth
> >> > >> > > introducing
> >> > >> > > > > one
> >> > >> > > > > > >> > >> generic
> >> > >> > > > > > >> > >> > > Admin
> >> > >> > > > > > >> > >> > > > > Request for all commands (
> >> > >> > > > > > >> > >> > > >
> >> https://issues.apache.org/jira/browse/KAFKA-1694
> >> > >> > > > > > >> > >> > > > > )
> >> > >> > > > > > >> > >> > > > > but then everybody agreed it would be
> >> better to
> >> > >> > have
> >> > >> > > > > > separate
> >> > >> > > > > > >> > >> message
> >> > >> > > > > > >> > >> > > for
> >> > >> > > > > > >> > >> > > > > each admin command. The Request part is
> >> really
> >> > >> > > dictated
> >> > >> > > > > > from
> >> > >> > > > > > >> the
> >> > >> > > > > > >> > >> > > command
> >> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so
> >> the
> >> > >> > proposed
> >> > >> > > > > > version
> >> > >> > > > > > >> > >> should
> >> > >> > > > > > >> > >> > be
> >> > >> > > > > > >> > >> > > > > fine (let's put aside for now remarks
> about
> >> > >> > Optional
> >> > >> > > > > type,
> >> > >> > > > > > >> > >> batching,
> >> > >> > > > > > >> > >> > > > > configs normalization - I agree with all
> of
> >> > >> them).
> >> > >> > > > > > >> > >> > > > > So the second part is Response. I see
> >> there are
> >> > >> two
> >> > >> > > > cases
> >> > >> > > > > > >> here.
> >> > >> > > > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ;
> >> b)
> >> > >> "Get"
> >> > >> > > > > > requests -
> >> > >> > > > > > >> > >> > > > > List/Describe...
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > > > a) should only hold request result
> >> (regardless
> >> > >> what
> >> > >> > > we
> >> > >> > > > > > decide
> >> > >> > > > > > >> > >> about
> >> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
> execution).
> >> > >> > > > > > >> > >> > > > > Usually we provide error code in response
> >> but
> >> > >> since
> >> > >> > > we
> >> > >> > > > > will
> >> > >> > > > > > >> use
> >> > >> > > > > > >> > >> this
> >> > >> > > > > > >> > >> > in
> >> > >> > > > > > >> > >> > > > > interactive shell we need some human
> >> readable
> >> > >> error
> >> > >> > > > > > >> description
> >> > >> > > > > > >> > -
> >> > >> > > > > > >> > >> so
> >> > >> > > > > > >> > >> > I
> >> > >> > > > > > >> > >> > > > > added errorDesription field where you can
> >> at
> >> > >> least
> >> > >> > > > leave
> >> > >> > > > > > >> > >> > > > > exception.getMessage.
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > > > b) in addition to previous item message
> >> should
> >> > >> hold
> >> > >> > > > > command
> >> > >> > > > > > >> > >> specific
> >> > >> > > > > > >> > >> > > > > response data. We can discuss in detail
> >> each of
> >> > >> > them
> >> > >> > > > but
> >> > >> > > > > > >> let's
> >> > >> > > > > > >> > for
> >> > >> > > > > > >> > >> > now
> >> > >> > > > > > >> > >> > > > > agree about the overall pattern.
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > > > Thanks,
> >> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay
> Kreps
> >> <
> >> > >> > > > > > >> jay.kreps@gmail.com
> >> > >> > > > > > >> > >
> >> > >> > > > > > >> > >> > > wrote:
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > > > > Hey Joe,
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > This is great. A few comments on KIP-4
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > 1. This is much needed functionality,
> >> but there
> >> > >> > > are a
> >> > >> > > > > lot
> >> > >> > > > > > >> of
> >> > >> > > > > > >> > >> the so
> >> > >> > > > > > >> > >> > > > let's
> >> > >> > > > > > >> > >> > > > > > really think these protocols through.
> We
> >> really
> >> > >> > > want
> >> > >> > > > to
> >> > >> > > > > > >> end up
> >> > >> > > > > > >> > >> > with a
> >> > >> > > > > > >> > >> > > > set
> >> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis.
> >> For this
> >> > >> > > > reason I
> >> > >> > > > > > >> think
> >> > >> > > > > > >> > >> it is
> >> > >> > > > > > >> > >> > > > > really
> >> > >> > > > > > >> > >> > > > > > important to think through the end
> state
> >> even
> >> > >> if
> >> > >> > > that
> >> > >> > > > > > >> includes
> >> > >> > > > > > >> > >> APIs
> >> > >> > > > > > >> > >> > > we
> >> > >> > > > > > >> > >> > > > > > won't implement in the first phase.
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait
> until
> >> we
> >> > >> have
> >> > >> > > > > switched
> >> > >> > > > > > >> the
> >> > >> > > > > > >> > >> > server
> >> > >> > > > > > >> > >> > > > over
> >> > >> > > > > > >> > >> > > > > > to the new java protocol definitions.
> If
> >> we add
> >> > >> > > > upteen
> >> > >> > > > > > >> more ad
> >> > >> > > > > > >> > >> hoc
> >> > >> > > > > > >> > >> > > > scala
> >> > >> > > > > > >> > >> > > > > > objects that is just generating more
> >> work for
> >> > >> the
> >> > >> > > > > > >> conversion
> >> > >> > > > > > >> > we
> >> > >> > > > > > >> > >> > know
> >> > >> > > > > > >> > >> > > we
> >> > >> > > > > > >> > >> > > > > > have to do.
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new type
> of
> >> > >> > optional
> >> > >> > > > > > >> parameter.
> >> > >> > > > > > >> > >> This
> >> > >> > > > > > >> > >> > is
> >> > >> > > > > > >> > >> > > > > > inconsistent with everything else in
> the
> >> > >> protocol
> >> > >> > > > where
> >> > >> > > > > > we
> >> > >> > > > > > >> use
> >> > >> > > > > > >> > >> -1
> >> > >> > > > > > >> > >> > or
> >> > >> > > > > > >> > >> > > > some
> >> > >> > > > > > >> > >> > > > > > other marker value. You could argue
> >> either way
> >> > >> > but
> >> > >> > > > > let's
> >> > >> > > > > > >> stick
> >> > >> > > > > > >> > >> with
> >> > >> > > > > > >> > >> > > > that
> >> > >> > > > > > >> > >> > > > > > for consistency. For clients that
> >> implemented
> >> > >> the
> >> > >> > > > > > protocol
> >> > >> > > > > > >> in
> >> > >> > > > > > >> > a
> >> > >> > > > > > >> > >> > > better
> >> > >> > > > > > >> > >> > > > > way
> >> > >> > > > > > >> > >> > > > > > than our scala code these basic
> >> primitives are
> >> > >> > hard
> >> > >> > > > to
> >> > >> > > > > > >> change.
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
> >> duplicate
> >> > >> > > > > > >> > TopicMetadataRequest
> >> > >> > > > > > >> > >> > > which
> >> > >> > > > > > >> > >> > > > > has
> >> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I
> think
> >> we
> >> > >> > should
> >> > >> > > > > rename
> >> > >> > > > > > >> that
> >> > >> > > > > > >> > >> > > request
> >> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> >> > >> MetadataRequest)
> >> > >> > > and
> >> > >> > > > > > >> include
> >> > >> > > > > > >> > >> the id
> >> > >> > > > > > >> > >> > > of
> >> > >> > > > > > >> > >> > > > > the
> >> > >> > > > > > >> > >> > > > > > controller. Or are there other things
> we
> >> could
> >> > >> > add
> >> > >> > > > > here?
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to make a
> >> lot of
> >> > >> > > > requests
> >> > >> > > > > > that
> >> > >> > > > > > >> > can
> >> > >> > > > > > >> > >> > only
> >> > >> > > > > > >> > >> > > go
> >> > >> > > > > > >> > >> > > > > to
> >> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of
> >> burden for
> >> > >> > > > client
> >> > >> > > > > > >> > >> > > implementations
> >> > >> > > > > > >> > >> > > > > (it
> >> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can fail
> >> in many
> >> > >> > > parts
> >> > >> > > > > so
> >> > >> > > > > > it
> >> > >> > > > > > >> > >> ends up
> >> > >> > > > > > >> > >> > > > > being a
> >> > >> > > > > > >> > >> > > > > > full state machine to do right). I
> think
> >> we
> >> > >> > should
> >> > >> > > > > > consider
> >> > >> > > > > > >> > >> making
> >> > >> > > > > > >> > >> > > > admin
> >> > >> > > > > > >> > >> > > > > > commands and ideally as many of the
> >> other apis
> >> > >> as
> >> > >> > > > > > possible
> >> > >> > > > > > >> > >> > available
> >> > >> > > > > > >> > >> > > on
> >> > >> > > > > > >> > >> > > > > all
> >> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
> >> controller on
> >> > >> > the
> >> > >> > > > > broker
> >> > >> > > > > > >> > side.
> >> > >> > > > > > >> > >> > > Perhaps
> >> > >> > > > > > >> > >> > > > > > there would be a general way to
> >> encapsulate
> >> > >> this
> >> > >> > > > > > re-routing
> >> > >> > > > > > >> > >> > behavior.
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > 6. We should probably normalize the key
> >> value
> >> > >> > pairs
> >> > >> > > > > used
> >> > >> > > > > > >> for
> >> > >> > > > > > >> > >> > configs
> >> > >> > > > > > >> > >> > > > > rather
> >> > >> > > > > > >> > >> > > > > > than embedding a new formatting. So two
> >> strings
> >> > >> > > > rather
> >> > >> > > > > > than
> >> > >> > > > > > >> > one
> >> > >> > > > > > >> > >> > with
> >> > >> > > > > > >> > >> > > an
> >> > >> > > > > > >> > >> > > > > > internal equals sign.
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs
> >> that the
> >> > >> > > > command
> >> > >> > > > > > has
> >> > >> > > > > > >> > >> begun or
> >> > >> > > > > > >> > >> > > > that
> >> > >> > > > > > >> > >> > > > > > the command has been completed? It is a
> >> lot
> >> > >> more
> >> > >> > > > usable
> >> > >> > > > > > if
> >> > >> > > > > > >> the
> >> > >> > > > > > >> > >> > > command
> >> > >> > > > > > >> > >> > > > > has
> >> > >> > > > > > >> > >> > > > > > been completed so you know that if you
> >> create a
> >> > >> > > topic
> >> > >> > > > > and
> >> > >> > > > > > >> then
> >> > >> > > > > > >> > >> > > publish
> >> > >> > > > > > >> > >> > > > to
> >> > >> > > > > > >> > >> > > > > > it you won't get an exception about
> >> there being
> >> > >> > no
> >> > >> > > > such
> >> > >> > > > > > >> topic.
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
> >> duplicate a
> >> > >> lot
> >> > >> > > of
> >> > >> > > > > > stuff
> >> > >> > > > > > >> in
> >> > >> > > > > > >> > >> the
> >> > >> > > > > > >> > >> > > > > metadata
> >> > >> > > > > > >> > >> > > > > > request. Is there a reason to give back
> >> topics
> >> > >> > > marked
> >> > >> > > > > for
> >> > >> > > > > > >> > >> > deletion? I
> >> > >> > > > > > >> > >> > > > > feel
> >> > >> > > > > > >> > >> > > > > > like if we just make the post-condition
> >> of the
> >> > >> > > delete
> >> > >> > > > > > >> command
> >> > >> > > > > > >> > be
> >> > >> > > > > > >> > >> > that
> >> > >> > > > > > >> > >> > > > the
> >> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid of
> >> the need
> >> > >> > for
> >> > >> > > > this
> >> > >> > > > > > >> right?
> >> > >> > > > > > >> > >> And
> >> > >> > > > > > >> > >> > it
> >> > >> > > > > > >> > >> > > > > will
> >> > >> > > > > > >> > >> > > > > > be much more intuitive.
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > 9. Should we consider batching these
> >> requests?
> >> > >> We
> >> > >> > > > have
> >> > >> > > > > > >> > generally
> >> > >> > > > > > >> > >> > > tried
> >> > >> > > > > > >> > >> > > > to
> >> > >> > > > > > >> > >> > > > > > allow multiple operations to be
> batched.
> >> My
> >> > >> > > suspicion
> >> > >> > > > > is
> >> > >> > > > > > >> that
> >> > >> > > > > > >> > >> > without
> >> > >> > > > > > >> > >> > > > > this
> >> > >> > > > > > >> > >> > > > > > we will get a lot of code that does
> >> something
> >> > >> > like
> >> > >> > > > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
> >> > >> > > > > > >> > >> > > > > >       adminClient.describeTopic(topic)
> >> > >> > > > > > >> > >> > > > > > this code will work great when you test
> >> on 5
> >> > >> > topics
> >> > >> > > > but
> >> > >> > > > > > >> not do
> >> > >> > > > > > >> > >> as
> >> > >> > > > > > >> > >> > > well
> >> > >> > > > > > >> > >> > > > if
> >> > >> > > > > > >> > >> > > > > > you have 50k.
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss how
> >> we want
> >> > >> to
> >> > >> > > > > expose
> >> > >> > > > > > a
> >> > >> > > > > > >> > >> > > programmatic
> >> > >> > > > > > >> > >> > > > > JVM
> >> > >> > > > > > >> > >> > > > > > client api for these operations.
> >> Currently
> >> > >> people
> >> > >> > > > rely
> >> > >> > > > > on
> >> > >> > > > > > >> > >> > AdminUtils
> >> > >> > > > > > >> > >> > > > > which
> >> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we probably
> >> need
> >> > >> > > another
> >> > >> > > > > > client
> >> > >> > > > > > >> > >> under
> >> > >> > > > > > >> > >> > > > > clients/
> >> > >> > > > > > >> > >> > > > > > that exposes administrative
> >> functionality. We
> >> > >> > will
> >> > >> > > > need
> >> > >> > > > > > >> this
> >> > >> > > > > > >> > >> just
> >> > >> > > > > > >> > >> > to
> >> > >> > > > > > >> > >> > > > > > properly test the new apis, I suspect.
> We
> >> > >> should
> >> > >> > > > figure
> >> > >> > > > > > out
> >> > >> > > > > > >> > that
> >> > >> > > > > > >> > >> > API.
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > 11. The other information that would be
> >> really
> >> > >> > > useful
> >> > >> > > > > to
> >> > >> > > > > > >> get
> >> > >> > > > > > >> > >> would
> >> > >> > > > > > >> > >> > be
> >> > >> > > > > > >> > >> > > > > > information about partitions--how much
> >> data is
> >> > >> in
> >> > >> > > the
> >> > >> > > > > > >> > partition,
> >> > >> > > > > > >> > >> > what
> >> > >> > > > > > >> > >> > > > are
> >> > >> > > > > > >> > >> > > > > > the segment offsets, what is the
> log-end
> >> offset
> >> > >> > > (i.e.
> >> > >> > > > > > last
> >> > >> > > > > > >> > >> offset),
> >> > >> > > > > > >> > >> > > > what
> >> > >> > > > > > >> > >> > > > > is
> >> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think that
> >> done
> >> > >> > right
> >> > >> > > > this
> >> > >> > > > > > >> would
> >> > >> > > > > > >> > be
> >> > >> > > > > > >> > >> > the
> >> > >> > > > > > >> > >> > > > > > successor to the very awkward
> >> OffsetRequest we
> >> > >> > have
> >> > >> > > > > > today.
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > -Jay
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe
> >> Stein <
> >> > >> > > > > > >> > >> joe.stein@stealth.ly>
> >> > >> > > > > > >> > >> > > > > wrote:
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> >> > >> > > > > > >> > >> > > > > > >
> >> > >> > > > > > >> > >> > > > > > >
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > >
> >> > >> > > > > > >> > >> > >
> >> > >> > > > > > >> > >> >
> >> > >> > > > > > >> > >>
> >> > >> > > > > > >> >
> >> > >> > > > > > >>
> >> > >> > > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> > >> > > > > > >> > >> > > > > > >
> >> > >> > > > > > >> > >> > > > > > > JIRA
> >> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> >> > >> > > > > > >> > >> > > > > > >
> >> > >> > > > > > >> > >> > > > > > >
> >> /*******************************************
> >> > >> > > > > > >> > >> > > > > > >  Joe Stein
> >> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> >> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
> >> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> >> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> >> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> >> > >> > > > > > >> > >> > > >
> >> > >> > > > > > >> > >> > > > > > >
> >> ********************************************/
> >> > >> > > > > > >> > >> > > > > > >
> >> > >> > > > > > >> > >> > > > > >
> >> > >> > > > > > >> > >> > > > >
> >> > >> > > > > > >> > >> > > >
> >> > >> > > > > > >> > >> > >
> >> > >> > > > > > >> > >> >
> >> > >> > > > > > >> > >>
> >> > >> > > > > > >> > >>
> >> > >> > > > > > >> > >>
> >> > >> > > > > > >> > >> --
> >> > >> > > > > > >> > >> -- Guozhang
> >> > >> > > > > > >> > >>
> >> > >> > > > > > >> > >
> >> > >> > > > > > >> > >
> >> > >> > > > > > >> >
> >> > >> > > > > > >>
> >> > >> > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >> >
> >> > >> >
> >> > >> > --
> >> > >> > Jeff Holoman
> >> > >> > Systems Engineer
> >> > >> >
> >> > >>
> >>
> >>
> >
> >
> > --
> > -- Guozhang
> >
>
>
>
> --
> -- Guozhang
>
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Joel Koshy <jj...@gmail.com>.
Yes that is what I was alluding to when I said that if we finally do
request rerouting in Kafka then the field would add little to no
value. I wasn't sure if we agreed that we _will_ do rerouting or
whether we agreed to evaluate it (KAFKA-1912). Andrii can you update
the KIP with this?

Thanks,

Joel

On Wed, Mar 18, 2015 at 02:55:00PM -0700, Jun Rao wrote:
> Joel, Andril,
> 
> I think we agreed that those admin requests can be issued to any broker.
> Because of that, there doesn't seem to be a strong need to know the
> controller. So, perhaps we can proceed by not making any change to the
> format of TMR right now. When we start using create topic request in the
> producer, we will need a new version of TMR that doesn't trigger auto topic
> creation. But that can be done later.
> 
> As a first cut implementation, I think the broker can just write to ZK
> directly for
> createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> requests, instead of forwarding them to the controller. This will simplify
> the implementation on the broker side.
> 
> Thanks,
> 
> Jun
> 
> 
> On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <jj...@gmail.com> wrote:
> 
> > (Thanks Andrii for the summary)
> >
> > For (1) yes we will circle back on that shortly after syncing up in
> > person. I think it is close to getting committed although development
> > for KAFKA-1927 can probably begin without it.
> >
> > There is one more item we covered at the hangout. i.e., whether we
> > want to add the coordinator to the topic metadata response or provide
> > a clearer ClusterMetadataRequest.
> >
> > There are two reasons I think we should try and avoid adding the
> > field:
> > - It is irrelevant to topic metadata
> > - If we finally do request rerouting in Kafka then the field would add
> >   little to no value. (It still helps to have a separate
> >   ClusterMetadataRequest to query for cluster-wide information such as
> >   'which broker is the controller?' as Joe mentioned.)
> >
> > I think it would be cleaner to have an explicit ClusterMetadataRequest
> > that you can send to any broker in order to obtain the controller (and
> > in the future possibly other cluster-wide information). I think the
> > main argument against doing this and instead adding it to the topic
> > metadata response was convenience - i.e., you don't have to discover
> > the controller in advance. However, I don't see much actual
> > benefit/convenience in this and in fact think it is a non-issue. Let
> > me know if I'm overlooking something here.
> >
> > As an example, say we need to initiate partition reassignment by
> > issuing the new ReassignPartitionsRequest to the controller (assume we
> > already have the desired manual partition assignment).  If we are to
> > augment topic metadata response then the flow be something like this :
> >
> > - Issue topic metadata request to any broker (and discover the
> >   controller
> > - Connect to controller if required (i.e., if the broker above !=
> >   controller)
> > - Issue the partition reassignment request to the controller.
> >
> > With an explicit cluster metadata request it would be:
> > - Issue cluster metadata request to any broker
> > - Connect to controller if required (i.e., if the broker above !=
> >   controller)
> > - Issue the partition reassignment request
> >
> > So it seems to add little practical value and bloats topic metadata
> > response with an irrelevant detail.
> >
> > The other angle to this is the following - is it a matter of naming?
> > Should we just rename topic metadata request/response to just
> > MetadataRequest/Response and add cluster metadata to it? By that same
> > token should we also allow querying for the consumer coordinator (and
> > in future transaction coordinator) as well? This leads to a bloated
> > request which isn't very appealing and altogether confusing.
> >
> > Thanks,
> >
> > Joel
> >
> > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > > Andri,
> > >
> > > Thanks for the summary.
> > >
> > > 1. I just realized that in order to start working on KAFKA-1927, we will
> > > need to merge the changes to OffsetCommitRequest (from 0.8.2) to trunk.
> > > This is planned to be done as part of KAFKA-1634. So, we will need
> > Guozhang
> > > and Joel's help to wrap this up.
> > >
> > > 2. Thinking about this a bit more, if the semantic of those "write"
> > > requests is async (i.e., after the client gets a response, it just means
> > > that the operation is initiated, but not necessarily completed), we don't
> > > really need to forward the requests to the controller. Instead, the
> > > receiving broker can just write the operation to ZK as the admin command
> > > line tool previously does. This will simplify the implementation.
> > >
> > > 8. There is another implementation detail for describe topic. Ideally, we
> > > want to read the topic config from the broker cache, instead of
> > ZooKeeper.
> > > Currently, every broker reads the topic-level config for all topics.
> > > However, it ignores those for topics not hosted on itself. So, we may
> > need
> > > to change TopicConfigManager a bit so that it caches the configs for all
> > > topics.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Guys,
> > > >
> > > > Thanks for a great discussion!
> > > > Here are the actions points:
> > > >
> > > > 1. Q: Get rid of all scala requests objects, use java protocol
> > definitions.
> > > >     A: Gwen kindly took that (KAFKA-1927). It's important to speed up
> > > > review procedure
> > > >          there since this ticket blocks other important changes.
> > > >
> > > > 2. Q: Generic re-reroute facility vs client maintaining cluster state.
> > > >     A: Jay has added pseudo code to KAFKA-1912 - need to consider
> > whether
> > > > this will be
> > > >         easy to implement as a server-side feature (comments are
> > > > welcomed!).
> > > >
> > > > 3. Q: Controller field in wire protocol.
> > > >     A: This might be useful for clients, add this to
> > TopicMetadataResponse
> > > > (already in KIP).
> > > >
> > > > 4. Q: Decoupling topic creation from TMR.
> > > >     A: I will add proposed by Jun solution (using clientId for that)
> > to the
> > > > KIP.
> > > >
> > > > 5. Q: Bumping new versions of TMR vs grabbing all protocol changes in
> > one
> > > > version.
> > > >     A: It was decided to try to gather all changes to protocol (before
> > > > release).
> > > >         In case of TMR it worth checking: KAFKA-2020 and KIP-13
> > (quotas)
> > > >
> > > > 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
> > > >     A: Use jackson for that, /tools project is a separate jar so
> > shouldn't
> > > > be a big deal.
> > > >
> > > > 7.  Q: VerifyReassingPartitions vs generic status check command.
> > > >      A: For long-running requests like reassign partitions *progress*
> > check
> > > > request is useful,
> > > >          it makes sense to introduce it.
> > > >
> > > >  Please add, correct me if I missed something.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Joel,
> > > > >
> > > > > You are right, I removed ClusterMetadata because we have partially
> > > > > what we need in TopicMetadata. Also, as Jay pointed out earlier, we
> > > > > would like to have "orthogonal" API, but at the same time we need
> > > > > to be backward compatible.
> > > > >
> > > > > But I like your idea and even have some other arguments for this
> > option:
> > > > > There is also DescribeTopicRequest which was proposed in this KIP,
> > > > > it returns topic configs, partitions, replication factor plus
> > partition
> > > > > ISR, ASR,
> > > > > leader replica. The later part is really already there in
> > > > > TopicMetadataRequest.
> > > > > So again we'll have to add stuff to TMR, not to duplicate some info
> > in
> > > > > newly added requests. However, this way we'll end up with "monster"
> > > > > request which returns cluster metadata, topic replication and config
> > info
> > > > > plus partition replication data. Seems logical to split TMR to
> > > > > - ClusterMetadata (brokers + controller, maybe smth else)
> > > > > - TopicMetadata (topic info + partition details)
> > > > > But since current TMR is involved in lots of places (including
> > network
> > > > > client,
> > > > > as I understand) this might be very serious change and it probably
> > makes
> > > > > sense to stick with current approach.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > >
> > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > > > >
> > > > >> I may be missing some context but hopefully this will also be
> > covered
> > > > >> today: I thought the earlier proposal where there was an explicit
> > > > >> ClusterMetadata request was clearer and explicit. During the course
> > of
> > > > >> this thread I think the conclusion was that the main need was for
> > > > >> controller information and that can be rolled into the topic
> > metadata
> > > > >> response but that seems a bit irrelevant to topic metadata. FWIW I
> > > > >> think the full broker-list is also irrelevant to topic metadata, but
> > > > >> it is already there and in use. I think there is still room for an
> > > > >> explicit ClusterMetadata request since there may be other
> > > > >> cluster-level information that we may want to add over time (and
> > that
> > > > >> have nothing to do with topic metadata).
> > > > >>
> > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
> > > > >> > Jun,
> > > > >> >
> > > > >> > 101. Okay, if you say that such use case is important. I also
> > think
> > > > >> > using clientId for these purposes is fine - if we already have
> > this
> > > > >> field
> > > > >> > as part of all Wire protocol messages, why not use that.
> > > > >> > I will update KIP-4 page if nobody has other ideas (which may
> > come up
> > > > >> > during the call today).
> > > > >> >
> > > > >> > 102.1 Agree, I'll update the KIP accordingly. I think we can add
> > new,
> > > > >> > fine-grained error codes if some error code received in specific
> > case
> > > > >> > won't give enough context to return a descriptive error message
> > for
> > > > >> user.
> > > > >> >
> > > > >> > Look forward to discussing all outstanding issues in detail today
> > > > during
> > > > >> > the call.
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Andrii Biletskyi
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > >> >
> > > > >> > > 101. There may be a use case where you only want the topics to
> > be
> > > > >> created
> > > > >> > > manually by admins. Currently, you can do that by disabling auto
> > > > topic
> > > > >> > > creation and issue topic creation from the TopicCommand. If we
> > > > >> disable auto
> > > > >> > > topic creation completely on the broker and don't have a way to
> > > > >> distinguish
> > > > >> > > between topic creation requests from the regular clients and the
> > > > >> admin, we
> > > > >> > > can't support manual topic creation any more. I was thinking
> > that
> > > > >> another
> > > > >> > > way of distinguishing the clients making the topic creation
> > requests
> > > > >> is
> > > > >> > > using clientId. For example, the admin tool can set it to
> > something
> > > > >> like
> > > > >> > > admin and the broker can treat that clientId specially.
> > > > >> > >
> > > > >> > > Also, there is a related discussion in KAFKA-2020. Currently,
> > we do
> > > > >> the
> > > > >> > > following in TopicMetadataResponse:
> > > > >> > >
> > > > >> > > 1. If leader is not available, we set the partition level error
> > code
> > > > >> to
> > > > >> > > LeaderNotAvailable.
> > > > >> > > 2. If a non-leader replica is not available, we take that
> > replica
> > > > out
> > > > >> of
> > > > >> > > the assigned replica list and isr in the response. As an
> > indication
> > > > >> for
> > > > >> > > doing that, we set the partition level error code to
> > > > >> ReplicaNotAvailable.
> > > > >> > >
> > > > >> > > This has a few problems. First, ReplicaNotAvailable probably
> > > > >> shouldn't be
> > > > >> > > an error, at least for the normal producer/consumer clients that
> > > > just
> > > > >> want
> > > > >> > > to find out the leader. Second, it can happen that both the
> > leader
> > > > and
> > > > >> > > another replica are not available at the same time. There is no
> > > > error
> > > > >> code
> > > > >> > > to indicate both. Third, even if a replica is not available,
> > it's
> > > > >> still
> > > > >> > > useful to return its replica id since some clients (e.g. admin
> > tool)
> > > > >> may
> > > > >> > > still make use of it.
> > > > >> > >
> > > > >> > > One way to address this issue is to always return the replica
> > id for
> > > > >> > > leader, assigned replicas, and isr regardless of whether the
> > > > >> corresponding
> > > > >> > > broker is live or not. Since we also return the list of live
> > > > brokers,
> > > > >> the
> > > > >> > > client can figure out whether a leader or a replica is live or
> > not
> > > > >> and act
> > > > >> > > accordingly. This way, we don't need to set the partition level
> > > > error
> > > > >> code
> > > > >> > > when the leader or a replica is not available. This doesn't
> > change
> > > > >> the wire
> > > > >> > > protocol, but does change the semantics. Since we are evolving
> > the
> > > > >> protocol
> > > > >> > > of TopicMetadataRequest here, we can potentially piggyback the
> > > > change.
> > > > >> > >
> > > > >> > > 102.1 For those types of errors due to invalid input, shouldn't
> > we
> > > > >> just
> > > > >> > > guard it at parameter validation time and throw
> > > > >> InvalidArgumentException
> > > > >> > > without even sending the request to the broker?
> > > > >> > >
> > > > >> > > Thanks,
> > > > >> > >
> > > > >> > > Jun
> > > > >> > >
> > > > >> > >
> > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > >
> > > > >> > > > Jun,
> > > > >> > > >
> > > > >> > > > Answering your questions:
> > > > >> > > >
> > > > >> > > > 101. If I understand you correctly, you are saying future
> > producer
> > > > >> > > versions
> > > > >> > > > (which
> > > > >> > > > will be ported to TMR_V1) won't be able to automatically
> > create
> > > > >> topic (if
> > > > >> > > > we
> > > > >> > > > unconditionally remove topic creation from there). But we
> > need to
> > > > >> this
> > > > >> > > > preserve logic.
> > > > >> > > > Ok, about your proposal: I'm not a big fan too, when it comes
> > to
> > > > >> > > > differentiating
> > > > >> > > > clients directly in protocol schema. And also I'm not sure I
> > > > >> understand
> > > > >> > > at
> > > > >> > > > all why
> > > > >> > > > auto.create.topics.enable is a server side configuration. Can
> > we
> > > > >> > > deprecate
> > > > >> > > > this setting
> > > > >> > > > in future versions, add this setting to producer and based on
> > that
> > > > >> upon
> > > > >> > > > receiving
> > > > >> > > > UnknownTopic create topic explicitly by a separate producer
> > call
> > > > via
> > > > >> > > > adminClient?
> > > > >> > > >
> > > > >> > > > 102.1. Hm, yes. It's because we want to support batching and
> > at
> > > > the
> > > > >> same
> > > > >> > > > time we
> > > > >> > > > want to give descriptive error messages for clients. Since
> > > > >> AdminClient
> > > > >> > > > holds the context
> > > > >> > > > to construct such messages (e.g. AdminClient layer can know
> > that
> > > > >> > > > InvalidArgumentsCode
> > > > >> > > > means two cases: either invalid number - e.g. -1; or
> > > > >> replication-factor
> > > > >> > > was
> > > > >> > > > provided while
> > > > >> > > > partitions argument wasn't) - I wrapped responses in
> > Exceptions.
> > > > >> But I'm
> > > > >> > > > open to any
> > > > >> > > > other ideas, this was just initial version.
> > > > >> > > > 102.2. Yes, I agree. I'll change that to probably some other
> > dto.
> > > > >> > > >
> > > > >> > > > Thanks,
> > > > >> > > > Andrii Biletskyi
> > > > >> > > >
> > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > >> > > >
> > > > >> > > > > Andrii,
> > > > >> > > > >
> > > > >> > > > > 101. That's what I was thinking too, but it may not be that
> > > > >> simple. In
> > > > >> > > > > TopicMetadataRequest_V1,
> > > > >> > > > > we can let it not trigger auto topic creation. Then, in the
> > > > >> producer
> > > > >> > > > side,
> > > > >> > > > > if it gets an UnknownTopicException, it can explicitly
> > issue a
> > > > >> > > > > createTopicRequest for auto topic creation. On the consumer
> > > > side,
> > > > >> it
> > > > >> > > will
> > > > >> > > > > never issue createTopicRequest. This works when auto topic
> > > > >> creation is
> > > > >> > > > > enabled on the broker side. However, I am not sure how
> > things
> > > > >> will work
> > > > >> > > > > when auto topic creation is disabled on the broker side. In
> > this
> > > > >> case,
> > > > >> > > we
> > > > >> > > > > want to have a way to manually create a topic, potentially
> > > > through
> > > > >> > > admin
> > > > >> > > > > commands. However, then we need a way to distinguish
> > > > >> createTopicRequest
> > > > >> > > > > issued from the producer clients and the admin tools. May
> > be we
> > > > >> can
> > > > >> > > add a
> > > > >> > > > > new field in createTopicRequest and set it differently in
> > the
> > > > >> producer
> > > > >> > > > > client and the admin client. However, I am not sure if
> > that's
> > > > the
> > > > >> best
> > > > >> > > > > approach.
> > > > >> > > > >
> > > > >> > > > > 2. Yes, refactoring existing requests is a non-trivial
> > amount of
> > > > >> work.
> > > > >> > > I
> > > > >> > > > > posted some comments in KAFKA-1927. We will probably have
> > to fix
> > > > >> > > > KAFKA-1927
> > > > >> > > > > first, before adding the new logic in KAFKA-1694.
> > Otherwise, the
> > > > >> > > changes
> > > > >> > > > > will be too big.
> > > > >> > > > >
> > > > >> > > > > 102. About the AdminClient:
> > > > >> > > > > 102.1. It's a bit weird that we return exception in the
> > api. It
> > > > >> seems
> > > > >> > > > that
> > > > >> > > > > we should either return error code or throw an exception
> > when
> > > > >> getting
> > > > >> > > the
> > > > >> > > > > response state.
> > > > >> > > > > 102.2. We probably shouldn't explicitly use the request
> > object
> > > > in
> > > > >> the
> > > > >> > > > api.
> > > > >> > > > > Not every request evolution requires an api change.
> > > > >> > > > >
> > > > >> > > > > Thanks,
> > > > >> > > > >
> > > > >> > > > > Jun
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > > > >
> > > > >> > > > > > Jun,
> > > > >> > > > > >
> > > > >> > > > > > Thanks for you comments. Answers inline:
> > > > >> > > > > >
> > > > >> > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > > >> > > > > > > ReassignPartitionRequest,
> > > > >> > > > > > > and PartitionsSerialized that are represented as a
> > string,
> > > > but
> > > > >> > > > contain
> > > > >> > > > > > > composite structures in json. Could we flatten them out
> > > > >> directly in
> > > > >> > > > the
> > > > >> > > > > > > protocol definition as arrays/records?
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > Yes, now with Admin Client this looks a bit weird. My
> > initial
> > > > >> > > > motivation
> > > > >> > > > > > was:
> > > > >> > > > > > ReassignPartitionCommand accepts input in json, we want to
> > > > >> remain
> > > > >> > > > tools'
> > > > >> > > > > > interfaces unchanged, where possible.
> > > > >> > > > > > If we port it to deserialized format, in CLI (/tools
> > project)
> > > > >> we will
> > > > >> > > > > have
> > > > >> > > > > > to add some
> > > > >> > > > > > json library since /tools is written in java and we'll
> > need to
> > > > >> > > > > deserialize
> > > > >> > > > > > json file
> > > > >> > > > > > provided by a user. Can we quickly agree on what this
> > library
> > > > >> should
> > > > >> > > be
> > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > >> > > > > >
> > > > >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> > > > >> creation?
> > > > >> > > > This
> > > > >> > > > > > > will be a bit weird now that we have a separate topic
> > > > >> creation api.
> > > > >> > > > > Have
> > > > >> > > > > > > you thought about how the new createTopicRequest and
> > > > >> > > > > TopicMetadataRequest
> > > > >> > > > > > > v1 will be used in the producer/consumer client, in
> > addition
> > > > >> to
> > > > >> > > admin
> > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > >> TopicMetadataRequest
> > > > >> > > from
> > > > >> > > > > the
> > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > I agree, this strange logic should be fixed. I'm not
> > confident
> > > > >> in
> > > > >> > > this
> > > > >> > > > > > Kafka part so
> > > > >> > > > > > correct me if I'm wrong, but it doesn't look like a hard
> > thing
> > > > >> to
> > > > >> > > do, I
> > > > >> > > > > > think we can
> > > > >> > > > > > leverage AdminClient for that in Producer and
> > unconditionally
> > > > >> remove
> > > > >> > > > > topic
> > > > >> > > > > > creation from the TopicMetadataRequest_V1.
> > > > >> > > > > >
> > > > >> > > > > > 2. I think Jay meant getting rid of scala classes
> > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > >> HeartbeatResponseAndHeader. We
> > > > >> > > did
> > > > >> > > > > > that
> > > > >> > > > > > > as a stop-gap thing when adding the new requests for the
> > > > >> consumers.
> > > > >> > > > > > > However, the long term plan is to get rid of all those
> > and
> > > > >> just
> > > > >> > > reuse
> > > > >> > > > > the
> > > > >> > > > > > > java request/response in the client. Since this KIP
> > proposes
> > > > >> to
> > > > >> > > add a
> > > > >> > > > > > > significant number of new requests, perhaps we should
> > bite
> > > > the
> > > > >> > > bullet
> > > > >> > > > > to
> > > > >> > > > > > > clean up the existing scala requests first before
> > adding new
> > > > >> ones?
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > > > Yes, looks like I misunderstood the point of
> > > > >> ...RequestAndHeader.
> > > > >> > > > Okay, I
> > > > >> > > > > > will
> > > > >> > > > > > rework that. The only thing is that I don't see any
> > example
> > > > how
> > > > >> it
> > > > >> > > was
> > > > >> > > > > done
> > > > >> > > > > > for at
> > > > >> > > > > > least one existing protocol message. Thus, as I
> > understand, I
> > > > >> have to
> > > > >> > > > > think
> > > > >> > > > > > how we
> > > > >> > > > > > are going to do it.
> > > > >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
> > > > reasonable,
> > > > >> but
> > > > >> > > if
> > > > >> > > > > it's
> > > > >> > > > > > an *obligatory*
> > > > >> > > > > > requirement to have Admin KIP done, I'm afraid this can
> > be a
> > > > >> serious
> > > > >> > > > > > blocker for us.
> > > > >> > > > > > There are 13 protocol messages and all that would require
> > not
> > > > >> only
> > > > >> > > unit
> > > > >> > > > > > tests but quite
> > > > >> > > > > > intensive manual testing, no? I'm afraid I'm not the
> > right guy
> > > > >> to
> > > > >> > > cover
> > > > >> > > > > > pretty much all
> > > > >> > > > > > Kafka core internals :). Let me know your thoughts on this
> > > > >> item. Btw
> > > > >> > > > > there
> > > > >> > > > > > is a ticket to
> > > > >> > > > > > follow-up this issue (
> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > >> > > > ).
> > > > >> > > > > >
> > > > >> > > > > > Thanks,
> > > > >> > > > > > Andrii Biletskyi
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <
> > jun@confluent.io>
> > > > >> wrote:
> > > > >> > > > > >
> > > > >> > > > > > > Andrii,
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > A few more comments.
> > > > >> > > > > > >
> > > > >> > > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > > >> > > > > > > ReassignPartitionRequest,
> > > > >> > > > > > > and PartitionsSerialized that are represented as a
> > string,
> > > > but
> > > > >> > > > contain
> > > > >> > > > > > > composite structures in json. Could we flatten them out
> > > > >> directly in
> > > > >> > > > the
> > > > >> > > > > > > protocol definition as arrays/records?
> > > > >> > > > > > >
> > > > >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto
> > topic
> > > > >> > > creation?
> > > > >> > > > > This
> > > > >> > > > > > > will be a bit weird now that we have a separate topic
> > > > >> creation api.
> > > > >> > > > > Have
> > > > >> > > > > > > you thought about how the new createTopicRequest and
> > > > >> > > > > TopicMetadataRequest
> > > > >> > > > > > > v1 will be used in the producer/consumer client, in
> > addition
> > > > >> to
> > > > >> > > admin
> > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > >> TopicMetadataRequest
> > > > >> > > from
> > > > >> > > > > the
> > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > >> > > > > > >
> > > > >> > > > > > > 2. I think Jay meant getting rid of scala classes
> > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > >> HeartbeatResponseAndHeader. We
> > > > >> > > did
> > > > >> > > > > > that
> > > > >> > > > > > > as a stop-gap thing when adding the new requests for the
> > > > >> consumers.
> > > > >> > > > > > > However, the long term plan is to get rid of all those
> > and
> > > > >> just
> > > > >> > > reuse
> > > > >> > > > > the
> > > > >> > > > > > > java request/response in the client. Since this KIP
> > proposes
> > > > >> to
> > > > >> > > add a
> > > > >> > > > > > > significant number of new requests, perhaps we should
> > bite
> > > > the
> > > > >> > > bullet
> > > > >> > > > > to
> > > > >> > > > > > > clean up the existing scala requests first before
> > adding new
> > > > >> ones?
> > > > >> > > > > > >
> > > > >> > > > > > > Thanks,
> > > > >> > > > > > >
> > > > >> > > > > > > Jun
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > > > > > >
> > > > >> > > > > > > > Hi,
> > > > >> > > > > > > >
> > > > >> > > > > > > > As said above - I list again all comments from this
> > thread
> > > > >> so we
> > > > >> > > > > > > > can see what's left and finalize all pending issues.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Comments from Jay:
> > > > >> > > > > > > > 1. This is much needed functionality, but there are a
> > lot
> > > > >> of the
> > > > >> > > so
> > > > >> > > > > > let's
> > > > >> > > > > > > > really think these protocols through. We really want
> > to
> > > > end
> > > > >> up
> > > > >> > > > with a
> > > > >> > > > > > set
> > > > >> > > > > > > > of well thought-out, orthoganol apis. For this reason
> > I
> > > > >> think it
> > > > >> > > is
> > > > >> > > > > > > really
> > > > >> > > > > > > > important to think through the end state even if that
> > > > >> includes
> > > > >> > > APIs
> > > > >> > > > > we
> > > > >> > > > > > > > won't implement in the first phase.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Definitely behind this. Would appreciate if there
> > are
> > > > >> concrete
> > > > >> > > > > > > comments
> > > > >> > > > > > > > how this can be improved.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 2. Let's please please please wait until we have
> > switched
> > > > >> the
> > > > >> > > > server
> > > > >> > > > > > over
> > > > >> > > > > > > > to the new java protocol definitions. If we add upteen
> > > > more
> > > > >> ad
> > > > >> > > hoc
> > > > >> > > > > > scala
> > > > >> > > > > > > > objects that is just generating more work for the
> > > > >> conversion we
> > > > >> > > > know
> > > > >> > > > > we
> > > > >> > > > > > > > have to do.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Fixed in the latest patch - removed scala protocol
> > > > >> classes.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 3. This proposal introduces a new type of optional
> > > > >> parameter.
> > > > >> > > This
> > > > >> > > > is
> > > > >> > > > > > > > inconsistent with everything else in the protocol
> > where we
> > > > >> use -1
> > > > >> > > > or
> > > > >> > > > > > some
> > > > >> > > > > > > > other marker value. You could argue either way but
> > let's
> > > > >> stick
> > > > >> > > with
> > > > >> > > > > > that
> > > > >> > > > > > > > for consistency. For clients that implemented the
> > protocol
> > > > >> in a
> > > > >> > > > > better
> > > > >> > > > > > > way
> > > > >> > > > > > > > than our scala code these basic primitives are hard to
> > > > >> change.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf type
> > and
> > > > >> changed
> > > > >> > > > > > protocol
> > > > >> > > > > > > > accordingly.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> > > > >> TopicMetadataRequest
> > > > >> > > > > which
> > > > >> > > > > > > has
> > > > >> > > > > > > > brokers, topics, and partitions. I think we should
> > rename
> > > > >> that
> > > > >> > > > > request
> > > > >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest) and
> > > > >> include the
> > > > >> > > id
> > > > >> > > > > of
> > > > >> > > > > > > the
> > > > >> > > > > > > > controller. Or are there other things we could add
> > here?
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: I agree. Updated the KIP. Let's extends
> > TopicMetadata
> > > > to
> > > > >> > > > version 2
> > > > >> > > > > > and
> > > > >> > > > > > > > include controller.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 5. We have a tendency to try to make a lot of requests
> > > > that
> > > > >> can
> > > > >> > > > only
> > > > >> > > > > go
> > > > >> > > > > > > to
> > > > >> > > > > > > > particular nodes. This adds a lot of burden for client
> > > > >> > > > > implementations
> > > > >> > > > > > > (it
> > > > >> > > > > > > > sounds easy but each discovery can fail in many parts
> > so
> > > > it
> > > > >> ends
> > > > >> > > up
> > > > >> > > > > > > being a
> > > > >> > > > > > > > full state machine to do right). I think we should
> > > > consider
> > > > >> > > making
> > > > >> > > > > > admin
> > > > >> > > > > > > > commands and ideally as many of the other apis as
> > possible
> > > > >> > > > available
> > > > >> > > > > on
> > > > >> > > > > > > all
> > > > >> > > > > > > > brokers and just redirect to the controller on the
> > broker
> > > > >> side.
> > > > >> > > > > Perhaps
> > > > >> > > > > > > > there would be a general way to encapsulate this
> > > > re-routing
> > > > >> > > > behavior.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: It's a very interesting idea, but seems there are
> > some
> > > > >> > > concerns
> > > > >> > > > > > about
> > > > >> > > > > > > > this
> > > > >> > > > > > > > feature (like performance considerations, how this
> > will
> > > > >> > > complicate
> > > > >> > > > > > server
> > > > >> > > > > > > > etc).
> > > > >> > > > > > > > I believe this shouldn't be a blocker. If this
> > feature is
> > > > >> > > > implemented
> > > > >> > > > > > at
> > > > >> > > > > > > > some
> > > > >> > > > > > > > point it won't affect Admin changes - at least no
> > changes
> > > > to
> > > > >> > > public
> > > > >> > > > > API
> > > > >> > > > > > > > will be required.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 6. We should probably normalize the key value pairs
> > used
> > > > for
> > > > >> > > > configs
> > > > >> > > > > > > rather
> > > > >> > > > > > > > than embedding a new formatting. So two strings rather
> > > > than
> > > > >> one
> > > > >> > > > with
> > > > >> > > > > an
> > > > >> > > > > > > > internal equals sign.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Fixed in the latest patch - normalized configs and
> > > > >> changed
> > > > >> > > > > protocol
> > > > >> > > > > > > > accordingly.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 7. Is the postcondition of these APIs that the
> > command has
> > > > >> begun
> > > > >> > > or
> > > > >> > > > > > that
> > > > >> > > > > > > > the command has been completed? It is a lot more
> > usable if
> > > > >> the
> > > > >> > > > > command
> > > > >> > > > > > > has
> > > > >> > > > > > > > been completed so you know that if you create a topic
> > and
> > > > >> then
> > > > >> > > > > publish
> > > > >> > > > > > to
> > > > >> > > > > > > > it you won't get an exception about there being no
> > such
> > > > >> topic.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: For long running requests (like reassign
> > partitions) -
> > > > >> the
> > > > >> > > post
> > > > >> > > > > > > > condition is
> > > > >> > > > > > > > command has begun - so we don't block the client. In
> > case
> > > > >> of your
> > > > >> > > > > > > example -
> > > > >> > > > > > > > topic commands, this will be refactored and topic
> > commands
> > > > >> will
> > > > >> > > be
> > > > >> > > > > > > executed
> > > > >> > > > > > > > immediately, since the Controller will serve Admin
> > > > requests
> > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > >> > > > > > > >
> > > > >> > > > > > > > 8. Describe topic and list topics duplicate a lot of
> > stuff
> > > > >> in the
> > > > >> > > > > > > metadata
> > > > >> > > > > > > > request. Is there a reason to give back topics marked
> > for
> > > > >> > > > deletion? I
> > > > >> > > > > > > feel
> > > > >> > > > > > > > like if we just make the post-condition of the delete
> > > > >> command be
> > > > >> > > > that
> > > > >> > > > > > the
> > > > >> > > > > > > > topic is deleted that will get rid of the need for
> > this
> > > > >> right?
> > > > >> > > And
> > > > >> > > > it
> > > > >> > > > > > > will
> > > > >> > > > > > > > be much more intuitive.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Fixed in the latest patch - removed topics marked
> > for
> > > > >> deletion
> > > > >> > > > in
> > > > >> > > > > > > > ListTopicsRequest.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 9. Should we consider batching these requests? We have
> > > > >> generally
> > > > >> > > > > tried
> > > > >> > > > > > to
> > > > >> > > > > > > > allow multiple operations to be batched. My suspicion
> > is
> > > > >> that
> > > > >> > > > without
> > > > >> > > > > > > this
> > > > >> > > > > > > > we will get a lot of code that does something like
> > > > >> > > > > > > >    for(topic: adminClient.listTopics())
> > > > >> > > > > > > >       adminClient.describeTopic(topic)
> > > > >> > > > > > > > this code will work great when you test on 5 topics
> > but
> > > > not
> > > > >> do as
> > > > >> > > > > well
> > > > >> > > > > > if
> > > > >> > > > > > > > you have 50k.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Updated the KIP - please check "Topic Admin Schema"
> > > > >> section.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 10. I think we should also discuss how we want to
> > expose a
> > > > >> > > > > programmatic
> > > > >> > > > > > > JVM
> > > > >> > > > > > > > client api for these operations. Currently people
> > rely on
> > > > >> > > > AdminUtils
> > > > >> > > > > > > which
> > > > >> > > > > > > > is totally sketchy. I think we probably need another
> > > > client
> > > > >> under
> > > > >> > > > > > > clients/
> > > > >> > > > > > > > that exposes administrative functionality. We will
> > need
> > > > >> this just
> > > > >> > > > to
> > > > >> > > > > > > > properly test the new apis, I suspect. We should
> > figure
> > > > out
> > > > >> that
> > > > >> > > > API.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Updated the KIP - please check "Admin Client"
> > section
> > > > >> with an
> > > > >> > > > > > initial
> > > > >> > > > > > > > API proposal.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 11. The other information that would be really useful
> > to
> > > > get
> > > > >> > > would
> > > > >> > > > be
> > > > >> > > > > > > > information about partitions--how much data is in the
> > > > >> partition,
> > > > >> > > > what
> > > > >> > > > > > are
> > > > >> > > > > > > > the segment offsets, what is the log-end offset (i.e.
> > last
> > > > >> > > offset),
> > > > >> > > > > > what
> > > > >> > > > > > > is
> > > > >> > > > > > > > the compaction point, etc. I think that done right
> > this
> > > > >> would be
> > > > >> > > > the
> > > > >> > > > > > > > successor to the very awkward OffsetRequest we have
> > today.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest
> > > > >> patch. I
> > > > >> > > > > believe
> > > > >> > > > > > > > this should
> > > > >> > > > > > > > be resolved in a separate KIP / jira ticket.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 12. Generally we can do good error handling without
> > > > needing
> > > > >> > > custom
> > > > >> > > > > > > > server-side
> > > > >> > > > > > > > messages. I.e. generally the client has the context to
> > > > know
> > > > >> that
> > > > >> > > if
> > > > >> > > > > it
> > > > >> > > > > > > got
> > > > >> > > > > > > > an error that the topic doesn't exist to say "Topic X
> > > > >> doesn't
> > > > >> > > > exist"
> > > > >> > > > > > > rather
> > > > >> > > > > > > > than "error code 14" (or whatever). Maybe there are
> > > > specific
> > > > >> > > cases
> > > > >> > > > > > where
> > > > >> > > > > > > > this is hard? If we want to add server-side error
> > messages
> > > > >> we
> > > > >> > > > really
> > > > >> > > > > do
> > > > >> > > > > > > > need to do this in a consistent way across the
> > protocol.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Updated the KIP - please check "Protocol Errors"
> > > > >> section. I
> > > > >> > > > added
> > > > >> > > > > > the
> > > > >> > > > > > > > comprehensive, fine-grained list of error codes.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Comments from Guozhang:
> > > > >> > > > > > > > 13. Describe topic request: it would be great to go
> > beyond
> > > > >> just
> > > > >> > > > > > batching
> > > > >> > > > > > > on
> > > > >> > > > > > > > topic name regex for this request. For example, a very
> > > > >> common use
> > > > >> > > > > case
> > > > >> > > > > > of
> > > > >> > > > > > > > the topic command is to list all topics whose config
> > A's
> > > > >> value is
> > > > >> > > > B.
> > > > >> > > > > > With
> > > > >> > > > > > > > topic name regex then we have to first retrieve
> > __all__
> > > > >> topics's
> > > > >> > > > > > > > description info and then filter at the client end,
> > which
> > > > >> will
> > > > >> > > be a
> > > > >> > > > > > huge
> > > > >> > > > > > > > burden on ZK.
> > > > >> > > > > > > > AND
> > > > >> > > > > > > > 14. Config K-Vs in create topic: this is related to
> > the
> > > > >> previous
> > > > >> > > > > point;
> > > > >> > > > > > > > maybe we can add another metadata K-V or just a
> > metadata
> > > > >> string
> > > > >> > > > along
> > > > >> > > > > > > side
> > > > >> > > > > > > > with config K-V in create topic like we did for offset
> > > > >> commit
> > > > >> > > > > request.
> > > > >> > > > > > > This
> > > > >> > > > > > > > field can be quite useful in storing information like
> > > > >> "owner" of
> > > > >> > > > the
> > > > >> > > > > > > topic
> > > > >> > > > > > > > who issue the create command, etc, which is quite
> > > > important
> > > > >> for a
> > > > >> > > > > > > > multi-tenant setting. Then in the describe topic
> > request
> > > > we
> > > > >> can
> > > > >> > > > also
> > > > >> > > > > > > batch
> > > > >> > > > > > > > on regex of the metadata field.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: As discussed it is very interesting but can be
> > > > >> implemented
> > > > >> > > later
> > > > >> > > > > > after
> > > > >> > > > > > > > we have some basic functionality there.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 15. Today all the admin operations are async in the
> > sense
> > > > >> that
> > > > >> > > > > command
> > > > >> > > > > > > will
> > > > >> > > > > > > > return once it is written in ZK, and that is why we
> > need
> > > > >> extra
> > > > >> > > > > > > verification
> > > > >> > > > > > > > like testUtil.waitForTopicCreated() / verify partition
> > > > >> > > reassignment
> > > > >> > > > > > > > request, etc. With admin requests we could add a flag
> > to
> > > > >> enable /
> > > > >> > > > > > disable
> > > > >> > > > > > > > synchronous requests; when it is turned on, the
> > response
> > > > >> will not
> > > > >> > > > > > return
> > > > >> > > > > > > > until the request has been completed. And for async
> > > > >> requests we
> > > > >> > > can
> > > > >> > > > > > add a
> > > > >> > > > > > > > "token" field in the response, and then only need a
> > > > general
> > > > >> > > "admin
> > > > >> > > > > > > > verification request" with the given token to check
> > if the
> > > > >> async
> > > > >> > > > > > request
> > > > >> > > > > > > > has been completed.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: I see your point. My idea was to provide specific
> > > > >> > > > Verify...Request
> > > > >> > > > > > per
> > > > >> > > > > > > > each
> > > > >> > > > > > > > long running request, where needed. We can do it the
> > way
> > > > you
> > > > >> > > > suggest.
> > > > >> > > > > > The
> > > > >> > > > > > > > only
> > > > >> > > > > > > > concern is that introducing a token we again will make
> > > > >> schema
> > > > >> > > > > > "dynamic".
> > > > >> > > > > > > We
> > > > >> > > > > > > > wanted
> > > > >> > > > > > > > to do similar thing introducing single AdminRequest
> > for
> > > > all
> > > > >> topic
> > > > >> > > > > > > commands
> > > > >> > > > > > > > but rejected
> > > > >> > > > > > > > this idea because we wanted to have schema defined. So
> > > > this
> > > > >> is
> > > > >> > > > more a
> > > > >> > > > > > > > choice between:
> > > > >> > > > > > > > a) have fixed schema but introduce each time new
> > > > >> Verify...Request
> > > > >> > > > for
> > > > >> > > > > > > > long-running requests
> > > > >> > > > > > > > b) use one request for verification but generalize it
> > with
> > > > >> token
> > > > >> > > > > > > > I'm fine with whatever decision community come to.
> > Just
> > > > let
> > > > >> me
> > > > >> > > know
> > > > >> > > > > > your
> > > > >> > > > > > > > thoughts.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Comment from Gwen:
> > > > >> > > > > > > > 16. Specifically for ownership, I think the plan is
> > to add
> > > > >> ACL
> > > > >> > > (it
> > > > >> > > > > > sounds
> > > > >> > > > > > > > like you are describing ACL) via an external system
> > > > (Argus,
> > > > >> > > > Sentry).
> > > > >> > > > > > > > I remember KIP-11 described this, but I can't find
> > the KIP
> > > > >> any
> > > > >> > > > > longer.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Okay, no problem. Not sure though how we are going
> > to
> > > > >> handle
> > > > >> > > it.
> > > > >> > > > > > Wait
> > > > >> > > > > > > > which KIP
> > > > >> > > > > > > > will be committed first and include changes to
> > > > >> TopicMetadata from
> > > > >> > > > the
> > > > >> > > > > > > later
> > > > >> > > > > > > > one?
> > > > >> > > > > > > > Anyway, I added this note to "Open Questions" section
> > so
> > > > we
> > > > >> don't
> > > > >> > > > > miss
> > > > >> > > > > > > this
> > > > >> > > > > > > > piece.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Thanks,
> > > > >> > > > > > > > Andrii Biletskyi
> > > > >> > > > > > > >
> > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > > > > > > >
> > > > >> > > > > > > > > Hi all,
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > Today I uploaded the patch that covers some of the
> > > > >> discussed
> > > > >> > > and
> > > > >> > > > > > agreed
> > > > >> > > > > > > > > items:
> > > > >> > > > > > > > > - removed MaybeOf optional type
> > > > >> > > > > > > > > - switched to java protocol definitions
> > > > >> > > > > > > > > - simplified messages (normalized configs, removed
> > topic
> > > > >> marked
> > > > >> > > > for
> > > > >> > > > > > > > > deletion)
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > I also updated the KIP-4 with respective changes and
> > > > >> wrote down
> > > > >> > > > my
> > > > >> > > > > > > > > proposal for
> > > > >> > > > > > > > > pending items:
> > > > >> > > > > > > > > - Batch Admin Operations -> updated Wire Protocol
> > schema
> > > > >> > > proposal
> > > > >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
> > > > >> > > > TopicMetadataRequest
> > > > >> > > > > > > > > - Admin Client -> updated my initial proposal to
> > reflect
> > > > >> > > batching
> > > > >> > > > > > > > > - Error codes -> proposed fine-grained error code
> > > > instead
> > > > >> of
> > > > >> > > > > > > > > AdminRequestFailed
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > I will also send a separate email to cover all
> > comments
> > > > >> from
> > > > >> > > this
> > > > >> > > > > > > thread.
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > Thanks,
> > > > >> > > > > > > > > Andrii Biletskyi
> > > > >> > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> > > > >> > > > > gshapira@cloudera.com
> > > > >> > > > > > >
> > > > >> > > > > > > > > wrote:
> > > > >> > > > > > > > >
> > > > >> > > > > > > > >> Found KIP-11 (
> > > > >> > > > > > > > >>
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >>
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > >> > > > > > > > >> )
> > > > >> > > > > > > > >> It actually specifies changes to the Metadata
> > protocol,
> > > > >> so
> > > > >> > > > making
> > > > >> > > > > > sure
> > > > >> > > > > > > > >> both KIPs are consistent in this regard will be
> > good.
> > > > >> > > > > > > > >>
> > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> > > > >> > > > > > gshapira@cloudera.com
> > > > >> > > > > > > >
> > > > >> > > > > > > > >> wrote:
> > > > >> > > > > > > > >> > Specifically for ownership, I think the plan is
> > to
> > > > add
> > > > >> ACL
> > > > >> > > (it
> > > > >> > > > > > > sounds
> > > > >> > > > > > > > >> > like you are describing ACL) via an external
> > system
> > > > >> (Argus,
> > > > >> > > > > > Sentry).
> > > > >> > > > > > > > >> > I remember KIP-11 described this, but I can't
> > find
> > > > the
> > > > >> KIP
> > > > >> > > any
> > > > >> > > > > > > longer.
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting
> > > > >> information
> > > > >> > > that
> > > > >> > > > > > > already
> > > > >> > > > > > > > >> > exists from Kafka brokers, not on adding
> > information
> > > > >> that
> > > > >> > > > > perhaps
> > > > >> > > > > > > > >> > should exist but doesn't yet?
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> > Gwen
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> > > > >> > > > > > wangguoz@gmail.com>
> > > > >> > > > > > > > >> wrote:
> > > > >> > > > > > > > >> >> Folks,
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Just want to elaborate a bit more on the
> > > > create-topic
> > > > >> > > > metadata
> > > > >> > > > > > and
> > > > >> > > > > > > > >> batching
> > > > >> > > > > > > > >> >> describe-topic based on config / metadata in my
> > > > >> previous
> > > > >> > > > email
> > > > >> > > > > as
> > > > >> > > > > > > we
> > > > >> > > > > > > > >> work
> > > > >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to have
> > some
> > > > >> sort of
> > > > >> > > > > topic
> > > > >> > > > > > > > >> management
> > > > >> > > > > > > > >> >> mechanisms, which I think is quite important in
> > a
> > > > >> > > > multi-tenant
> > > > >> > > > > /
> > > > >> > > > > > > > cloud
> > > > >> > > > > > > > >> >> architecture: today anyone can create topics in
> > a
> > > > >> shared
> > > > >> > > > Kafka
> > > > >> > > > > > > > >> cluster, but
> > > > >> > > > > > > > >> >> there is no concept or "ownership" of topics
> > that
> > > > are
> > > > >> > > created
> > > > >> > > > > by
> > > > >> > > > > > > > >> different
> > > > >> > > > > > > > >> >> users. For example, at LinkedIn we basically
> > > > >> distinguish
> > > > >> > > > topic
> > > > >> > > > > > > owners
> > > > >> > > > > > > > >> via
> > > > >> > > > > > > > >> >> some casual topic name prefix, which is a bit
> > > > awkward
> > > > >> and
> > > > >> > > > does
> > > > >> > > > > > not
> > > > >> > > > > > > > fly
> > > > >> > > > > > > > >> as
> > > > >> > > > > > > > >> >> we scale our customers. It would be great to use
> > > > >> > > > > describe-topics
> > > > >> > > > > > > such
> > > > >> > > > > > > > >> as:
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Describe all topics that is created by me.
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Describe all topics whose retention time is
> > > > overriden
> > > > >> to X.
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Describe all topics whose writable group include
> > > > user
> > > > >> Y
> > > > >> > > (this
> > > > >> > > > > is
> > > > >> > > > > > > > >> related to
> > > > >> > > > > > > > >> >> authorization), etc..
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> One possible way to achieve this is to add a
> > > > metadata
> > > > >> file
> > > > >> > > in
> > > > >> > > > > the
> > > > >> > > > > > > > >> >> create-topic request, whose value will also be
> > > > >> written ZK
> > > > >> > > as
> > > > >> > > > we
> > > > >> > > > > > > > create
> > > > >> > > > > > > > >> the
> > > > >> > > > > > > > >> >> topic; then describe-topics can choose to batch
> > > > topics
> > > > >> > > based
> > > > >> > > > on
> > > > >> > > > > > 1)
> > > > >> > > > > > > > name
> > > > >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata
> > regex,
> > > > etc.
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Thoughts?
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Guozhang
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> > > > >> > > > > > wangguoz@gmail.com>
> > > > >> > > > > > > > >> wrote:
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >>> Thanks for the updated wiki. A few comments
> > below:
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> 1. Error description in response: I think if
> > some
> > > > >> > > errorCode
> > > > >> > > > > > could
> > > > >> > > > > > > > >> indicate
> > > > >> > > > > > > > >> >>> several different error cases then we should
> > really
> > > > >> change
> > > > >> > > > it
> > > > >> > > > > to
> > > > >> > > > > > > > >> multiple
> > > > >> > > > > > > > >> >>> codes. In general the errorCode itself would be
> > > > >> precise
> > > > >> > > and
> > > > >> > > > > > > > >> sufficient for
> > > > >> > > > > > > > >> >>> describing the server side errors.
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> 2. Describe topic request: it would be great
> > to go
> > > > >> beyond
> > > > >> > > > just
> > > > >> > > > > > > > >> batching on
> > > > >> > > > > > > > >> >>> topic name regex for this request. For
> > example, a
> > > > >> very
> > > > >> > > > common
> > > > >> > > > > > use
> > > > >> > > > > > > > >> case of
> > > > >> > > > > > > > >> >>> the topic command is to list all topics whose
> > > > config
> > > > >> A's
> > > > >> > > > value
> > > > >> > > > > > is
> > > > >> > > > > > > B.
> > > > >> > > > > > > > >> With
> > > > >> > > > > > > > >> >>> topic name regex then we have to first retrieve
> > > > >> __all__
> > > > >> > > > > topics's
> > > > >> > > > > > > > >> >>> description info and then filter at the client
> > end,
> > > > >> which
> > > > >> > > > will
> > > > >> > > > > > be
> > > > >> > > > > > > a
> > > > >> > > > > > > > >> huge
> > > > >> > > > > > > > >> >>> burden on ZK.
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is
> > related to
> > > > >> the
> > > > >> > > > > previous
> > > > >> > > > > > > > point;
> > > > >> > > > > > > > >> >>> maybe we can add another metadata K-V or just a
> > > > >> metadata
> > > > >> > > > > string
> > > > >> > > > > > > > along
> > > > >> > > > > > > > >> side
> > > > >> > > > > > > > >> >>> with config K-V in create topic like we did for
> > > > >> offset
> > > > >> > > > commit
> > > > >> > > > > > > > >> request. This
> > > > >> > > > > > > > >> >>> field can be quite useful in storing
> > information
> > > > like
> > > > >> > > > "owner"
> > > > >> > > > > of
> > > > >> > > > > > > the
> > > > >> > > > > > > > >> topic
> > > > >> > > > > > > > >> >>> who issue the create command, etc, which is
> > quite
> > > > >> > > important
> > > > >> > > > > for
> > > > >> > > > > > a
> > > > >> > > > > > > > >> >>> multi-tenant setting. Then in the describe
> > topic
> > > > >> request
> > > > >> > > we
> > > > >> > > > > can
> > > > >> > > > > > > also
> > > > >> > > > > > > > >> batch
> > > > >> > > > > > > > >> >>> on regex of the metadata field.
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> 4. Today all the admin operations are async in
> > the
> > > > >> sense
> > > > >> > > > that
> > > > >> > > > > > > > command
> > > > >> > > > > > > > >> will
> > > > >> > > > > > > > >> >>> return once it is written in ZK, and that is
> > why we
> > > > >> need
> > > > >> > > > extra
> > > > >> > > > > > > > >> verification
> > > > >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify
> > > > >> partition
> > > > >> > > > > > > reassignment
> > > > >> > > > > > > > >> >>> request, etc. With admin requests we could add
> > a
> > > > >> flag to
> > > > >> > > > > enable
> > > > >> > > > > > /
> > > > >> > > > > > > > >> disable
> > > > >> > > > > > > > >> >>> synchronous requests; when it is turned on, the
> > > > >> response
> > > > >> > > > will
> > > > >> > > > > > not
> > > > >> > > > > > > > >> return
> > > > >> > > > > > > > >> >>> until the request has been completed. And for
> > async
> > > > >> > > requests
> > > > >> > > > > we
> > > > >> > > > > > > can
> > > > >> > > > > > > > >> add a
> > > > >> > > > > > > > >> >>> "token" field in the response, and then only
> > need a
> > > > >> > > general
> > > > >> > > > > > "admin
> > > > >> > > > > > > > >> >>> verification request" with the given token to
> > check
> > > > >> if the
> > > > >> > > > > async
> > > > >> > > > > > > > >> request
> > > > >> > > > > > > > >> >>> has been completed.
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> 5. +1 for extending Metadata request to include
> > > > >> > > controller /
> > > > >> > > > > > > > >> coordinator
> > > > >> > > > > > > > >> >>> information, and then we can remove the
> > > > >> ConsumerMetadata /
> > > > >> > > > > > > > >> ClusterMetadata
> > > > >> > > > > > > > >> >>> requests.
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> Guozhang
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > >> > > > > > > > >> wrote:
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't
> > think I
> > > > >> will be
> > > > >> > > > > able
> > > > >> > > > > > to
> > > > >> > > > > > > > >> make
> > > > >> > > > > > > > >> >>>> it today, so if notes can be sent out
> > afterward
> > > > that
> > > > >> > > would
> > > > >> > > > be
> > > > >> > > > > > > > great.
> > > > >> > > > > > > > >> >>>>
> > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen
> > > > >> Shapira
> > > > >> > > > wrote:
> > > > >> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking
> > forward
> > > > >> to
> > > > >> > > > > chatting
> > > > >> > > > > > > with
> > > > >> > > > > > > > >> >>>> everyone :)
> > > > >> > > > > > > > >> >>>> >
> > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> > > > >> > > > > > > joe.stein@stealth.ly>
> > > > >> > > > > > > > >> wrote:
> > > > >> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout
> > invite
> > > > to
> > > > >> all
> > > > >> > > > pmc,
> > > > >> > > > > > > > >> committers
> > > > >> > > > > > > > >> >>>> and
> > > > >> > > > > > > > >> >>>> > > everyone I found working on a KIP. If I
> > missed
> > > > >> anyone
> > > > >> > > > in
> > > > >> > > > > > the
> > > > >> > > > > > > > >> invite
> > > > >> > > > > > > > >> >>>> please
> > > > >> > > > > > > > >> >>>> > > let me know and can update it, np.
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm
> > Eastern
> > > > >> Time.
> > > > >> > > > Maybe
> > > > >> > > > > > we
> > > > >> > > > > > > > can
> > > > >> > > > > > > > >> get
> > > > >> > > > > > > > >> >>>> INFRA
> > > > >> > > > > > > > >> >>>> > > help to make a google account so we can
> > manage
> > > > >> > > better?
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>> > > To discuss
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>>
> > > > >> > > > > > > > >>
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >>
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > >> > > > > > > > >> >>>> > > in progress and related JIRA that are
> > > > >> interdependent
> > > > >> > > > and
> > > > >> > > > > > > common
> > > > >> > > > > > > > >> work.
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay
> > Kreps <
> > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > >> > > > > > > > >> >>>> wrote:
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that will
> > also
> > > > >> record
> > > > >> > > > and
> > > > >> > > > > > make
> > > > >> > > > > > > > the
> > > > >> > > > > > > > >> >>>> sessions
> > > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > > >> > > > > > > > >> >>>> > >>
> > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > >> > > > > > > > >> >>>> > >>
> > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff
> > > > Holoman
> > > > >> <
> > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > >> > > > > > > > >> >>>> > >>
> > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for
> > this
> > > > >> purpose.
> > > > >> > > We
> > > > >> > > > > > could
> > > > >> > > > > > > > >> record
> > > > >> > > > > > > > >> >>>> the
> > > > >> > > > > > > > >> >>>> > >> > sessions if there is interest and
> > publish
> > > > >> them
> > > > >> > > out.
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay
> > > > Kreps <
> > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > >> > > > > > > > >> >>>> wrote:
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >> > > Let's try to get the technical
> > hang-ups
> > > > >> sorted
> > > > >> > > > out,
> > > > >> > > > > > > > though.
> > > > >> > > > > > > > >> I
> > > > >> > > > > > > > >> >>>> really
> > > > >> > > > > > > > >> >>>> > >> > think
> > > > >> > > > > > > > >> >>>> > >> > > there is some benefit to live
> > discussion
> > > > vs
> > > > >> > > > > writing. I
> > > > >> > > > > > > am
> > > > >> > > > > > > > >> >>>> hopeful that
> > > > >> > > > > > > > >> >>>> > >> if
> > > > >> > > > > > > > >> >>>> > >> > > we post instructions and give
> > ourselves a
> > > > >> few
> > > > >> > > > > attempts
> > > > >> > > > > > > we
> > > > >> > > > > > > > >> can
> > > > >> > > > > > > > >> >>>> get it
> > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > >> > > > > > > > >> >>>> > >> > >
> > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work for
> > > > >> me...any
> > > > >> > > > > > objections?
> > > > >> > > > > > > > >> >>>> > >> > >
> > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > >> > > > > > > > >> >>>> > >> > >
> > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe
> > > > Stein
> > > > >> <
> > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > >> > > > > > > > >> >>>> >
> > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > >> > > > > > > > >> >>>> > >> > >
> > > > >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe like
> > every
> > > > >> > > Tuesday ~
> > > > >> > > > > 1pm
> > > > >> > > > > > > ET
> > > > >> > > > > > > > /
> > > > >> > > > > > > > >> 10am
> > > > >> > > > > > > > >> >>>> PT
> > > > >> > > > > > > > >> >>>> > >> ????
> > > > >> > > > > > > > >> >>>> > >> > > >
> > > > >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but
> > there
> > > > is
> > > > >> > > always
> > > > >> > > > > some
> > > > >> > > > > > > > >> issue or
> > > > >> > > > > > > > >> >>>> > >> whatever
> > > > >> > > > > > > > >> >>>> > >> > so
> > > > >> > > > > > > > >> >>>> > >> > > > we know the apache irc channel
> > works.
> > > > We
> > > > >> can
> > > > >> > > > start
> > > > >> > > > > > > there
> > > > >> > > > > > > > >> and
> > > > >> > > > > > > > >> >>>> see how
> > > > >> > > > > > > > >> >>>> > >> it
> > > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too
> > and
> > > > >> > > associate
> > > > >> > > > to
> > > > >> > > > > > > > >> tickets if
> > > > >> > > > > > > > >> >>>> need be
> > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> > > > >> > > > > > > > >> >>>> > >> > > >
> > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > >> > > > > > > > >> >>>> > >> > > >
> > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM,
> > Jay
> > > > >> Kreps <
> > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > >> > > > > > > > >> >>>> > >> > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a Google
> > > > >> Hangout to
> > > > >> > > > chat
> > > > >> > > > > > > about
> > > > >> > > > > > > > >> this.
> > > > >> > > > > > > > >> >>>> What
> > > > >> > > > > > > > >> >>>> > >> > about
> > > > >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
> > > > further...I
> > > > >> > > > actually
> > > > >> > > > > > > think
> > > > >> > > > > > > > it
> > > > >> > > > > > > > >> >>>> would be
> > > > >> > > > > > > > >> >>>> > >> > good
> > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > >> > > > > > > > >> >>>> > >> > > > > everyone spending a reasonable
> > chunk
> > > > of
> > > > >> > > their
> > > > >> > > > > week
> > > > >> > > > > > > on
> > > > >> > > > > > > > >> Kafka
> > > > >> > > > > > > > >> >>>> stuff
> > > > >> > > > > > > > >> >>>> > >> to
> > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think we
> > could
> > > > >> use
> > > > >> > > time
> > > > >> > > > > to
> > > > >> > > > > > > talk
> > > > >> > > > > > > > >> >>>> through
> > > > >> > > > > > > > >> >>>> > >> design
> > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of
> > > > code
> > > > >> > > > reviews,
> > > > >> > > > > > talk
> > > > >> > > > > > > > >> through
> > > > >> > > > > > > > >> >>>> any
> > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > We can make it publicly
> > available so
> > > > >> that
> > > > >> > > any
> > > > >> > > > > one
> > > > >> > > > > > > can
> > > > >> > > > > > > > >> follow
> > > > >> > > > > > > > >> >>>> along
> > > > >> > > > > > > > >> >>>> > >> > who
> > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this? If so
> > > > I'll
> > > > >> try
> > > > >> > > to
> > > > >> > > > > set
> > > > >> > > > > > it
> > > > >> > > > > > > > up
> > > > >> > > > > > > > >> >>>> starting
> > > > >> > > > > > > > >> >>>> > >> next
> > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM,
> > > > Andrii
> > > > >> > > > > Biletskyi
> > > > >> > > > > > <
> > > > >> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly>
> > wrote:
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed /
> > > > >> aligned
> > > > >> > > > > document
> > > > >> > > > > > > > >> structure.
> > > > >> > > > > > > > >> >>>> Also I
> > > > >> > > > > > > > >> >>>> > >> > > added
> > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > > >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
> > > > >> AdminClient so
> > > > >> > > we
> > > > >> > > > > have
> > > > >> > > > > > > > >> something
> > > > >> > > > > > > > >> >>>> to
> > > > >> > > > > > > > >> >>>> > >> start
> > > > >> > > > > > > > >> >>>> > >> > > > from
> > > > >> > > > > > > > >> >>>> > >> > > > > > while
> > > > >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > >
> > > > >> > > > > > > > >> >>>> > >> > >
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >>
> > > > >> > > > > > > > >> >>>>
> > > > >> > > > > > > > >>
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >>
> > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > > > >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01
> > PM,
> > > > >> Andrii
> > > > >> > > > > > Biletskyi
> > > > >> > > > > > > <
> > > > >> > > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly>
> > > > wrote:
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you are
> > right,
> > > > >> in
> > > > >> > > most
> > > > >> > > > > > cases
> > > > >> > > > > > > > >> client
> > > > >> > > > > > > > >> >>>> will
> > > > >> > > > > > > > >> >>>> > >> have
> > > > >> > > > > > > > >> >>>> > >> > > > enough
> > > > >> > > > > > > > >> >>>> > >> > > > > > > context to show descriptive
> > error
> > > > >> > > message.
> > > > >> > > > > My
> > > > >> > > > > > > > >> concern is
> > > > >> > > > > > > > >> >>>> that
> > > > >> > > > > > > > >> >>>> > >> we
> > > > >> > > > > > > > >> >>>> > >> > > will
> > > > >> > > > > > > > >> >>>> > >> > > > > > have
> > > > >> > > > > > > > >> >>>> > >> > > > > > > to
> > > > >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error codes
> > for
> > > > >> each
> > > > >> > > > > possible
> > > > >> > > > > > > > >> error. Of
> > > > >> > > > > > > > >> >>>> course,
> > > > >> > > > > > > > >> >>>> > >> > we
> > > > >> > > > > > > > >> >>>> > >> > > > > could
> > > > >> > > > > > > > >> >>>> > >> > > > > > > reuse
> > > > >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
> > > > >> > > > > > > UknownTopicOrPartitionCode,
> > > > >> > > > > > > > >> but we
> > > > >> > > > > > > > >> >>>> will
> > > > >> > > > > > > > >> >>>> > >> > also
> > > > >> > > > > > > > >> >>>> > >> > > > need
> > > > >> > > > > > > > >> >>>> > >> > > > > > to
> > > > >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> > > > >> TopicAlreadyExistsCode,
> > > > >> > > > > > > > >> >>>> TopicConfigInvalid (both
> > > > >> > > > > > > > >> >>>> > >> > for
> > > > >> > > > > > > > >> >>>> > >> > > > > topic
> > > > >> > > > > > > > >> >>>> > >> > > > > > > name and config, and probably
> > > > user
> > > > >> would
> > > > >> > > > > like
> > > > >> > > > > > to
> > > > >> > > > > > > > >> know
> > > > >> > > > > > > > >> >>>> what
> > > > >> > > > > > > > >> >>>> > >> > exactly
> > > > >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
> > > > >> > > > > > > InvalidReplicaAssignment,
> > > > >> > > > > > > > >> >>>> > >> InternalError
> > > > >> > > > > > > > >> >>>> > >> > > > (e.g.
> > > > >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > > >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
> > > > TopicCommand,
> > > > >> we
> > > > >> > > will
> > > > >> > > > > > also
> > > > >> > > > > > > > >> need to
> > > > >> > > > > > > > >> >>>> add
> > > > >> > > > > > > > >> >>>> > >> > similar
> > > > >> > > > > > > > >> >>>> > >> > > > > stuff
> > > > >> > > > > > > > >> >>>> > >> > > > > > > for
> > > > >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> > > > >> PreferredReplica. So
> > > > >> > > > > we'll
> > > > >> > > > > > > end
> > > > >> > > > > > > > >> up
> > > > >> > > > > > > > >> >>>> with a
> > > > >> > > > > > > > >> >>>> > >> > large
> > > > >> > > > > > > > >> >>>> > >> > > > list
> > > > >> > > > > > > > >> >>>> > >> > > > > > of
> > > > >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in
> > Admin
> > > > >> > > protocol.
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree my
> > > > >> proposal is
> > > > >> > > > not
> > > > >> > > > > > > > >> consistent
> > > > >> > > > > > > > >> >>>> with
> > > > >> > > > > > > > >> >>>> > >> > other
> > > > >> > > > > > > > >> >>>> > >> > > > > cases.
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better
> > solution
> > > > >> or
> > > > >> > > > > something
> > > > >> > > > > > > > >> >>>> in-between.
> > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it
> > is a
> > > > >> great
> > > > >> > > > idea.
> > > > >> > > > > > > This
> > > > >> > > > > > > > >> way we
> > > > >> > > > > > > > >> >>>> can
> > > > >> > > > > > > > >> >>>> > >> move
> > > > >> > > > > > > > >> >>>> > >> > > on
> > > > >> > > > > > > > >> >>>> > >> > > > > > > faster.
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on
> > date/time
> > > > so
> > > > >> > > people
> > > > >> > > > > can
> > > > >> > > > > > > > join.
> > > > >> > > > > > > > >> >>>> Will work
> > > > >> > > > > > > > >> >>>> > >> > for
> > > > >> > > > > > > > >> >>>> > >> > > me
> > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > >> > > > > > > > >> >>>> > >> > > > > > > and
> > > > >> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime if
> > > > agreed
> > > > >> in
> > > > >> > > > > advance.
> > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09
> > PM,
> > > > >> Jay
> > > > >> > > > Kreps <
> > > > >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > >> > > > > > > > >> >>>> > >> > > > > wrote:
> > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good
> > error
> > > > >> handling
> > > > >> > > > > > without
> > > > >> > > > > > > > >> needing
> > > > >> > > > > > > > >> >>>> custom
> > > > >> > > > > > > > >> >>>> > >> > > > > > server-side
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the
> > > > >> client has
> > > > >> > > > the
> > > > >> > > > > > > > >> context to
> > > > >> > > > > > > > >> >>>> know
> > > > >> > > > > > > > >> >>>> > >> that
> > > > >> > > > > > > > >> >>>> > >> > > if
> > > > >> > > > > > > > >> >>>> > >> > > > it
> > > > >> > > > > > > > >> >>>> > >> > > > > > got
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic
> > doesn't
> > > > >> exist
> > > > >> > > to
> > > > >> > > > > say
> > > > >> > > > > > > > >> "Topic X
> > > > >> > > > > > > > >> >>>> doesn't
> > > > >> > > > > > > > >> >>>> > >> > > exist"
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> rather
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or
> > > > >> whatever).
> > > > >> > > Maybe
> > > > >> > > > > > there
> > > > >> > > > > > > > are
> > > > >> > > > > > > > >> >>>> specific
> > > > >> > > > > > > > >> >>>> > >> > cases
> > > > >> > > > > > > > >> >>>> > >> > > > > where
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to
> > add
> > > > >> > > > server-side
> > > > >> > > > > > > error
> > > > >> > > > > > > > >> >>>> messages we
> > > > >> > > > > > > > >> >>>> > >> > > really
> > > > >> > > > > > > > >> >>>> > >> > > > > do
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a
> > consistent
> > > > >> way
> > > > >> > > > across
> > > > >> > > > > > the
> > > > >> > > > > > > > >> protocol.
> > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open
> > > > >> questions
> > > > >> > > > here
> > > > >> > > > > > > from
> > > > >> > > > > > > > my
> > > > >> > > > > > > > >> >>>> previous
> > > > >> > > > > > > > >> >>>> > >> > > list. I
> > > > >> > > > > > > > >> >>>> > >> > > > > > will
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few
> > days for
> > > > >> Strata
> > > > >> > > > > > though.
> > > > >> > > > > > > > >> Maybe
> > > > >> > > > > > > > >> >>>> we could
> > > > >> > > > > > > > >> >>>> > >> > do
> > > > >> > > > > > > > >> >>>> > >> > > a
> > > > >> > > > > > > > >> >>>> > >> > > > > > Google
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open
> > issues
> > > > >> some
> > > > >> > > time
> > > > >> > > > > > > towards
> > > > >> > > > > > > > >> the
> > > > >> > > > > > > > >> >>>> end of
> > > > >> > > > > > > > >> >>>> > >> > next
> > > > >> > > > > > > > >> >>>> > >> > > > week
> > > > >> > > > > > > > >> >>>> > >> > > > > > for
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in this
> > > > ticket?
> > > > >> I
> > > > >> > > have
> > > > >> > > > a
> > > > >> > > > > > > > feeling
> > > > >> > > > > > > > >> that
> > > > >> > > > > > > > >> >>>> might
> > > > >> > > > > > > > >> >>>> > >> > > > progress
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster than
> > > > >> email--I
> > > > >> > > > think
> > > > >> > > > > we
> > > > >> > > > > > > > >> could talk
> > > > >> > > > > > > > >> >>>> > >> through
> > > > >> > > > > > > > >> >>>> > >> > > > those
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly
> > > > >> quickly...
> > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at
> > 7:27 AM,
> > > > >> Andrii
> > > > >> > > > > > > > Biletskyi <
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly
> > >
> > > > >> wrote:
> > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address
> > some of
> > > > >> the
> > > > >> > > > issues
> > > > >> > > > > > > which
> > > > >> > > > > > > > >> were
> > > > >> > > > > > > > >> >>>> > >> mentioned
> > > > >> > > > > > > > >> >>>> > >> > > > > earlier
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> about
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of
> > > > >> those was
> > > > >> > > > > about
> > > > >> > > > > > > > >> batching
> > > > >> > > > > > > > >> >>>> > >> > operations.
> > > > >> > > > > > > > >> >>>> > >> > > > What
> > > > >> > > > > > > > >> >>>> > >> > > > > > if
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> we
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand
> > approach
> > > > >> and let
> > > > >> > > > > people
> > > > >> > > > > > > > >> specify
> > > > >> > > > > > > > >> >>>> > >> topic-name
> > > > >> > > > > > > > >> >>>> > >> > > by
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most of
> > the
> > > > use
> > > > >> > > cases?
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what
> > information
> > > > >> should
> > > > >> > > we
> > > > >> > > > > > > > generally
> > > > >> > > > > > > > >> >>>> provide in
> > > > >> > > > > > > > >> >>>> > >> > > Admin
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin
> > commands
> > > > >> don't
> > > > >> > > > imply
> > > > >> > > > > > > they
> > > > >> > > > > > > > >> will
> > > > >> > > > > > > > >> >>>> be used
> > > > >> > > > > > > > >> >>>> > >> > only
> > > > >> > > > > > > > >> >>>> > >> > > > in
> > > > >> > > > > > > > >> >>>> > >> > > > > > CLI
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > but,
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a
> > very
> > > > >> > > important
> > > > >> > > > > > > client
> > > > >> > > > > > > > >> of this
> > > > >> > > > > > > > >> >>>> > >> > feature.
> > > > >> > > > > > > > >> >>>> > >> > > In
> > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > case,
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would
> > like
> > > > to
> > > > >> > > provide
> > > > >> > > > > > users
> > > > >> > > > > > > > >> with
> > > > >> > > > > > > > >> >>>> rich
> > > > >> > > > > > > > >> >>>> > >> > > experience
> > > > >> > > > > > > > >> >>>> > >> > > > > in
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> terms
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > of
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > getting results / errors
> > of
> > > > the
> > > > >> > > > executed
> > > > >> > > > > > > > >> commands.
> > > > >> > > > > > > > >> >>>> Usually
> > > > >> > > > > > > > >> >>>> > >> we
> > > > >> > > > > > > > >> >>>> > >> > > > supply
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> with
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses only errorCode,
> > > > which
> > > > >> looks
> > > > >> > > > > very
> > > > >> > > > > > > > >> limiting,
> > > > >> > > > > > > > >> >>>> in case
> > > > >> > > > > > > > >> >>>> > >> > of
> > > > >> > > > > > > > >> >>>> > >> > > > CLI
> > > > >> > > > > > > > >> >>>> > >> > > > > we
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> may
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human
> > readable
> > > > >> error
> > > > >> > > > > > > description.
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into account
> > > > >> previous item
> > > > >> > > > > about
> > > > >> > > > > > > > >> batching,
> > > > >> > > > > > > > >> >>>> what
> > > > >> > > > > > > > >> >>>> > >> do
> > > > >> > > > > > > > >> >>>> > >> > > you
> > > > >> > > > > > > > >> >>>> > >> > > > > > think
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > about
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support
> > > > >> regexp)
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest =>
> > > > TopicName
> > > > >> > > > > Partitions
> > > > >> > > > > > > > >> Replicas
> > > > >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse =>
> > > > ErrorCode
> > > > >> > > > > > > > ErrorDescription
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription =>
> > string
> > > > >> (empty
> > > > >> > > if
> > > > >> > > > > > > > >> successful)
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
> > > > >> TopicNameRegexp
> > > > >> > > >
> > > > >
> > > > > ...
> > > > >
> > > > > [Message clipped]
> > > >
> >
> >


Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
Jay,

That's a valid concern. However, since a topic can be deleted immediately
after creation, the client can't assume that the topic metadata is always
available after topic creation, even if the creation is synchronous.

Currently, the client flow is the following.

1. If topic metadata doesn't exist, backoff if needed and issue
TopicMetadataRequest.
2. On receiving TopicMetadataResponse, set topic metadata accordingly.

I was imagining that the new client follow (with topic creation) will be:

0. If topic creation is needed, issue TopicCreationRequest.
1. If topic metadata doesn't exist, backoff if needed and issue
TopicMetadataRequest.
2. On receiving TopicMetadataResponse, set topic metadata accordingly. If
the error code in the response is UnknownTopic, mark topic creation as
needed.

We probably don't need to backoff topic creation explicitly since it can
just be piggybacked on the backoff of TopicMetadataRequest. Does that make
sense?

Thanks,

Jun


On Sun, Mar 22, 2015 at 2:12 PM, Jay Kreps <ja...@gmail.com> wrote:

> Jun,
>
> The case I am most concerned about is create. We are proposing to get rid
> of the confusing "auto create on metadata fetch" behavior. To make this
> work every producer will need to support the create topic request. If each
> client has to implement the create and then do a non-blocking poll with
> exponential backoff to find out when the request is complete I'm concerned
> that will be a bit complex to do right and significantly add to the client
> burden.
>
> -Jay
>
> On Fri, Mar 20, 2015 at 3:18 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Andrii,
> >
> > A few points.
> >
> > 1. Create/Alter can typically complete quickly. So, it's possible to make
> > the request block until it's completed. However, currently, doing this at
> > the broker is a bit involved. To make Create block, we will need to add
> > some callbacks in KafkaController. This is possible. However, the
> > controller logic currently is pretty completed. It would probably be
> better
> > if we clean it up first before adding more complexity to it. Alter is
> even
> > trickier. Adding partition is currently handled through KafkaController.
> So
> > it can be dealt with in a similar way. However, Alter config is done
> > completely differently. It doesn't go through the controller. Instead,
> each
> > broker listens to ZooKeeper directly. So, it's not clear if there is an
> > easy way on the broker to figure out whether a config is applied on every
> > broker.
> >
> > 2. Delete can potentially take long if a replica to be deleted is
> offline.
> > PreferredLeader/PartitionReassign can also take long. So, we can't really
> > make those requests block on the broker.
> >
> > As you can see, at this moment it's not easy to make all admin requests
> > block on the broker. So, if we want the blocking feature in the admin
> > utility in the short term, doing the completion check at the admin client
> > is probably an easier route, even though it may not be ideal.
> >
> > Thanks,
> >
> > Jun
> >
> > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Jun,
> > >
> > > I see your point. But wouldn't that lead to a "fat" client
> > implementations?
> > > Suppose someone would like to implement client for Admin Wire protocol.
> > > Not only people will have to code quite complicated logic like "send
> > > describe
> > > request to each broker" (again state machin?) but it will also mean
> > people
> > > must understand internal kafka logic related to topic storage and how
> > > information is propageted from the controller to brokers.
> > > I see this like a dilemma between having a concise Wire Protocol and
> > > self-sufficient API to make client implementations simple.
> > > I don't have a win-win solution though.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > >
> > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > For 1), 2) and 3), blocking would probably mean that the new metadata
> > is
> > > > propagated to every broker. To achieve that, the client can keep
> > issuing
> > > > the describe topic request to every broker until it sees the new
> > metadata
> > > > in the response.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Hm, actually the ticket you linked, Guozhang, brings as back
> > > > > to the problem what should be considered a post-condition for
> > > > > each of the admin commands.
> > > > > In my understanding:
> > > > >
> > > > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > > > (Not the controller picked up changes from zk and broadcasted
> > > > > LeaderAndIsr and UpdateMetadata)
> > > > >
> > > > > 2) AlterTopic - same as 1) - broker changed assignment data
> > > > > in zookeeper or created admin path for topic config change
> > > > >
> > > > > 3) DeleteTopic - admin path /admin/delete_topics is created
> > > > >
> > > > > 4) ReassignPartitions and PreferredReplica - corresponding admin
> > > > > path is created
> > > > >
> > > > > Now what can be considered a completed operation from the client's
> > > > > perspective?
> > > > > 1) Topic is created once corresponding data is in zk
> > > > > (I remember there were some thoughts that it'd be good to consider
> > > > > topic created once all replicas receive information about it and
> thus
> > > > > clients can produce/consume from it, but as was discussed this
> seems
> > > > > to be a hard thing to do)
> > > > >
> > > > > 2) Probably same as 1), so right after AlterTopic is issued
> > > > >
> > > > > 3) The topic has been removed from /brokers/topics
> > > > >
> > > > > 4) ReassignPartitions and PrefferedReplica were discussed earlier -
> > > > > in short the former is completed once partition state info in zk
> > > matches
> > > > > reassignment request and admin path is empty, the latter - once
> data
> > > > > in zk shows that head of assignned replicas of the partition and
> > leader
> > > > > is the same replica
> > > > >
> > > > > Thoughts?
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <wangguoz@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > I think while loop is fine for supporting blocking, just that we
> > need
> > > > to
> > > > > > add back off to avoid bombarding brokers with DescribeTopic
> > requests.
> > > > > >
> > > > > > Also I have linked KAFKA-1125
> > > > > > <https://issues.apache.org/jira/browse/KAFKA-1125> to your
> > proposal,
> > > > and
> > > > > > when KAFKA-1694 is done this ticket can also be closed.
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Great.
> > > > > > > I want to elaborate this a bit more, to see we are on the same
> > page
> > > > > > > concerning the client code.
> > > > > > >
> > > > > > > So with all topic commands being async a client (AdminClient in
> > our
> > > > > > > case or any other other client people would like to implement)
> to
> > > > > support
> > > > > > > a blocking operation (which seems to be a natural use-case e.g.
> > for
> > > > > topic
> > > > > > > creation): would have to do:
> > > > > > > 1. issue CreateTopicRequest
> > > > > > > 2. if successful, in a "while" loop send DescribeTopicRequest
> and
> > > > > > > break the loop once all topics are returned in response (or
> upon
> > > > > > timeout).
> > > > > > > 3. if unsuccessful throw exception
> > > > > > > Would it be okay?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Andrii,
> > > > > > > >
> > > > > > > > I think you are right. It seems that only ReassignPartitions
> > > needs
> > > > a
> > > > > > > > separate verification request.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >
> > > > > > > > > Guys,
> > > > > > > > > I like this idea too. Let's stick with that. I'll update
> KIP
> > > > > > > accordingly.
> > > > > > > > >
> > > > > > > > > I was also thinking we can avoid adding dedicated status
> > check
> > > > > > > > > requests for topic commands. - We have everything in
> > > > DescribeTopic
> > > > > > > > > for that! E.g.:
> > > > > > > > > User issued CreateTopic - to check the status client sends
> > > > > > > DescribeTopic
> > > > > > > > > and checks whether is something returned for that topic.
> The
> > > same
> > > > > for
> > > > > > > > > alteration, deletion.
> > > > > > > > > Btw, PreferredReplica status can be also checked with
> > > > > > > > DescribeTopicRequest
> > > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > > For ReassignPartitions as discussed we'll need to have a
> > > separate
> > > > > > > > Verify...
> > > > > > > > > request.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Andrii Biletskyi
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> > > > wangguoz@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > +1 on broker writing to ZK for async handling. I was
> > thinking
> > > > > that
> > > > > > in
> > > > > > > > the
> > > > > > > > > > end state the admin requests would be eventually sent to
> > > > > controller
> > > > > > > > > either
> > > > > > > > > > through re-routing or clients discovering them, instead
> of
> > > > > letting
> > > > > > > > > > controller listen on ZK admin path. But thinking about
> it a
> > > > > second
> > > > > > > > time,
> > > > > > > > > I
> > > > > > > > > > think it is actually simpler to let controller manage
> > > > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > > > >
> > > > > > > > > > Guozhang
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > > > jjkoshy.w@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > +1 as well. I think it helps to keep the rerouting
> > approach
> > > > > > > > orthogonal
> > > > > > > > > > > to this KIP.
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps
> > wrote:
> > > > > > > > > > > > I'm +1 on Jun's suggestion as long as it can work for
> > all
> > > > the
> > > > > > > > > requests.
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <
> > > jun@confluent.io
> > > > >
> > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > >
> > > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > > >
> > > > > > > > > > > > > (a) Admin requests can be sent to and handled by
> any
> > > > > broker.
> > > > > > > > > > > > > (b) Admin requests are processed asynchronously, at
> > > least
> > > > > for
> > > > > > > > now.
> > > > > > > > > > > That is,
> > > > > > > > > > > > > when the client gets a response, it just means that
> > the
> > > > > > request
> > > > > > > > is
> > > > > > > > > > > > > initiated, but not necessarily completed. Then,
> it's
> > up
> > > > to
> > > > > > the
> > > > > > > > > client
> > > > > > > > > > > to
> > > > > > > > > > > > > issue another request to check the status for
> > > completion.
> > > > > > > > > > > > >
> > > > > > > > > > > > > To support (a), we were thinking of doing request
> > > > > forwarding
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > > > controller (utilizing KAFKA-1912). I am making an
> > > > > alternative
> > > > > > > > > > proposal.
> > > > > > > > > > > > > Basically, the broker can just write to ZooKeeper
> to
> > > > inform
> > > > > > the
> > > > > > > > > > > controller
> > > > > > > > > > > > > about the request. For example, to handle
> > > > > > > partitionReassignment,
> > > > > > > > > the
> > > > > > > > > > > broker
> > > > > > > > > > > > > will just write the requested partitions to
> > > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > > (like what AdminUtils currently does) and then
> send a
> > > > > > response
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > > client. This shouldn't take long and the
> > implementation
> > > > > will
> > > > > > be
> > > > > > > > > > simpler
> > > > > > > > > > > > > than forwarding the requests to the controller
> > through
> > > > RPC.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I might be wrong but didn't we agree we will let
> > any
> > > > > broker
> > > > > > > > from
> > > > > > > > > > the
> > > > > > > > > > > > > > cluster handle *long-running* admin requests (at
> > this
> > > > > time
> > > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > reassignPartitions), via zk admin path. Thus
> > > > CreateTopics
> > > > > > etc
> > > > > > > > > > should
> > > > > > > > > > > be
> > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> > > > > > jun@confluent.io>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I think we agreed that those admin requests can
> > be
> > > > > issued
> > > > > > > to
> > > > > > > > > any
> > > > > > > > > > > > > broker.
> > > > > > > > > > > > > > > Because of that, there doesn't seem to be a
> > strong
> > > > need
> > > > > > to
> > > > > > > > know
> > > > > > > > > > the
> > > > > > > > > > > > > > > controller. So, perhaps we can proceed by not
> > > making
> > > > > any
> > > > > > > > change
> > > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > > format of TMR right now. When we start using
> > create
> > > > > topic
> > > > > > > > > request
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > producer, we will need a new version of TMR
> that
> > > > > doesn't
> > > > > > > > > trigger
> > > > > > > > > > > auto
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > As a first cut implementation, I think the
> broker
> > > can
> > > > > > just
> > > > > > > > > write
> > > > > > > > > > > to ZK
> > > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > > requests, instead of forwarding them to the
> > > > controller.
> > > > > > > This
> > > > > > > > > will
> > > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> > > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > For (1) yes we will circle back on that
> shortly
> > > > after
> > > > > > > > syncing
> > > > > > > > > > up
> > > > > > > > > > > in
> > > > > > > > > > > > > > > > person. I think it is close to getting
> > committed
> > > > > > although
> > > > > > > > > > > development
> > > > > > > > > > > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > There is one more item we covered at the
> > hangout.
> > > > > i.e.,
> > > > > > > > > whether
> > > > > > > > > > > we
> > > > > > > > > > > > > > > > want to add the coordinator to the topic
> > metadata
> > > > > > > response
> > > > > > > > or
> > > > > > > > > > > provide
> > > > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > There are two reasons I think we should try
> and
> > > > avoid
> > > > > > > > adding
> > > > > > > > > > the
> > > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > > > > > - If we finally do request rerouting in Kafka
> > > then
> > > > > the
> > > > > > > > field
> > > > > > > > > > > would
> > > > > > > > > > > > > add
> > > > > > > > > > > > > > > >   little to no value. (It still helps to
> have a
> > > > > > separate
> > > > > > > > > > > > > > > >   ClusterMetadataRequest to query for
> > > cluster-wide
> > > > > > > > > information
> > > > > > > > > > > such
> > > > > > > > > > > > > as
> > > > > > > > > > > > > > > >   'which broker is the controller?' as Joe
> > > > > mentioned.)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I think it would be cleaner to have an
> explicit
> > > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > > that you can send to any broker in order to
> > > obtain
> > > > > the
> > > > > > > > > > controller
> > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > in the future possibly other cluster-wide
> > > > > > information). I
> > > > > > > > > think
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > main argument against doing this and instead
> > > adding
> > > > > it
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > > metadata response was convenience - i.e., you
> > > don't
> > > > > > have
> > > > > > > to
> > > > > > > > > > > discover
> > > > > > > > > > > > > > > > the controller in advance. However, I don't
> see
> > > > much
> > > > > > > actual
> > > > > > > > > > > > > > > > benefit/convenience in this and in fact think
> > it
> > > > is a
> > > > > > > > > > non-issue.
> > > > > > > > > > > Let
> > > > > > > > > > > > > > > > me know if I'm overlooking something here.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > As an example, say we need to initiate
> > partition
> > > > > > > > reassignment
> > > > > > > > > > by
> > > > > > > > > > > > > > > > issuing the new ReassignPartitionsRequest to
> > the
> > > > > > > controller
> > > > > > > > > > > (assume
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > already have the desired manual partition
> > > > > assignment).
> > > > > > > If
> > > > > > > > we
> > > > > > > > > > > are to
> > > > > > > > > > > > > > > > augment topic metadata response then the flow
> > be
> > > > > > > something
> > > > > > > > > like
> > > > > > > > > > > this
> > > > > > > > > > > > > :
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - Issue topic metadata request to any broker
> > (and
> > > > > > > discover
> > > > > > > > > the
> > > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > > - Connect to controller if required (i.e., if
> > the
> > > > > > broker
> > > > > > > > > above
> > > > > > > > > > !=
> > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > - Issue the partition reassignment request to
> > the
> > > > > > > > controller.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > With an explicit cluster metadata request it
> > > would
> > > > > be:
> > > > > > > > > > > > > > > > - Issue cluster metadata request to any
> broker
> > > > > > > > > > > > > > > > - Connect to controller if required (i.e., if
> > the
> > > > > > broker
> > > > > > > > > above
> > > > > > > > > > !=
> > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > - Issue the partition reassignment request
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > So it seems to add little practical value and
> > > > bloats
> > > > > > > topic
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > The other angle to this is the following - is
> > it
> > > a
> > > > > > matter
> > > > > > > > of
> > > > > > > > > > > naming?
> > > > > > > > > > > > > > > > Should we just rename topic metadata
> > > > request/response
> > > > > > to
> > > > > > > > just
> > > > > > > > > > > > > > > > MetadataRequest/Response and add cluster
> > metadata
> > > > to
> > > > > > it?
> > > > > > > By
> > > > > > > > > > that
> > > > > > > > > > > same
> > > > > > > > > > > > > > > > token should we also allow querying for the
> > > > consumer
> > > > > > > > > > coordinator
> > > > > > > > > > > (and
> > > > > > > > > > > > > > > > in future transaction coordinator) as well?
> > This
> > > > > leads
> > > > > > > to a
> > > > > > > > > > > bloated
> > > > > > > > > > > > > > > > request which isn't very appealing and
> > altogether
> > > > > > > > confusing.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun
> > Rao
> > > > > > wrote:
> > > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 1. I just realized that in order to start
> > > working
> > > > > on
> > > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > need to merge the changes to
> > > OffsetCommitRequest
> > > > > > (from
> > > > > > > > > 0.8.2)
> > > > > > > > > > > to
> > > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > > This is planned to be done as part of
> > > KAFKA-1634.
> > > > > So,
> > > > > > > we
> > > > > > > > > will
> > > > > > > > > > > need
> > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 2. Thinking about this a bit more, if the
> > > > semantic
> > > > > of
> > > > > > > > those
> > > > > > > > > > > "write"
> > > > > > > > > > > > > > > > > requests is async (i.e., after the client
> > gets
> > > a
> > > > > > > > response,
> > > > > > > > > it
> > > > > > > > > > > just
> > > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > > that the operation is initiated, but not
> > > > > necessarily
> > > > > > > > > > > completed), we
> > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > really need to forward the requests to the
> > > > > > controller.
> > > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > > receiving broker can just write the
> operation
> > > to
> > > > ZK
> > > > > > as
> > > > > > > > the
> > > > > > > > > > > admin
> > > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > > line tool previously does. This will
> simplify
> > > the
> > > > > > > > > > > implementation.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 8. There is another implementation detail
> for
> > > > > > describe
> > > > > > > > > topic.
> > > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > want to read the topic config from the
> broker
> > > > > cache,
> > > > > > > > > instead
> > > > > > > > > > of
> > > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > > Currently, every broker reads the
> topic-level
> > > > > config
> > > > > > > for
> > > > > > > > > all
> > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > However, it ignores those for topics not
> > hosted
> > > > on
> > > > > > > > itself.
> > > > > > > > > > So,
> > > > > > > > > > > we
> > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > to change TopicConfigManager a bit so that
> it
> > > > > caches
> > > > > > > the
> > > > > > > > > > > configs
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii
> > > > Biletskyi <
> > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala requests
> > objects,
> > > > use
> > > > > > java
> > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > > >     A: Gwen kindly took that
> (KAFKA-1927).
> > > It's
> > > > > > > > important
> > > > > > > > > > to
> > > > > > > > > > > > > speed
> > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > > >          there since this ticket blocks
> > other
> > > > > > > important
> > > > > > > > > > > changes.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute facility vs
> client
> > > > > > > maintaining
> > > > > > > > > > > cluster
> > > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > > >     A: Jay has added pseudo code to
> > > KAFKA-1912
> > > > -
> > > > > > need
> > > > > > > > to
> > > > > > > > > > > consider
> > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > > >         easy to implement as a
> server-side
> > > > > feature
> > > > > > > > > > (comments
> > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > > > > > > > > > > >     A: This might be useful for clients,
> > add
> > > > this
> > > > > > to
> > > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > > > > > > > > > > >     A: I will add proposed by Jun
> solution
> > > > (using
> > > > > > > > > clientId
> > > > > > > > > > > for
> > > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs
> > grabbing
> > > > all
> > > > > > > > > protocol
> > > > > > > > > > > > > changes
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > >     A: It was decided to try to gather
> all
> > > > > changes
> > > > > > to
> > > > > > > > > > > protocol
> > > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > > >         In case of TMR it worth checking:
> > > > > > KAFKA-2020
> > > > > > > > and
> > > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to deserialize
> > > user's
> > > > > > input
> > > > > > > in
> > > > > > > > > CLI
> > > > > > > > > > > tool.
> > > > > > > > > > > > > > > > > >     A: Use jackson for that, /tools
> project
> > > is
> > > > a
> > > > > > > > separate
> > > > > > > > > > > jar so
> > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs
> generic
> > > > status
> > > > > > > check
> > > > > > > > > > > command.
> > > > > > > > > > > > > > > > > >      A: For long-running requests like
> > > reassign
> > > > > > > > > partitions
> > > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > > >          it makes sense to introduce it.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >  Please add, correct me if I missed
> > > something.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii
> > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > You are right, I removed
> ClusterMetadata
> > > > > because
> > > > > > we
> > > > > > > > > have
> > > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > > what we need in TopicMetadata. Also, as
> > Jay
> > > > > > pointed
> > > > > > > > out
> > > > > > > > > > > > > earlier,
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > would like to have "orthogonal" API,
> but
> > at
> > > > the
> > > > > > > same
> > > > > > > > > time
> > > > > > > > > > > we
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > But I like your idea and even have some
> > > other
> > > > > > > > arguments
> > > > > > > > > > for
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > > There is also DescribeTopicRequest
> which
> > > was
> > > > > > > proposed
> > > > > > > > > in
> > > > > > > > > > > this
> > > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > > it returns topic configs, partitions,
> > > > > replication
> > > > > > > > > factor
> > > > > > > > > > > plus
> > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > > leader replica. The later part is
> really
> > > > > already
> > > > > > > > there
> > > > > > > > > in
> > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > > So again we'll have to add stuff to
> TMR,
> > > not
> > > > to
> > > > > > > > > duplicate
> > > > > > > > > > > some
> > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > newly added requests. However, this way
> > > we'll
> > > > > end
> > > > > > > up
> > > > > > > > > with
> > > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > > request which returns cluster metadata,
> > > topic
> > > > > > > > > replication
> > > > > > > > > > > and
> > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > plus partition replication data. Seems
> > > > logical
> > > > > to
> > > > > > > > split
> > > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > > - ClusterMetadata (brokers +
> controller,
> > > > maybe
> > > > > > smth
> > > > > > > > > else)
> > > > > > > > > > > > > > > > > > > - TopicMetadata (topic info + partition
> > > > > details)
> > > > > > > > > > > > > > > > > > > But since current TMR is involved in
> lots
> > > of
> > > > > > places
> > > > > > > > > > > (including
> > > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > as I understand) this might be very
> > serious
> > > > > > change
> > > > > > > > and
> > > > > > > > > it
> > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel
> > > Koshy <
> > > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >> I may be missing some context but
> > > hopefully
> > > > > this
> > > > > > > > will
> > > > > > > > > > > also be
> > > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > > >> today: I thought the earlier proposal
> > > where
> > > > > > there
> > > > > > > > was
> > > > > > > > > an
> > > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > > >> ClusterMetadata request was clearer
> and
> > > > > > explicit.
> > > > > > > > > During
> > > > > > > > > > > the
> > > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > >> this thread I think the conclusion was
> > > that
> > > > > the
> > > > > > > main
> > > > > > > > > > need
> > > > > > > > > > > was
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> controller information and that can be
> > > > rolled
> > > > > > into
> > > > > > > > the
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > >> response but that seems a bit
> irrelevant
> > > to
> > > > > > topic
> > > > > > > > > > > metadata.
> > > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > > >> think the full broker-list is also
> > > > irrelevant
> > > > > to
> > > > > > > > topic
> > > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > >> it is already there and in use. I
> think
> > > > there
> > > > > is
> > > > > > > > still
> > > > > > > > > > > room
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > >> explicit ClusterMetadata request since
> > > there
> > > > > may
> > > > > > > be
> > > > > > > > > > other
> > > > > > > > > > > > > > > > > > >> cluster-level information that we may
> > want
> > > > to
> > > > > > add
> > > > > > > > over
> > > > > > > > > > > time
> > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> have nothing to do with topic
> metadata).
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM
> > +0200,
> > > > > Andrii
> > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > 101. Okay, if you say that such use
> > case
> > > > is
> > > > > > > > > > important. I
> > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > >> > using clientId for these purposes is
> > > fine
> > > > -
> > > > > if
> > > > > > > we
> > > > > > > > > > > already
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > > >> > as part of all Wire protocol
> messages,
> > > why
> > > > > not
> > > > > > > use
> > > > > > > > > > that.
> > > > > > > > > > > > > > > > > > >> > I will update KIP-4 page if nobody
> has
> > > > other
> > > > > > > ideas
> > > > > > > > > > > (which
> > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP
> > > > > accordingly.
> > > > > > I
> > > > > > > > > think
> > > > > > > > > > > we can
> > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > > >> > fine-grained error codes if some
> error
> > > > code
> > > > > > > > received
> > > > > > > > > > in
> > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > >> > won't give enough context to return
> a
> > > > > > > descriptive
> > > > > > > > > > error
> > > > > > > > > > > > > > message
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > Look forward to discussing all
> > > outstanding
> > > > > > > issues
> > > > > > > > in
> > > > > > > > > > > detail
> > > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM,
> Jun
> > > Rao
> > > > <
> > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > 101. There may be a use case where
> > you
> > > > > only
> > > > > > > want
> > > > > > > > > the
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > > >> > > manually by admins. Currently, you
> > can
> > > > do
> > > > > > that
> > > > > > > > by
> > > > > > > > > > > > > disabling
> > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> > > creation and issue topic creation
> > from
> > > > the
> > > > > > > > > > > TopicCommand.
> > > > > > > > > > > > > If
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > > >> > > topic creation completely on the
> > > broker
> > > > > and
> > > > > > > > don't
> > > > > > > > > > > have a
> > > > > > > > > > > > > way
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > >> > > between topic creation requests
> from
> > > the
> > > > > > > regular
> > > > > > > > > > > clients
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > > >> > > can't support manual topic
> creation
> > > any
> > > > > > more.
> > > > > > > I
> > > > > > > > > was
> > > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > > >> > > way of distinguishing the clients
> > > making
> > > > > the
> > > > > > > > topic
> > > > > > > > > > > > > creation
> > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > >> > > using clientId. For example, the
> > admin
> > > > > tool
> > > > > > > can
> > > > > > > > > set
> > > > > > > > > > > it to
> > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > > >> > > admin and the broker can treat
> that
> > > > > clientId
> > > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > Also, there is a related
> discussion
> > in
> > > > > > > > KAFKA-2020.
> > > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > following in
> TopicMetadataResponse:
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > 1. If leader is not available, we
> > set
> > > > the
> > > > > > > > > partition
> > > > > > > > > > > level
> > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > > >> > > 2. If a non-leader replica is not
> > > > > available,
> > > > > > > we
> > > > > > > > > take
> > > > > > > > > > > that
> > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > >> > > the assigned replica list and isr
> in
> > > the
> > > > > > > > response.
> > > > > > > > > > As
> > > > > > > > > > > an
> > > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > > >> > > doing that, we set the partition
> > level
> > > > > error
> > > > > > > > code
> > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > This has a few problems. First,
> > > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > > >> > > an error, at least for the normal
> > > > > > > > > producer/consumer
> > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > > >> > > to find out the leader. Second, it
> > can
> > > > > > happen
> > > > > > > > that
> > > > > > > > > > > both
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> > > another replica are not available
> at
> > > the
> > > > > > same
> > > > > > > > > time.
> > > > > > > > > > > There
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > >> > > to indicate both. Third, even if a
> > > > replica
> > > > > > is
> > > > > > > > not
> > > > > > > > > > > > > available,
> > > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > > >> > > useful to return its replica id
> > since
> > > > some
> > > > > > > > clients
> > > > > > > > > > > (e.g.
> > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > One way to address this issue is
> to
> > > > always
> > > > > > > > return
> > > > > > > > > > the
> > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > > >> > > leader, assigned replicas, and isr
> > > > > > regardless
> > > > > > > of
> > > > > > > > > > > whether
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > > >> > > broker is live or not. Since we
> also
> > > > > return
> > > > > > > the
> > > > > > > > > list
> > > > > > > > > > > of
> > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > client can figure out whether a
> > leader
> > > > or
> > > > > a
> > > > > > > > > replica
> > > > > > > > > > is
> > > > > > > > > > > > > live
> > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > > >> > > accordingly. This way, we don't
> need
> > > to
> > > > > set
> > > > > > > the
> > > > > > > > > > > partition
> > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > >> > > when the leader or a replica is
> not
> > > > > > available.
> > > > > > > > > This
> > > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > > >> > > protocol, but does change the
> > > semantics.
> > > > > > Since
> > > > > > > > we
> > > > > > > > > > are
> > > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > > >> > > of TopicMetadataRequest here, we
> can
> > > > > > > potentially
> > > > > > > > > > > piggyback
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > 102.1 For those types of errors
> due
> > to
> > > > > > invalid
> > > > > > > > > > input,
> > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > >> > > guard it at parameter validation
> > time
> > > > and
> > > > > > > throw
> > > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > > >> > > without even sending the request
> to
> > > the
> > > > > > > broker?
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM,
> > > Andrii
> > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly>
> wrote:
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > 101. If I understand you
> > correctly,
> > > > you
> > > > > > are
> > > > > > > > > saying
> > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > > >> > > > will be ported to TMR_V1) won't
> be
> > > > able
> > > > > to
> > > > > > > > > > > automatically
> > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > >> > > > unconditionally remove topic
> > > creation
> > > > > from
> > > > > > > > > there).
> > > > > > > > > > > But
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > > >> > > > Ok, about your proposal: I'm
> not a
> > > big
> > > > > fan
> > > > > > > > too,
> > > > > > > > > > > when it
> > > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > > >> > > > clients directly in protocol
> > schema.
> > > > And
> > > > > > > also
> > > > > > > > > I'm
> > > > > > > > > > > not
> > > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > > >> > > > auto.create.topics.enable is a
> > > server
> > > > > side
> > > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > > >> > > > in future versions, add this
> > setting
> > > > to
> > > > > > > > producer
> > > > > > > > > > and
> > > > > > > > > > > > > based
> > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > > >> > > > UnknownTopic create topic
> > explicitly
> > > > by
> > > > > a
> > > > > > > > > separate
> > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because we
> > want
> > > > to
> > > > > > > > support
> > > > > > > > > > > batching
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > > >> > > > want to give descriptive error
> > > > messages
> > > > > > for
> > > > > > > > > > clients.
> > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > > >> > > > to construct such messages (e.g.
> > > > > > AdminClient
> > > > > > > > > layer
> > > > > > > > > > > can
> > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > > > >> > > > means two cases: either invalid
> > > > number -
> > > > > > > e.g.
> > > > > > > > > -1;
> > > > > > > > > > or
> > > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > > >> > > > partitions argument wasn't) - I
> > > > wrapped
> > > > > > > > > responses
> > > > > > > > > > in
> > > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > > >> > > > other ideas, this was just
> initial
> > > > > > version.
> > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change
> > > that
> > > > to
> > > > > > > > > probably
> > > > > > > > > > > some
> > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM,
> > Jun
> > > > > Rao <
> > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > 101. That's what I was
> thinking
> > > too,
> > > > > but
> > > > > > > it
> > > > > > > > > may
> > > > > > > > > > > not be
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > > >> > > > > we can let it not trigger auto
> > > topic
> > > > > > > > creation.
> > > > > > > > > > > Then,
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > > UnknownTopicException,
> > > > > it
> > > > > > > can
> > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > > >> > > > > createTopicRequest for auto
> > topic
> > > > > > > creation.
> > > > > > > > On
> > > > > > > > > > the
> > > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > > >> > > > > never issue
> createTopicRequest.
> > > This
> > > > > > works
> > > > > > > > > when
> > > > > > > > > > > auto
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > > >> > > > > enabled on the broker side.
> > > > However, I
> > > > > > am
> > > > > > > > not
> > > > > > > > > > > sure how
> > > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > > >> > > > > when auto topic creation is
> > > disabled
> > > > > on
> > > > > > > the
> > > > > > > > > > broker
> > > > > > > > > > > > > side.
> > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > >> > > > > want to have a way to manually
> > > > create
> > > > > a
> > > > > > > > topic,
> > > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > >> > > > > commands. However, then we
> need
> > a
> > > > way
> > > > > to
> > > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > > >> > > > > issued from the producer
> clients
> > > and
> > > > > the
> > > > > > > > admin
> > > > > > > > > > > tools.
> > > > > > > > > > > > > > May
> > > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > >> > > > > new field in
> createTopicRequest
> > > and
> > > > > set
> > > > > > it
> > > > > > > > > > > differently
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > >> > > > > client and the admin client.
> > > > However,
> > > > > I
> > > > > > am
> > > > > > > > not
> > > > > > > > > > > sure if
> > > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring existing
> > > > requests
> > > > > > is a
> > > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > > >> > > > > posted some comments in
> > > KAFKA-1927.
> > > > We
> > > > > > > will
> > > > > > > > > > > probably
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > > >> > > > > first, before adding the new
> > logic
> > > > in
> > > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird that
> we
> > > > return
> > > > > > > > > exception
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > >> > > > > we should either return error
> > code
> > > > or
> > > > > > > throw
> > > > > > > > an
> > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > > >> > > > > 102.2. We probably shouldn't
> > > > > explicitly
> > > > > > > use
> > > > > > > > > the
> > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > > >> > > > > Not every request evolution
> > > requires
> > > > > an
> > > > > > > api
> > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08
> AM,
> > > > > Andrii
> > > > > > > > > > Biletskyi
> > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > Thanks for you comments.
> > Answers
> > > > > > inline:
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > 100. There are a few fields
> > such
> > > > as
> > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized
> > that
> > > > are
> > > > > > > > > > represented
> > > > > > > > > > > as a
> > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > >> > > > > > > composite structures in
> > json.
> > > > > Could
> > > > > > we
> > > > > > > > > > flatten
> > > > > > > > > > > > > them
> > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > Yes, now with Admin Client
> > this
> > > > > looks
> > > > > > a
> > > > > > > > bit
> > > > > > > > > > > weird.
> > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > > >> > > > > > ReassignPartitionCommand
> > accepts
> > > > > input
> > > > > > > in
> > > > > > > > > > json,
> > > > > > > > > > > we
> > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > > >> > > > > > interfaces unchanged, where
> > > > > possible.
> > > > > > > > > > > > > > > > > > >> > > > > > If we port it to
> deserialized
> > > > > format,
> > > > > > in
> > > > > > > > CLI
> > > > > > > > > > > (/tools
> > > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > > >> > > > > > json library since /tools is
> > > > written
> > > > > > in
> > > > > > > > java
> > > > > > > > > > and
> > > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > > >> > > > > > provided by a user. Can we
> > > quickly
> > > > > > agree
> > > > > > > > on
> > > > > > > > > > what
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> TopicMetadataRequest
> > > v1
> > > > > > still
> > > > > > > > > > trigger
> > > > > > > > > > > auto
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now
> that
> > > we
> > > > > > have a
> > > > > > > > > > > separate
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > >> > > > > > > you thought about how the
> > new
> > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > > producer/consumer
> > > > > > > > > > > client,
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> ideally,
> > > we
> > > > > > don't
> > > > > > > > want
> > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto
> > topic
> > > > > > > creation.
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > I agree, this strange logic
> > > should
> > > > > be
> > > > > > > > fixed.
> > > > > > > > > > > I'm not
> > > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > > >> > > > > > correct me if I'm wrong, but
> > it
> > > > > > doesn't
> > > > > > > > look
> > > > > > > > > > > like a
> > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > > >> > > > > > leverage AdminClient for
> that
> > in
> > > > > > > Producer
> > > > > > > > > and
> > > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant getting
> > rid
> > > > of
> > > > > > > scala
> > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > >> > > > > > > like
> > HeartbeatRequestAndHeader
> > > > and
> > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> > > adding
> > > > > the
> > > > > > > new
> > > > > > > > > > > requests
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > >> > > > > > > However, the long term
> plan
> > is
> > > > to
> > > > > > get
> > > > > > > > rid
> > > > > > > > > of
> > > > > > > > > > > all
> > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > java request/response in
> the
> > > > > client.
> > > > > > > > Since
> > > > > > > > > > > this
> > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > > > > requests,
> > > > > > > > > perhaps
> > > > > > > > > > we
> > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing
> scala
> > > > > requests
> > > > > > > > first
> > > > > > > > > > > before
> > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks like I
> > misunderstood
> > > > the
> > > > > > > point
> > > > > > > > of
> > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > > >> > > > > > rework that. The only thing
> is
> > > > that
> > > > > I
> > > > > > > > don't
> > > > > > > > > > see
> > > > > > > > > > > any
> > > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > > >> > > > > > least one existing protocol
> > > > message.
> > > > > > > Thus,
> > > > > > > > > as
> > > > > > > > > > I
> > > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > > > > > >> > > > > > Re porting all existing
> RQ/RP
> > in
> > > > > this
> > > > > > > > patch.
> > > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > > > >> > > > > > requirement to have Admin
> KIP
> > > > done,
> > > > > > I'm
> > > > > > > > > afraid
> > > > > > > > > > > this
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > > > >> > > > > > There are 13 protocol
> messages
> > > and
> > > > > all
> > > > > > > > that
> > > > > > > > > > > would
> > > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > > > >> > > > > > intensive manual testing,
> no?
> > > I'm
> > > > > > afraid
> > > > > > > > I'm
> > > > > > > > > > > not the
> > > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > > > >> > > > > > Kafka core internals :). Let
> > me
> > > > know
> > > > > > > your
> > > > > > > > > > > thoughts
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40
> > AM,
> > > > Jun
> > > > > > > Rao <
> > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > 100. There are a few
> fields
> > > such
> > > > > as
> > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized
> > that
> > > > are
> > > > > > > > > > represented
> > > > > > > > > > > as a
> > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > >> > > > > > > composite structures in
> > json.
> > > > > Could
> > > > > > we
> > > > > > > > > > flatten
> > > > > > > > > > > > > them
> > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> > TopicMetadataRequest
> > > > v1
> > > > > > > still
> > > > > > > > > > > trigger
> > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now
> that
> > > we
> > > > > > have a
> > > > > > > > > > > separate
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > >> > > > > > > you thought about how the
> > new
> > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > > producer/consumer
> > > > > > > > > > > client,
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> ideally,
> > > we
> > > > > > don't
> > > > > > > > want
> > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto
> > topic
> > > > > > > creation.
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant
> getting
> > > rid
> > > > > of
> > > > > > > > scala
> > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > >> > > > > > > like
> > HeartbeatRequestAndHeader
> > > > and
> > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> > > adding
> > > > > the
> > > > > > > new
> > > > > > > > > > > requests
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > >> > > > > > > However, the long term
> plan
> > is
> > > > to
> > > > > > get
> > > > > > > > rid
> > > > > > > > > of
> > > > > > > > > > > all
> > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > java request/response in
> the
> > > > > client.
> > > > > > > > Since
> > > > > > > > > > > this
> > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > > > > requests,
> > > > > > > > > perhaps
> > > > > > > > > > we
> > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing
> scala
> > > > > requests
> > > > > > > > first
> > > > > > > > > > > before
> > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at
> 3:37
> > > PM,
> > > > > > > Andrii
> > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > >
> andrii.biletskyi@stealth.ly
> > >
> > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > As said above - I list
> > again
> > > > all
> > > > > > > > > comments
> > > > > > > > > > > from
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > > >> > > > > > > > can see what's left and
> > > > finalize
> > > > > > all
> > > > > > > > > > pending
> > > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is much needed
> > > > > > > functionality,
> > > > > > > > > but
> > > > > > > > > > > there
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > > >> > > > > > > > really think these
> > protocols
> > > > > > > through.
> > > > > > > > We
> > > > > > > > > > > really
> > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > > >> > > > > > > > of well thought-out,
> > > > orthoganol
> > > > > > > apis.
> > > > > > > > > For
> > > > > > > > > > > this
> > > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > > >> > > > > > > > important to think
> through
> > > the
> > > > > end
> > > > > > > > state
> > > > > > > > > > > even if
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > won't implement in the
> > first
> > > > > > phase.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Definitely behind
> this.
> > > > Would
> > > > > > > > > > appreciate
> > > > > > > > > > > if
> > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > > >> > > > > > > > how this can be
> improved.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's please please
> > > please
> > > > > wait
> > > > > > > > until
> > > > > > > > > > we
> > > > > > > > > > > have
> > > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > > >> > > > > > > > to the new java protocol
> > > > > > > definitions.
> > > > > > > > If
> > > > > > > > > > we
> > > > > > > > > > > add
> > > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > > >> > > > > > > > objects that is just
> > > > generating
> > > > > > more
> > > > > > > > > work
> > > > > > > > > > > for
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > patch -
> > > > > > > removed
> > > > > > > > > > scala
> > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This proposal
> > introduces
> > > a
> > > > > new
> > > > > > > type
> > > > > > > > > of
> > > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > > >> > > > > > > > inconsistent with
> > everything
> > > > > else
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > > >> > > > > > > > other marker value. You
> > > could
> > > > > > argue
> > > > > > > > > either
> > > > > > > > > > > way
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > > for consistency. For
> > clients
> > > > > that
> > > > > > > > > > > implemented
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > > >> > > > > > > > than our scala code
> these
> > > > basic
> > > > > > > > > primitives
> > > > > > > > > > > are
> > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > patch -
> > > > > > > removed
> > > > > > > > > > > MaybeOf
> > > > > > > > > > > > > > type
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This
> > > seems
> > > > > to
> > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > >> > > > > > > > brokers, topics, and
> > > > > partitions. I
> > > > > > > > think
> > > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > > >> > > > > > > > ClusterMetadataRequest
> (or
> > > > just
> > > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > controller. Or are there
> > > other
> > > > > > > things
> > > > > > > > we
> > > > > > > > > > > could
> > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: I agree. Updated the
> > KIP.
> > > > > Let's
> > > > > > > > > extends
> > > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a tendency to
> > try
> > > > to
> > > > > > > make a
> > > > > > > > > lot
> > > > > > > > > > > of
> > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > particular nodes. This
> > adds
> > > a
> > > > > lot
> > > > > > of
> > > > > > > > > > burden
> > > > > > > > > > > for
> > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > > >> > > > > > > > sounds easy but each
> > > discovery
> > > > > can
> > > > > > > > fail
> > > > > > > > > in
> > > > > > > > > > > many
> > > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > > >> > > > > > > > full state machine to do
> > > > > right). I
> > > > > > > > think
> > > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > > >> > > > > > > > commands and ideally as
> > many
> > > > of
> > > > > > the
> > > > > > > > > other
> > > > > > > > > > > apis
> > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > > >> > > > > > > > brokers and just
> redirect
> > to
> > > > the
> > > > > > > > > > controller
> > > > > > > > > > > on
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > > >> > > > > > > > there would be a general
> > way
> > > > to
> > > > > > > > > > encapsulate
> > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a very
> interesting
> > > > idea,
> > > > > > but
> > > > > > > > > seems
> > > > > > > > > > > there
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > > >> > > > > > > > feature (like
> performance
> > > > > > > > > considerations,
> > > > > > > > > > > how
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > > >> > > > > > > > I believe this shouldn't
> > be
> > > a
> > > > > > > blocker.
> > > > > > > > > If
> > > > > > > > > > > this
> > > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > > >> > > > > > > > point it won't affect
> > Admin
> > > > > > changes
> > > > > > > -
> > > > > > > > at
> > > > > > > > > > > least
> > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We should probably
> > > > normalize
> > > > > > the
> > > > > > > > key
> > > > > > > > > > > value
> > > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > >> > > > > > > > than embedding a new
> > > > formatting.
> > > > > > So
> > > > > > > > two
> > > > > > > > > > > strings
> > > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > patch -
> > > > > > > > > normalized
> > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the postcondition
> of
> > > > these
> > > > > > > APIs
> > > > > > > > > that
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > > the command has been
> > > > completed?
> > > > > It
> > > > > > > is
> > > > > > > > a
> > > > > > > > > > lot
> > > > > > > > > > > more
> > > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > >> > > > > > > > been completed so you
> know
> > > > that
> > > > > if
> > > > > > > you
> > > > > > > > > > > create a
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > it you won't get an
> > > exception
> > > > > > about
> > > > > > > > > there
> > > > > > > > > > > being
> > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: For long running
> > requests
> > > > > (like
> > > > > > > > > > reassign
> > > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > > > >> > > > > > > > command has begun - so
> we
> > > > don't
> > > > > > > block
> > > > > > > > > the
> > > > > > > > > > > > > client.
> > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > > >> > > > > > > > topic commands, this
> will
> > be
> > > > > > > > refactored
> > > > > > > > > > and
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > > >> > > > > > > > immediately, since the
> > > > > Controller
> > > > > > > will
> > > > > > > > > > serve
> > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up ticket
> > > KAFKA-1777).
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe topic and
> list
> > > > > topics
> > > > > > > > > > duplicate
> > > > > > > > > > > a
> > > > > > > > > > > > > lot
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > > >> > > > > > > > request. Is there a
> reason
> > > to
> > > > > give
> > > > > > > > back
> > > > > > > > > > > topics
> > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > > >> > > > > > > > like if we just make the
> > > > > > > > post-condition
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > topic is deleted that
> will
> > > get
> > > > > rid
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > > need
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > patch -
> > > > > > > removed
> > > > > > > > > > > topics
> > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we consider
> > > batching
> > > > > > these
> > > > > > > > > > > requests?
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > allow multiple
> operations
> > to
> > > > be
> > > > > > > > batched.
> > > > > > > > > > My
> > > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > >> > > > > > > > we will get a lot of
> code
> > > that
> > > > > > does
> > > > > > > > > > > something
> > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > > >> > > > > > > > this code will work
> great
> > > when
> > > > > you
> > > > > > > > test
> > > > > > > > > > on 5
> > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > please
> > > > > check
> > > > > > > > "Topic
> > > > > > > > > > > Admin
> > > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I think we should
> also
> > > > > discuss
> > > > > > > how
> > > > > > > > > we
> > > > > > > > > > > want
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > > >> > > > > > > > client api for these
> > > > operations.
> > > > > > > > > Currently
> > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > > >> > > > > > > > is totally sketchy. I
> > think
> > > we
> > > > > > > > probably
> > > > > > > > > > need
> > > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > > >> > > > > > > > that exposes
> > administrative
> > > > > > > > > functionality.
> > > > > > > > > > > We
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > properly test the new
> > apis,
> > > I
> > > > > > > suspect.
> > > > > > > > > We
> > > > > > > > > > > should
> > > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > please
> > > > > check
> > > > > > > > "Admin
> > > > > > > > > > > Client"
> > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The other
> information
> > > that
> > > > > > would
> > > > > > > > be
> > > > > > > > > > > really
> > > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > >> > > > > > > > information about
> > > > > partitions--how
> > > > > > > much
> > > > > > > > > > data
> > > > > > > > > > > is
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > > >> > > > > > > > the segment offsets,
> what
> > is
> > > > the
> > > > > > > > log-end
> > > > > > > > > > > offset
> > > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > > >> > > > > > > > the compaction point,
> > etc. I
> > > > > think
> > > > > > > > that
> > > > > > > > > > done
> > > > > > > > > > > > > right
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > successor to the very
> > > awkward
> > > > > > > > > > OffsetRequest
> > > > > > > > > > > we
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > > > >> > > > > > > > be resolved in a
> separate
> > > KIP
> > > > /
> > > > > > jira
> > > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 12. Generally we can do
> > good
> > > > > error
> > > > > > > > > > handling
> > > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > > > >> > > > > > > > messages. I.e. generally
> > the
> > > > > > client
> > > > > > > > has
> > > > > > > > > > the
> > > > > > > > > > > > > > context
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > > >> > > > > > > > an error that the topic
> > > > doesn't
> > > > > > > exist
> > > > > > > > to
> > > > > > > > > > say
> > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > >> > > > > > > > than "error code 14" (or
> > > > > > whatever).
> > > > > > > > > Maybe
> > > > > > > > > > > there
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > > >> > > > > > > > this is hard? If we want
> > to
> > > > add
> > > > > > > > > > server-side
> > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > > >> > > > > > > > need to do this in a
> > > > consistent
> > > > > > way
> > > > > > > > > across
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > please
> > > > > check
> > > > > > > > > > "Protocol
> > > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > comprehensive,
> > fine-grained
> > > > list
> > > > > > of
> > > > > > > > > error
> > > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe topic
> > request:
> > > it
> > > > > > would
> > > > > > > > be
> > > > > > > > > > > great to
> > > > > > > > > > > > > > go
> > > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex for
> this
> > > > > request.
> > > > > > > For
> > > > > > > > > > > example,
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > > >> > > > > > > > the topic command is to
> > list
> > > > all
> > > > > > > > topics
> > > > > > > > > > > whose
> > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex then we
> > > have
> > > > to
> > > > > > > first
> > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > > >> > > > > > > > description info and
> then
> > > > filter
> > > > > > at
> > > > > > > > the
> > > > > > > > > > > client
> > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in
> create
> > > > topic:
> > > > > > > this
> > > > > > > > is
> > > > > > > > > > > related
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > > >> > > > > > > > maybe we can add another
> > > > > metadata
> > > > > > > K-V
> > > > > > > > or
> > > > > > > > > > > just a
> > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > > >> > > > > > > > with config K-V in
> create
> > > > topic
> > > > > > like
> > > > > > > > we
> > > > > > > > > > did
> > > > > > > > > > > for
> > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > > >> > > > > > > > field can be quite
> useful
> > in
> > > > > > storing
> > > > > > > > > > > information
> > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > > >> > > > > > > > who issue the create
> > > command,
> > > > > etc,
> > > > > > > > which
> > > > > > > > > > is
> > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant setting.
> Then
> > > in
> > > > > the
> > > > > > > > > describe
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > > >> > > > > > > > on regex of the metadata
> > > > field.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: As discussed it is
> very
> > > > > > > interesting
> > > > > > > > > but
> > > > > > > > > > > can
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > > >> > > > > > > > we have some basic
> > > > functionality
> > > > > > > > there.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today all the admin
> > > > > operations
> > > > > > > are
> > > > > > > > > > > async in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > >> > > > > > > > return once it is
> written
> > in
> > > > ZK,
> > > > > > and
> > > > > > > > > that
> > > > > > > > > > > is why
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > > testUtil.waitForTopicCreated()
> > > > > > > /
> > > > > > > > > > verify
> > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > > >> > > > > > > > request, etc. With admin
> > > > > requests
> > > > > > we
> > > > > > > > > could
> > > > > > > > > > > add a
> > > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > > >> > > > > > > > synchronous requests;
> when
> > > it
> > > > is
> > > > > > > > turned
> > > > > > > > > > on,
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > > >> > > > > > > > until the request has
> been
> > > > > > > completed.
> > > > > > > > > And
> > > > > > > > > > > for
> > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > > >> > > > > > > > "token" field in the
> > > response,
> > > > > and
> > > > > > > > then
> > > > > > > > > > only
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > > >> > > > > > > > verification request"
> with
> > > the
> > > > > > given
> > > > > > > > > token
> > > > > > > > > > > to
> > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: I see your point. My
> > idea
> > > > was
> > > > > > to
> > > > > > > > > > provide
> > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > > >> > > > > > > > long running request,
> > where
> > > > > > needed.
> > > > > > > We
> > > > > > > > > can
> > > > > > > > > > > do it
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > > >> > > > > > > > concern is that
> > introducing
> > > a
> > > > > > token
> > > > > > > we
> > > > > > > > > > again
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > > >> > > > > > > > to do similar thing
> > > > introducing
> > > > > > > single
> > > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > > > >> > > > > > > > this idea because we
> > wanted
> > > to
> > > > > > have
> > > > > > > > > schema
> > > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > > > > > >> > > > > > > > a) have fixed schema but
> > > > > introduce
> > > > > > > > each
> > > > > > > > > > > time new
> > > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > > > > > > > > > >> > > > > > > > b) use one request for
> > > > > > verification
> > > > > > > > but
> > > > > > > > > > > > > generalize
> > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine with whatever
> > > > decision
> > > > > > > > > community
> > > > > > > > > > > come
> > > > > > > > > > > > > to.
> > > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > > > > > > >> > > > > > > > 16. Specifically for
> > > > ownership,
> > > > > I
> > > > > > > > think
> > > > > > > > > > the
> > > > > > > > > > > plan
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > > >> > > > > > > > like you are describing
> > ACL)
> > > > via
> > > > > > an
> > > > > > > > > > external
> > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > > >> > > > > > > > I remember KIP-11
> > described
> > > > > this,
> > > > > > > but
> > > > > > > > I
> > > > > > > > > > > can't
> > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no problem. Not
> > > sure
> > > > > > though
> > > > > > > > how
> > > > > > > > > > we
> > > > > > > > > > > are
> > > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > > >> > > > > > > > will be committed first
> > and
> > > > > > include
> > > > > > > > > > changes
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I added this
> note
> > to
> > > > > "Open
> > > > > > > > > > > Questions"
> > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at
> > > 12:34
> > > > > AM,
> > > > > > > > Andrii
> > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > andrii.biletskyi@stealth.ly
> > > >
> > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > Today I uploaded the
> > patch
> > > > > that
> > > > > > > > covers
> > > > > > > > > > > some of
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf
> > optional
> > > > > type
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - switched to java
> > > protocol
> > > > > > > > > definitions
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - simplified messages
> > > > > > (normalized
> > > > > > > > > > configs,
> > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > I also updated the
> KIP-4
> > > > with
> > > > > > > > > respective
> > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin
> Operations
> > > ->
> > > > > > > updated
> > > > > > > > > Wire
> > > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove
> ClusterMetadata
> > > ->
> > > > > > > changed
> > > > > > > > to
> > > > > > > > > > > extend
> > > > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin Client ->
> > updated
> > > my
> > > > > > > initial
> > > > > > > > > > > proposal
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - Error codes ->
> > proposed
> > > > > > > > fine-grained
> > > > > > > > > > > error
> > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > I will also send a
> > > separate
> > > > > > email
> > > > > > > to
> > > > > > > > > > > cover all
> > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015
> at
> > > 9:26
> > > > > PM,
> > > > > > > > Gwen
> > > > > > > > > > > Shapira
> > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> It actually specifies
> > > > changes
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are
> > consistent
> > > in
> > > > > > this
> > > > > > > > > regard
> > > > > > > > > > > will
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015
> at
> > > > 12:21
> > > > > > PM,
> > > > > > > > > Gwen
> > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Specifically for
> > > > > ownership, I
> > > > > > > > think
> > > > > > > > > > the
> > > > > > > > > > > > > plan
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > like you are
> > describing
> > > > > ACL)
> > > > > > > via
> > > > > > > > an
> > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I remember KIP-11
> > > > described
> > > > > > > this,
> > > > > > > > > > but I
> > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I think
> > > KIP-4
> > > > > > > focuses
> > > > > > > > > on
> > > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > exists from Kafka
> > > > brokers,
> > > > > > not
> > > > > > > on
> > > > > > > > > > > adding
> > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > should exist but
> > > doesn't
> > > > > yet?
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12,
> 2015
> > at
> > > > > 6:37
> > > > > > > AM,
> > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just want to
> > > elaborate a
> > > > > bit
> > > > > > > > more
> > > > > > > > > on
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> describe-topic
> based
> > > on
> > > > > > > config /
> > > > > > > > > > > metadata
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The
> > > main
> > > > > > > > motivation
> > > > > > > > > > is
> > > > > > > > > > > to
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms, which
> I
> > > > think
> > > > > is
> > > > > > > > quite
> > > > > > > > > > > > > important
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> architecture:
> today
> > > > anyone
> > > > > > can
> > > > > > > > > > create
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> there is no
> concept
> > or
> > > > > > > > "ownership"
> > > > > > > > > > of
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> users. For
> example,
> > at
> > > > > > > LinkedIn
> > > > > > > > we
> > > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some casual topic
> > name
> > > > > > prefix,
> > > > > > > > > which
> > > > > > > > > > > is a
> > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale our
> > > customers.
> > > > It
> > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > great
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> topics
> > > that
> > > > > is
> > > > > > > > > created
> > > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> topics
> > > > whose
> > > > > > > > > retention
> > > > > > > > > > > time
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> topics
> > > > whose
> > > > > > > > writable
> > > > > > > > > > > group
> > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> authorization),
> > etc..
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One possible way
> to
> > > > > achieve
> > > > > > > this
> > > > > > > > > is
> > > > > > > > > > to
> > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> create-topic
> > request,
> > > > > whose
> > > > > > > > value
> > > > > > > > > > will
> > > > > > > > > > > > > also
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic; then
> > > > > describe-topics
> > > > > > > can
> > > > > > > > > > > choose to
> > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2) config
> K-V
> > > > > > matching,
> > > > > > > > 3)
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5,
> 2015
> > at
> > > > > 4:37
> > > > > > > PM,
> > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for the
> > > updated
> > > > > > wiki.
> > > > > > > A
> > > > > > > > > few
> > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error
> > description
> > > in
> > > > > > > > > response: I
> > > > > > > > > > > think
> > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> several different
> > > error
> > > > > > cases
> > > > > > > > > then
> > > > > > > > > > we
> > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In general
> > the
> > > > > > > errorCode
> > > > > > > > > > > itself
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> describing the
> > server
> > > > > side
> > > > > > > > > errors.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic
> > > > > request:
> > > > > > it
> > > > > > > > > would
> > > > > > > > > > > be
> > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex
> > for
> > > > this
> > > > > > > > > request.
> > > > > > > > > > > For
> > > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic command
> > is
> > > to
> > > > > > list
> > > > > > > > all
> > > > > > > > > > > topics
> > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex
> > then
> > > > we
> > > > > > have
> > > > > > > > to
> > > > > > > > > > > first
> > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> description info
> > and
> > > > then
> > > > > > > > filter
> > > > > > > > > at
> > > > > > > > > > > the
> > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in
> > > > create
> > > > > > > topic:
> > > > > > > > > > this
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can add
> > > > another
> > > > > > > > metadata
> > > > > > > > > > > K-V or
> > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with config K-V
> in
> > > > create
> > > > > > > topic
> > > > > > > > > > like
> > > > > > > > > > > we
> > > > > > > > > > > > > did
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field can be
> quite
> > > > useful
> > > > > > in
> > > > > > > > > > storing
> > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue the
> > create
> > > > > > command,
> > > > > > > > > etc,
> > > > > > > > > > > which
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant
> > setting.
> > > > > Then
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of the
> > > > metadata
> > > > > > > field.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all the
> > > admin
> > > > > > > > operations
> > > > > > > > > > are
> > > > > > > > > > > > > async
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> return once it is
> > > > written
> > > > > > in
> > > > > > > > ZK,
> > > > > > > > > > and
> > > > > > > > > > > that
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> request, etc.
> With
> > > > admin
> > > > > > > > requests
> > > > > > > > > > we
> > > > > > > > > > > > > could
> > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous
> > requests;
> > > > > when
> > > > > > it
> > > > > > > > is
> > > > > > > > > > > turned
> > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until the request
> > has
> > > > > been
> > > > > > > > > > > completed. And
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token" field in
> > the
> > > > > > > response,
> > > > > > > > > and
> > > > > > > > > > > then
> > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> verification
> > request"
> > > > > with
> > > > > > > the
> > > > > > > > > > given
> > > > > > > > > > > > > token
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has been
> completed.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for
> extending
> > > > > > Metadata
> > > > > > > > > > request
> > > > > > > > > > > to
> > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> information, and
> > then
> > > > we
> > > > > > can
> > > > > > > > > remove
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3,
> 2015
> > > at
> > > > > > 10:23
> > > > > > > > AM,
> > > > > > > > > > Joel
> > > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for
> sending
> > > > that
> > > > > > out
> > > > > > > > Joe
> > > > > > > > > -
> > > > > > > > > > I
> > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so if
> > > notes
> > > > > can
> > > > > > be
> > > > > > > > > sent
> > > > > > > > > > > out
> > > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02,
> > 2015
> > > > at
> > > > > > > > > 09:16:13AM
> > > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for
> > sending
> > > > > this
> > > > > > > out
> > > > > > > > > Joe.
> > > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2,
> > > 2015
> > > > at
> > > > > > > 6:46
> > > > > > > > > AM,
> > > > > > > > > > > Joe
> > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just
> > sent
> > > > > out a
> > > > > > > > > google
> > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I
> > found
> > > > > > working
> > > > > > > > on
> > > > > > > > > a
> > > > > > > > > > > KIP.
> > > > > > > > > > > > > If
> > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me know
> > and
> > > > can
> > > > > > > update
> > > > > > > > > it,
> > > > > > > > > > > np.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We should do
> > > this
> > > > > > every
> > > > > > > > > > Tuesday
> > > > > > > > > > > @
> > > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to
> make a
> > > > > google
> > > > > > > > > account
> > > > > > > > > > > so we
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in progress
> > and
> > > > > > related
> > > > > > > > JIRA
> > > > > > > > > > > that
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb
> > 24,
> > > > 2015
> > > > > > at
> > > > > > > > 2:59
> > > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay
> on
> > > > > Google
> > > > > > > > > hangouts
> > > > > > > > > > > that
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> available
> on
> > > > > youtube.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb
> > 24,
> > > > > 2015
> > > > > > at
> > > > > > > > > 11:49
> > > > > > > > > > > AM,
> > > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > jholoman@cloudera.com
> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're
> happy
> > > to
> > > > > send
> > > > > > > > out a
> > > > > > > > > > > Webex
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions
> if
> > > > there
> > > > > > is
> > > > > > > > > > > interest and
> > > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue,
> Feb
> > > 24,
> > > > > > 2015
> > > > > > > at
> > > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's
> try
> > > to
> > > > > get
> > > > > > > the
> > > > > > > > > > > technical
> > > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there
> is
> > > some
> > > > > > > benefit
> > > > > > > > > to
> > > > > > > > > > > live
> > > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post
> > > > > > > instructions
> > > > > > > > > and
> > > > > > > > > > > give
> > > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> working.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday
> > at
> > > > that
> > > > > > > time
> > > > > > > > > > would
> > > > > > > > > > > work
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue,
> > Feb
> > > > 24,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > 8:18
> > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> Weekly
> > > > would
> > > > > be
> > > > > > > > great
> > > > > > > > > > > maybe
> > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I
> don't
> > > > mind
> > > > > > > google
> > > > > > > > > > > hangout
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we
> know
> > > the
> > > > > > > apache
> > > > > > > > > irc
> > > > > > > > > > > > > channel
> > > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes?
> > We
> > > > can
> > > > > > pull
> > > > > > > > > > > transcripts
> > > > > > > > > > > > > > too
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it
> > > helpful
> > > > > for
> > > > > > > > > things.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~
> > > Joestein
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On
> Tue,
> > > Feb
> > > > > 24,
> > > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > 11:10
> > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > jay.kreps@gmail.com
> > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> We'd
> > > > talked
> > > > > > > about
> > > > > > > > > > > doing a
> > > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > generalizing
> > > > > > > > that a
> > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > everyone
> > > > > > > > spending a
> > > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> sync
> > up
> > > > > once
> > > > > > a
> > > > > > > > > week.
> > > > > > > > > > I
> > > > > > > > > > > > > think
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > stuff,
> > > > make
> > > > > > > sure
> > > > > > > > we
> > > > > > > > > > > are on
> > > > > > > > > > > > > > top
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > issues,
> > > > > etc.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We
> > can
> > > > make
> > > > > > it
> > > > > > > > > > publicly
> > > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > likes.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any
> > > > > interest
> > > > > > in
> > > > > > > > > doing
> > > > > > > > > > > this?
> > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> -Jay
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On
> > Tue,
> > > > Feb
> > > > > > 24,
> > > > > > > > > 2015
> > > > > > > > > > at
> > > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> Hi
> > > all,
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > I've
> > > > > > updated
> > > > > > > > KIP
> > > > > > > > > > > page,
> > > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > some
> > > > > > > > >
> > > > > > > > ...
> > > > > > > >
> > > > > > > > [Message clipped]
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
Jun,

The case I am most concerned about is create. We are proposing to get rid
of the confusing "auto create on metadata fetch" behavior. To make this
work every producer will need to support the create topic request. If each
client has to implement the create and then do a non-blocking poll with
exponential backoff to find out when the request is complete I'm concerned
that will be a bit complex to do right and significantly add to the client
burden.

-Jay

On Fri, Mar 20, 2015 at 3:18 PM, Jun Rao <ju...@confluent.io> wrote:

> Andrii,
>
> A few points.
>
> 1. Create/Alter can typically complete quickly. So, it's possible to make
> the request block until it's completed. However, currently, doing this at
> the broker is a bit involved. To make Create block, we will need to add
> some callbacks in KafkaController. This is possible. However, the
> controller logic currently is pretty completed. It would probably be better
> if we clean it up first before adding more complexity to it. Alter is even
> trickier. Adding partition is currently handled through KafkaController. So
> it can be dealt with in a similar way. However, Alter config is done
> completely differently. It doesn't go through the controller. Instead, each
> broker listens to ZooKeeper directly. So, it's not clear if there is an
> easy way on the broker to figure out whether a config is applied on every
> broker.
>
> 2. Delete can potentially take long if a replica to be deleted is offline.
> PreferredLeader/PartitionReassign can also take long. So, we can't really
> make those requests block on the broker.
>
> As you can see, at this moment it's not easy to make all admin requests
> block on the broker. So, if we want the blocking feature in the admin
> utility in the short term, doing the completion check at the admin client
> is probably an easier route, even though it may not be ideal.
>
> Thanks,
>
> Jun
>
> On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Jun,
> >
> > I see your point. But wouldn't that lead to a "fat" client
> implementations?
> > Suppose someone would like to implement client for Admin Wire protocol.
> > Not only people will have to code quite complicated logic like "send
> > describe
> > request to each broker" (again state machin?) but it will also mean
> people
> > must understand internal kafka logic related to topic storage and how
> > information is propageted from the controller to brokers.
> > I see this like a dilemma between having a concise Wire Protocol and
> > self-sufficient API to make client implementations simple.
> > I don't have a win-win solution though.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> >
> > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > For 1), 2) and 3), blocking would probably mean that the new metadata
> is
> > > propagated to every broker. To achieve that, the client can keep
> issuing
> > > the describe topic request to every broker until it sees the new
> metadata
> > > in the response.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Hm, actually the ticket you linked, Guozhang, brings as back
> > > > to the problem what should be considered a post-condition for
> > > > each of the admin commands.
> > > > In my understanding:
> > > >
> > > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > > (Not the controller picked up changes from zk and broadcasted
> > > > LeaderAndIsr and UpdateMetadata)
> > > >
> > > > 2) AlterTopic - same as 1) - broker changed assignment data
> > > > in zookeeper or created admin path for topic config change
> > > >
> > > > 3) DeleteTopic - admin path /admin/delete_topics is created
> > > >
> > > > 4) ReassignPartitions and PreferredReplica - corresponding admin
> > > > path is created
> > > >
> > > > Now what can be considered a completed operation from the client's
> > > > perspective?
> > > > 1) Topic is created once corresponding data is in zk
> > > > (I remember there were some thoughts that it'd be good to consider
> > > > topic created once all replicas receive information about it and thus
> > > > clients can produce/consume from it, but as was discussed this seems
> > > > to be a hard thing to do)
> > > >
> > > > 2) Probably same as 1), so right after AlterTopic is issued
> > > >
> > > > 3) The topic has been removed from /brokers/topics
> > > >
> > > > 4) ReassignPartitions and PrefferedReplica were discussed earlier -
> > > > in short the former is completed once partition state info in zk
> > matches
> > > > reassignment request and admin path is empty, the latter - once data
> > > > in zk shows that head of assignned replicas of the partition and
> leader
> > > > is the same replica
> > > >
> > > > Thoughts?
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > I think while loop is fine for supporting blocking, just that we
> need
> > > to
> > > > > add back off to avoid bombarding brokers with DescribeTopic
> requests.
> > > > >
> > > > > Also I have linked KAFKA-1125
> > > > > <https://issues.apache.org/jira/browse/KAFKA-1125> to your
> proposal,
> > > and
> > > > > when KAFKA-1694 is done this ticket can also be closed.
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Great.
> > > > > > I want to elaborate this a bit more, to see we are on the same
> page
> > > > > > concerning the client code.
> > > > > >
> > > > > > So with all topic commands being async a client (AdminClient in
> our
> > > > > > case or any other other client people would like to implement) to
> > > > support
> > > > > > a blocking operation (which seems to be a natural use-case e.g.
> for
> > > > topic
> > > > > > creation): would have to do:
> > > > > > 1. issue CreateTopicRequest
> > > > > > 2. if successful, in a "while" loop send DescribeTopicRequest and
> > > > > > break the loop once all topics are returned in response (or upon
> > > > > timeout).
> > > > > > 3. if unsuccessful throw exception
> > > > > > Would it be okay?
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > >
> > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Andrii,
> > > > > > >
> > > > > > > I think you are right. It seems that only ReassignPartitions
> > needs
> > > a
> > > > > > > separate verification request.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >
> > > > > > > > Guys,
> > > > > > > > I like this idea too. Let's stick with that. I'll update KIP
> > > > > > accordingly.
> > > > > > > >
> > > > > > > > I was also thinking we can avoid adding dedicated status
> check
> > > > > > > > requests for topic commands. - We have everything in
> > > DescribeTopic
> > > > > > > > for that! E.g.:
> > > > > > > > User issued CreateTopic - to check the status client sends
> > > > > > DescribeTopic
> > > > > > > > and checks whether is something returned for that topic. The
> > same
> > > > for
> > > > > > > > alteration, deletion.
> > > > > > > > Btw, PreferredReplica status can be also checked with
> > > > > > > DescribeTopicRequest
> > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > For ReassignPartitions as discussed we'll need to have a
> > separate
> > > > > > > Verify...
> > > > > > > > request.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Andrii Biletskyi
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> > > wangguoz@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > +1 on broker writing to ZK for async handling. I was
> thinking
> > > > that
> > > > > in
> > > > > > > the
> > > > > > > > > end state the admin requests would be eventually sent to
> > > > controller
> > > > > > > > either
> > > > > > > > > through re-routing or clients discovering them, instead of
> > > > letting
> > > > > > > > > controller listen on ZK admin path. But thinking about it a
> > > > second
> > > > > > > time,
> > > > > > > > I
> > > > > > > > > think it is actually simpler to let controller manage
> > > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > > >
> > > > > > > > > Guozhang
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > > jjkoshy.w@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > +1 as well. I think it helps to keep the rerouting
> approach
> > > > > > > orthogonal
> > > > > > > > > > to this KIP.
> > > > > > > > > >
> > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps
> wrote:
> > > > > > > > > > > I'm +1 on Jun's suggestion as long as it can work for
> all
> > > the
> > > > > > > > requests.
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <
> > jun@confluent.io
> > > >
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Andrii,
> > > > > > > > > > > >
> > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > >
> > > > > > > > > > > > (a) Admin requests can be sent to and handled by any
> > > > broker.
> > > > > > > > > > > > (b) Admin requests are processed asynchronously, at
> > least
> > > > for
> > > > > > > now.
> > > > > > > > > > That is,
> > > > > > > > > > > > when the client gets a response, it just means that
> the
> > > > > request
> > > > > > > is
> > > > > > > > > > > > initiated, but not necessarily completed. Then, it's
> up
> > > to
> > > > > the
> > > > > > > > client
> > > > > > > > > > to
> > > > > > > > > > > > issue another request to check the status for
> > completion.
> > > > > > > > > > > >
> > > > > > > > > > > > To support (a), we were thinking of doing request
> > > > forwarding
> > > > > to
> > > > > > > the
> > > > > > > > > > > > controller (utilizing KAFKA-1912). I am making an
> > > > alternative
> > > > > > > > > proposal.
> > > > > > > > > > > > Basically, the broker can just write to ZooKeeper to
> > > inform
> > > > > the
> > > > > > > > > > controller
> > > > > > > > > > > > about the request. For example, to handle
> > > > > > partitionReassignment,
> > > > > > > > the
> > > > > > > > > > broker
> > > > > > > > > > > > will just write the requested partitions to
> > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > (like what AdminUtils currently does) and then send a
> > > > > response
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > > client. This shouldn't take long and the
> implementation
> > > > will
> > > > > be
> > > > > > > > > simpler
> > > > > > > > > > > > than forwarding the requests to the controller
> through
> > > RPC.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Jun,
> > > > > > > > > > > > >
> > > > > > > > > > > > > I might be wrong but didn't we agree we will let
> any
> > > > broker
> > > > > > > from
> > > > > > > > > the
> > > > > > > > > > > > > cluster handle *long-running* admin requests (at
> this
> > > > time
> > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > and
> > > > > > > > > > > > > reassignPartitions), via zk admin path. Thus
> > > CreateTopics
> > > > > etc
> > > > > > > > > should
> > > > > > > > > > be
> > > > > > > > > > > > > sent
> > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> > > > > jun@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I think we agreed that those admin requests can
> be
> > > > issued
> > > > > > to
> > > > > > > > any
> > > > > > > > > > > > broker.
> > > > > > > > > > > > > > Because of that, there doesn't seem to be a
> strong
> > > need
> > > > > to
> > > > > > > know
> > > > > > > > > the
> > > > > > > > > > > > > > controller. So, perhaps we can proceed by not
> > making
> > > > any
> > > > > > > change
> > > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > > format of TMR right now. When we start using
> create
> > > > topic
> > > > > > > > request
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > producer, we will need a new version of TMR that
> > > > doesn't
> > > > > > > > trigger
> > > > > > > > > > auto
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > As a first cut implementation, I think the broker
> > can
> > > > > just
> > > > > > > > write
> > > > > > > > > > to ZK
> > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > requests, instead of forwarding them to the
> > > controller.
> > > > > > This
> > > > > > > > will
> > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > For (1) yes we will circle back on that shortly
> > > after
> > > > > > > syncing
> > > > > > > > > up
> > > > > > > > > > in
> > > > > > > > > > > > > > > person. I think it is close to getting
> committed
> > > > > although
> > > > > > > > > > development
> > > > > > > > > > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > There is one more item we covered at the
> hangout.
> > > > i.e.,
> > > > > > > > whether
> > > > > > > > > > we
> > > > > > > > > > > > > > > want to add the coordinator to the topic
> metadata
> > > > > > response
> > > > > > > or
> > > > > > > > > > provide
> > > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > There are two reasons I think we should try and
> > > avoid
> > > > > > > adding
> > > > > > > > > the
> > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > > > > - If we finally do request rerouting in Kafka
> > then
> > > > the
> > > > > > > field
> > > > > > > > > > would
> > > > > > > > > > > > add
> > > > > > > > > > > > > > >   little to no value. (It still helps to have a
> > > > > separate
> > > > > > > > > > > > > > >   ClusterMetadataRequest to query for
> > cluster-wide
> > > > > > > > information
> > > > > > > > > > such
> > > > > > > > > > > > as
> > > > > > > > > > > > > > >   'which broker is the controller?' as Joe
> > > > mentioned.)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I think it would be cleaner to have an explicit
> > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > that you can send to any broker in order to
> > obtain
> > > > the
> > > > > > > > > controller
> > > > > > > > > > > > (and
> > > > > > > > > > > > > > > in the future possibly other cluster-wide
> > > > > information). I
> > > > > > > > think
> > > > > > > > > > the
> > > > > > > > > > > > > > > main argument against doing this and instead
> > adding
> > > > it
> > > > > to
> > > > > > > the
> > > > > > > > > > topic
> > > > > > > > > > > > > > > metadata response was convenience - i.e., you
> > don't
> > > > > have
> > > > > > to
> > > > > > > > > > discover
> > > > > > > > > > > > > > > the controller in advance. However, I don't see
> > > much
> > > > > > actual
> > > > > > > > > > > > > > > benefit/convenience in this and in fact think
> it
> > > is a
> > > > > > > > > non-issue.
> > > > > > > > > > Let
> > > > > > > > > > > > > > > me know if I'm overlooking something here.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > As an example, say we need to initiate
> partition
> > > > > > > reassignment
> > > > > > > > > by
> > > > > > > > > > > > > > > issuing the new ReassignPartitionsRequest to
> the
> > > > > > controller
> > > > > > > > > > (assume
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > already have the desired manual partition
> > > > assignment).
> > > > > > If
> > > > > > > we
> > > > > > > > > > are to
> > > > > > > > > > > > > > > augment topic metadata response then the flow
> be
> > > > > > something
> > > > > > > > like
> > > > > > > > > > this
> > > > > > > > > > > > :
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Issue topic metadata request to any broker
> (and
> > > > > > discover
> > > > > > > > the
> > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > - Connect to controller if required (i.e., if
> the
> > > > > broker
> > > > > > > > above
> > > > > > > > > !=
> > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > - Issue the partition reassignment request to
> the
> > > > > > > controller.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > With an explicit cluster metadata request it
> > would
> > > > be:
> > > > > > > > > > > > > > > - Issue cluster metadata request to any broker
> > > > > > > > > > > > > > > - Connect to controller if required (i.e., if
> the
> > > > > broker
> > > > > > > > above
> > > > > > > > > !=
> > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > - Issue the partition reassignment request
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > So it seems to add little practical value and
> > > bloats
> > > > > > topic
> > > > > > > > > > metadata
> > > > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > The other angle to this is the following - is
> it
> > a
> > > > > matter
> > > > > > > of
> > > > > > > > > > naming?
> > > > > > > > > > > > > > > Should we just rename topic metadata
> > > request/response
> > > > > to
> > > > > > > just
> > > > > > > > > > > > > > > MetadataRequest/Response and add cluster
> metadata
> > > to
> > > > > it?
> > > > > > By
> > > > > > > > > that
> > > > > > > > > > same
> > > > > > > > > > > > > > > token should we also allow querying for the
> > > consumer
> > > > > > > > > coordinator
> > > > > > > > > > (and
> > > > > > > > > > > > > > > in future transaction coordinator) as well?
> This
> > > > leads
> > > > > > to a
> > > > > > > > > > bloated
> > > > > > > > > > > > > > > request which isn't very appealing and
> altogether
> > > > > > > confusing.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun
> Rao
> > > > > wrote:
> > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1. I just realized that in order to start
> > working
> > > > on
> > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > need to merge the changes to
> > OffsetCommitRequest
> > > > > (from
> > > > > > > > 0.8.2)
> > > > > > > > > > to
> > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > This is planned to be done as part of
> > KAFKA-1634.
> > > > So,
> > > > > > we
> > > > > > > > will
> > > > > > > > > > need
> > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 2. Thinking about this a bit more, if the
> > > semantic
> > > > of
> > > > > > > those
> > > > > > > > > > "write"
> > > > > > > > > > > > > > > > requests is async (i.e., after the client
> gets
> > a
> > > > > > > response,
> > > > > > > > it
> > > > > > > > > > just
> > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > that the operation is initiated, but not
> > > > necessarily
> > > > > > > > > > completed), we
> > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > really need to forward the requests to the
> > > > > controller.
> > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > receiving broker can just write the operation
> > to
> > > ZK
> > > > > as
> > > > > > > the
> > > > > > > > > > admin
> > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > line tool previously does. This will simplify
> > the
> > > > > > > > > > implementation.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 8. There is another implementation detail for
> > > > > describe
> > > > > > > > topic.
> > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > want to read the topic config from the broker
> > > > cache,
> > > > > > > > instead
> > > > > > > > > of
> > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > Currently, every broker reads the topic-level
> > > > config
> > > > > > for
> > > > > > > > all
> > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > However, it ignores those for topics not
> hosted
> > > on
> > > > > > > itself.
> > > > > > > > > So,
> > > > > > > > > > we
> > > > > > > > > > > > may
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > to change TopicConfigManager a bit so that it
> > > > caches
> > > > > > the
> > > > > > > > > > configs
> > > > > > > > > > > > for
> > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii
> > > Biletskyi <
> > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala requests
> objects,
> > > use
> > > > > java
> > > > > > > > > > protocol
> > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > >     A: Gwen kindly took that (KAFKA-1927).
> > It's
> > > > > > > important
> > > > > > > > > to
> > > > > > > > > > > > speed
> > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > >          there since this ticket blocks
> other
> > > > > > important
> > > > > > > > > > changes.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute facility vs client
> > > > > > maintaining
> > > > > > > > > > cluster
> > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > >     A: Jay has added pseudo code to
> > KAFKA-1912
> > > -
> > > > > need
> > > > > > > to
> > > > > > > > > > consider
> > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > >         easy to implement as a server-side
> > > > feature
> > > > > > > > > (comments
> > > > > > > > > > are
> > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > > > > > > > > > >     A: This might be useful for clients,
> add
> > > this
> > > > > to
> > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > > > > > > > > > >     A: I will add proposed by Jun solution
> > > (using
> > > > > > > > clientId
> > > > > > > > > > for
> > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs
> grabbing
> > > all
> > > > > > > > protocol
> > > > > > > > > > > > changes
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > >     A: It was decided to try to gather all
> > > > changes
> > > > > to
> > > > > > > > > > protocol
> > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > >         In case of TMR it worth checking:
> > > > > KAFKA-2020
> > > > > > > and
> > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to deserialize
> > user's
> > > > > input
> > > > > > in
> > > > > > > > CLI
> > > > > > > > > > tool.
> > > > > > > > > > > > > > > > >     A: Use jackson for that, /tools project
> > is
> > > a
> > > > > > > separate
> > > > > > > > > > jar so
> > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs generic
> > > status
> > > > > > check
> > > > > > > > > > command.
> > > > > > > > > > > > > > > > >      A: For long-running requests like
> > reassign
> > > > > > > > partitions
> > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > >          it makes sense to introduce it.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >  Please add, correct me if I missed
> > something.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii
> > > > Biletskyi <
> > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > You are right, I removed ClusterMetadata
> > > > because
> > > > > we
> > > > > > > > have
> > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > what we need in TopicMetadata. Also, as
> Jay
> > > > > pointed
> > > > > > > out
> > > > > > > > > > > > earlier,
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > would like to have "orthogonal" API, but
> at
> > > the
> > > > > > same
> > > > > > > > time
> > > > > > > > > > we
> > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > But I like your idea and even have some
> > other
> > > > > > > arguments
> > > > > > > > > for
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > There is also DescribeTopicRequest which
> > was
> > > > > > proposed
> > > > > > > > in
> > > > > > > > > > this
> > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > it returns topic configs, partitions,
> > > > replication
> > > > > > > > factor
> > > > > > > > > > plus
> > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > leader replica. The later part is really
> > > > already
> > > > > > > there
> > > > > > > > in
> > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > So again we'll have to add stuff to TMR,
> > not
> > > to
> > > > > > > > duplicate
> > > > > > > > > > some
> > > > > > > > > > > > > info
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > newly added requests. However, this way
> > we'll
> > > > end
> > > > > > up
> > > > > > > > with
> > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > request which returns cluster metadata,
> > topic
> > > > > > > > replication
> > > > > > > > > > and
> > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > plus partition replication data. Seems
> > > logical
> > > > to
> > > > > > > split
> > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > - ClusterMetadata (brokers + controller,
> > > maybe
> > > > > smth
> > > > > > > > else)
> > > > > > > > > > > > > > > > > > - TopicMetadata (topic info + partition
> > > > details)
> > > > > > > > > > > > > > > > > > But since current TMR is involved in lots
> > of
> > > > > places
> > > > > > > > > > (including
> > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > as I understand) this might be very
> serious
> > > > > change
> > > > > > > and
> > > > > > > > it
> > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel
> > Koshy <
> > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >> I may be missing some context but
> > hopefully
> > > > this
> > > > > > > will
> > > > > > > > > > also be
> > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > >> today: I thought the earlier proposal
> > where
> > > > > there
> > > > > > > was
> > > > > > > > an
> > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > >> ClusterMetadata request was clearer and
> > > > > explicit.
> > > > > > > > During
> > > > > > > > > > the
> > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > >> this thread I think the conclusion was
> > that
> > > > the
> > > > > > main
> > > > > > > > > need
> > > > > > > > > > was
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> controller information and that can be
> > > rolled
> > > > > into
> > > > > > > the
> > > > > > > > > > topic
> > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > >> response but that seems a bit irrelevant
> > to
> > > > > topic
> > > > > > > > > > metadata.
> > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > >> think the full broker-list is also
> > > irrelevant
> > > > to
> > > > > > > topic
> > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > >> it is already there and in use. I think
> > > there
> > > > is
> > > > > > > still
> > > > > > > > > > room
> > > > > > > > > > > > for
> > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > >> explicit ClusterMetadata request since
> > there
> > > > may
> > > > > > be
> > > > > > > > > other
> > > > > > > > > > > > > > > > > >> cluster-level information that we may
> want
> > > to
> > > > > add
> > > > > > > over
> > > > > > > > > > time
> > > > > > > > > > > > (and
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> have nothing to do with topic metadata).
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM
> +0200,
> > > > Andrii
> > > > > > > > > Biletskyi
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > 101. Okay, if you say that such use
> case
> > > is
> > > > > > > > > important. I
> > > > > > > > > > > > also
> > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > >> > using clientId for these purposes is
> > fine
> > > -
> > > > if
> > > > > > we
> > > > > > > > > > already
> > > > > > > > > > > > have
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > >> > as part of all Wire protocol messages,
> > why
> > > > not
> > > > > > use
> > > > > > > > > that.
> > > > > > > > > > > > > > > > > >> > I will update KIP-4 page if nobody has
> > > other
> > > > > > ideas
> > > > > > > > > > (which
> > > > > > > > > > > > may
> > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP
> > > > accordingly.
> > > > > I
> > > > > > > > think
> > > > > > > > > > we can
> > > > > > > > > > > > > add
> > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > >> > fine-grained error codes if some error
> > > code
> > > > > > > received
> > > > > > > > > in
> > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > >> > won't give enough context to return a
> > > > > > descriptive
> > > > > > > > > error
> > > > > > > > > > > > > message
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > Look forward to discussing all
> > outstanding
> > > > > > issues
> > > > > > > in
> > > > > > > > > > detail
> > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun
> > Rao
> > > <
> > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > 101. There may be a use case where
> you
> > > > only
> > > > > > want
> > > > > > > > the
> > > > > > > > > > > > topics
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > >> > > manually by admins. Currently, you
> can
> > > do
> > > > > that
> > > > > > > by
> > > > > > > > > > > > disabling
> > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> > > creation and issue topic creation
> from
> > > the
> > > > > > > > > > TopicCommand.
> > > > > > > > > > > > If
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > >> > > topic creation completely on the
> > broker
> > > > and
> > > > > > > don't
> > > > > > > > > > have a
> > > > > > > > > > > > way
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > >> > > between topic creation requests from
> > the
> > > > > > regular
> > > > > > > > > > clients
> > > > > > > > > > > > and
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > >> > > can't support manual topic creation
> > any
> > > > > more.
> > > > > > I
> > > > > > > > was
> > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > >> > > way of distinguishing the clients
> > making
> > > > the
> > > > > > > topic
> > > > > > > > > > > > creation
> > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > >> > > using clientId. For example, the
> admin
> > > > tool
> > > > > > can
> > > > > > > > set
> > > > > > > > > > it to
> > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > >> > > admin and the broker can treat that
> > > > clientId
> > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > Also, there is a related discussion
> in
> > > > > > > KAFKA-2020.
> > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > 1. If leader is not available, we
> set
> > > the
> > > > > > > > partition
> > > > > > > > > > level
> > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > >> > > 2. If a non-leader replica is not
> > > > available,
> > > > > > we
> > > > > > > > take
> > > > > > > > > > that
> > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > >> > > the assigned replica list and isr in
> > the
> > > > > > > response.
> > > > > > > > > As
> > > > > > > > > > an
> > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > >> > > doing that, we set the partition
> level
> > > > error
> > > > > > > code
> > > > > > > > to
> > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > This has a few problems. First,
> > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > >> > > an error, at least for the normal
> > > > > > > > producer/consumer
> > > > > > > > > > > > clients
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > >> > > to find out the leader. Second, it
> can
> > > > > happen
> > > > > > > that
> > > > > > > > > > both
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> > > another replica are not available at
> > the
> > > > > same
> > > > > > > > time.
> > > > > > > > > > There
> > > > > > > > > > > > is
> > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > >> > > to indicate both. Third, even if a
> > > replica
> > > > > is
> > > > > > > not
> > > > > > > > > > > > available,
> > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > >> > > useful to return its replica id
> since
> > > some
> > > > > > > clients
> > > > > > > > > > (e.g.
> > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > One way to address this issue is to
> > > always
> > > > > > > return
> > > > > > > > > the
> > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > >> > > leader, assigned replicas, and isr
> > > > > regardless
> > > > > > of
> > > > > > > > > > whether
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > >> > > broker is live or not. Since we also
> > > > return
> > > > > > the
> > > > > > > > list
> > > > > > > > > > of
> > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > client can figure out whether a
> leader
> > > or
> > > > a
> > > > > > > > replica
> > > > > > > > > is
> > > > > > > > > > > > live
> > > > > > > > > > > > > or
> > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > >> > > accordingly. This way, we don't need
> > to
> > > > set
> > > > > > the
> > > > > > > > > > partition
> > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > >> > > when the leader or a replica is not
> > > > > available.
> > > > > > > > This
> > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > >> > > protocol, but does change the
> > semantics.
> > > > > Since
> > > > > > > we
> > > > > > > > > are
> > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > >> > > of TopicMetadataRequest here, we can
> > > > > > potentially
> > > > > > > > > > piggyback
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > 102.1 For those types of errors due
> to
> > > > > invalid
> > > > > > > > > input,
> > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > >> > > guard it at parameter validation
> time
> > > and
> > > > > > throw
> > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > >> > > without even sending the request to
> > the
> > > > > > broker?
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM,
> > Andrii
> > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > 101. If I understand you
> correctly,
> > > you
> > > > > are
> > > > > > > > saying
> > > > > > > > > > > > future
> > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > >> > > > will be ported to TMR_V1) won't be
> > > able
> > > > to
> > > > > > > > > > automatically
> > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > >> > > > unconditionally remove topic
> > creation
> > > > from
> > > > > > > > there).
> > > > > > > > > > But
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > >> > > > Ok, about your proposal: I'm not a
> > big
> > > > fan
> > > > > > > too,
> > > > > > > > > > when it
> > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > >> > > > clients directly in protocol
> schema.
> > > And
> > > > > > also
> > > > > > > > I'm
> > > > > > > > > > not
> > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > >> > > > auto.create.topics.enable is a
> > server
> > > > side
> > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > >> > > > in future versions, add this
> setting
> > > to
> > > > > > > producer
> > > > > > > > > and
> > > > > > > > > > > > based
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > >> > > > UnknownTopic create topic
> explicitly
> > > by
> > > > a
> > > > > > > > separate
> > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because we
> want
> > > to
> > > > > > > support
> > > > > > > > > > batching
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > >> > > > want to give descriptive error
> > > messages
> > > > > for
> > > > > > > > > clients.
> > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > >> > > > to construct such messages (e.g.
> > > > > AdminClient
> > > > > > > > layer
> > > > > > > > > > can
> > > > > > > > > > > > > know
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > > >> > > > means two cases: either invalid
> > > number -
> > > > > > e.g.
> > > > > > > > -1;
> > > > > > > > > or
> > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > >> > > > partitions argument wasn't) - I
> > > wrapped
> > > > > > > > responses
> > > > > > > > > in
> > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > >> > > > other ideas, this was just initial
> > > > > version.
> > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change
> > that
> > > to
> > > > > > > > probably
> > > > > > > > > > some
> > > > > > > > > > > > > other
> > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM,
> Jun
> > > > Rao <
> > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > 101. That's what I was thinking
> > too,
> > > > but
> > > > > > it
> > > > > > > > may
> > > > > > > > > > not be
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > >> > > > > we can let it not trigger auto
> > topic
> > > > > > > creation.
> > > > > > > > > > Then,
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > UnknownTopicException,
> > > > it
> > > > > > can
> > > > > > > > > > explicitly
> > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > >> > > > > createTopicRequest for auto
> topic
> > > > > > creation.
> > > > > > > On
> > > > > > > > > the
> > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > >> > > > > never issue createTopicRequest.
> > This
> > > > > works
> > > > > > > > when
> > > > > > > > > > auto
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > >> > > > > enabled on the broker side.
> > > However, I
> > > > > am
> > > > > > > not
> > > > > > > > > > sure how
> > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > >> > > > > when auto topic creation is
> > disabled
> > > > on
> > > > > > the
> > > > > > > > > broker
> > > > > > > > > > > > side.
> > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > >> > > > > want to have a way to manually
> > > create
> > > > a
> > > > > > > topic,
> > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > >> > > > > commands. However, then we need
> a
> > > way
> > > > to
> > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > >> > > > > issued from the producer clients
> > and
> > > > the
> > > > > > > admin
> > > > > > > > > > tools.
> > > > > > > > > > > > > May
> > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > >> > > > > new field in createTopicRequest
> > and
> > > > set
> > > > > it
> > > > > > > > > > differently
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > >> > > > > client and the admin client.
> > > However,
> > > > I
> > > > > am
> > > > > > > not
> > > > > > > > > > sure if
> > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring existing
> > > requests
> > > > > is a
> > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > >> > > > > posted some comments in
> > KAFKA-1927.
> > > We
> > > > > > will
> > > > > > > > > > probably
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > >> > > > > first, before adding the new
> logic
> > > in
> > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird that we
> > > return
> > > > > > > > exception
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > >> > > > > we should either return error
> code
> > > or
> > > > > > throw
> > > > > > > an
> > > > > > > > > > > > exception
> > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > >> > > > > 102.2. We probably shouldn't
> > > > explicitly
> > > > > > use
> > > > > > > > the
> > > > > > > > > > > > request
> > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > >> > > > > Not every request evolution
> > requires
> > > > an
> > > > > > api
> > > > > > > > > > change.
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM,
> > > > Andrii
> > > > > > > > > Biletskyi
> > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly>
> > wrote:
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > Thanks for you comments.
> Answers
> > > > > inline:
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > 100. There are a few fields
> such
> > > as
> > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized
> that
> > > are
> > > > > > > > > represented
> > > > > > > > > > as a
> > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > >> > > > > > > composite structures in
> json.
> > > > Could
> > > > > we
> > > > > > > > > flatten
> > > > > > > > > > > > them
> > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > arrays/records?
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > Yes, now with Admin Client
> this
> > > > looks
> > > > > a
> > > > > > > bit
> > > > > > > > > > weird.
> > > > > > > > > > > > My
> > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > >> > > > > > ReassignPartitionCommand
> accepts
> > > > input
> > > > > > in
> > > > > > > > > json,
> > > > > > > > > > we
> > > > > > > > > > > > > want
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > >> > > > > > interfaces unchanged, where
> > > > possible.
> > > > > > > > > > > > > > > > > >> > > > > > If we port it to deserialized
> > > > format,
> > > > > in
> > > > > > > CLI
> > > > > > > > > > (/tools
> > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > >> > > > > > json library since /tools is
> > > written
> > > > > in
> > > > > > > java
> > > > > > > > > and
> > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > >> > > > > > provided by a user. Can we
> > quickly
> > > > > agree
> > > > > > > on
> > > > > > > > > what
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > 101. Does TopicMetadataRequest
> > v1
> > > > > still
> > > > > > > > > trigger
> > > > > > > > > > auto
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now that
> > we
> > > > > have a
> > > > > > > > > > separate
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > >> > > > > > > you thought about how the
> new
> > > > > > > > > > createTopicRequest
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > producer/consumer
> > > > > > > > > > client,
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > >> > > > > > > tools? For example, ideally,
> > we
> > > > > don't
> > > > > > > want
> > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto
> topic
> > > > > > creation.
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > I agree, this strange logic
> > should
> > > > be
> > > > > > > fixed.
> > > > > > > > > > I'm not
> > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > >> > > > > > correct me if I'm wrong, but
> it
> > > > > doesn't
> > > > > > > look
> > > > > > > > > > like a
> > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > >> > > > > > leverage AdminClient for that
> in
> > > > > > Producer
> > > > > > > > and
> > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant getting
> rid
> > > of
> > > > > > scala
> > > > > > > > > > classes
> > > > > > > > > > > > > > > > > >> > > > > > > like
> HeartbeatRequestAndHeader
> > > and
> > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> > adding
> > > > the
> > > > > > new
> > > > > > > > > > requests
> > > > > > > > > > > > for
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > >> > > > > > > However, the long term plan
> is
> > > to
> > > > > get
> > > > > > > rid
> > > > > > > > of
> > > > > > > > > > all
> > > > > > > > > > > > > those
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > java request/response in the
> > > > client.
> > > > > > > Since
> > > > > > > > > > this
> > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > > > requests,
> > > > > > > > perhaps
> > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing scala
> > > > requests
> > > > > > > first
> > > > > > > > > > before
> > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > Yes, looks like I
> misunderstood
> > > the
> > > > > > point
> > > > > > > of
> > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > >> > > > > > rework that. The only thing is
> > > that
> > > > I
> > > > > > > don't
> > > > > > > > > see
> > > > > > > > > > any
> > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > >> > > > > > least one existing protocol
> > > message.
> > > > > > Thus,
> > > > > > > > as
> > > > > > > > > I
> > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > > > > >> > > > > > Re porting all existing RQ/RP
> in
> > > > this
> > > > > > > patch.
> > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > > >> > > > > > requirement to have Admin KIP
> > > done,
> > > > > I'm
> > > > > > > > afraid
> > > > > > > > > > this
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > > >> > > > > > There are 13 protocol messages
> > and
> > > > all
> > > > > > > that
> > > > > > > > > > would
> > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > > >> > > > > > intensive manual testing, no?
> > I'm
> > > > > afraid
> > > > > > > I'm
> > > > > > > > > > not the
> > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > > >> > > > > > Kafka core internals :). Let
> me
> > > know
> > > > > > your
> > > > > > > > > > thoughts
> > > > > > > > > > > > on
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40
> AM,
> > > Jun
> > > > > > Rao <
> > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > 100. There are a few fields
> > such
> > > > as
> > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized
> that
> > > are
> > > > > > > > > represented
> > > > > > > > > > as a
> > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > >> > > > > > > composite structures in
> json.
> > > > Could
> > > > > we
> > > > > > > > > flatten
> > > > > > > > > > > > them
> > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > arrays/records?
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> TopicMetadataRequest
> > > v1
> > > > > > still
> > > > > > > > > > trigger
> > > > > > > > > > > > auto
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now that
> > we
> > > > > have a
> > > > > > > > > > separate
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > >> > > > > > > you thought about how the
> new
> > > > > > > > > > createTopicRequest
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > producer/consumer
> > > > > > > > > > client,
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > >> > > > > > > tools? For example, ideally,
> > we
> > > > > don't
> > > > > > > want
> > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto
> topic
> > > > > > creation.
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant getting
> > rid
> > > > of
> > > > > > > scala
> > > > > > > > > > classes
> > > > > > > > > > > > > > > > > >> > > > > > > like
> HeartbeatRequestAndHeader
> > > and
> > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> > adding
> > > > the
> > > > > > new
> > > > > > > > > > requests
> > > > > > > > > > > > for
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > >> > > > > > > However, the long term plan
> is
> > > to
> > > > > get
> > > > > > > rid
> > > > > > > > of
> > > > > > > > > > all
> > > > > > > > > > > > > those
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > java request/response in the
> > > > client.
> > > > > > > Since
> > > > > > > > > > this
> > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > > > requests,
> > > > > > > > perhaps
> > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing scala
> > > > requests
> > > > > > > first
> > > > > > > > > > before
> > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37
> > PM,
> > > > > > Andrii
> > > > > > > > > > Biletskyi
> > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly
> >
> > > > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > As said above - I list
> again
> > > all
> > > > > > > > comments
> > > > > > > > > > from
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > >> > > > > > > > can see what's left and
> > > finalize
> > > > > all
> > > > > > > > > pending
> > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is much needed
> > > > > > functionality,
> > > > > > > > but
> > > > > > > > > > there
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > >> > > > > > > > really think these
> protocols
> > > > > > through.
> > > > > > > We
> > > > > > > > > > really
> > > > > > > > > > > > > want
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > >> > > > > > > > of well thought-out,
> > > orthoganol
> > > > > > apis.
> > > > > > > > For
> > > > > > > > > > this
> > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > >> > > > > > > > important to think through
> > the
> > > > end
> > > > > > > state
> > > > > > > > > > even if
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > won't implement in the
> first
> > > > > phase.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Definitely behind this.
> > > Would
> > > > > > > > > appreciate
> > > > > > > > > > if
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's please please
> > please
> > > > wait
> > > > > > > until
> > > > > > > > > we
> > > > > > > > > > have
> > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > >> > > > > > > > to the new java protocol
> > > > > > definitions.
> > > > > > > If
> > > > > > > > > we
> > > > > > > > > > add
> > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > >> > > > > > > > objects that is just
> > > generating
> > > > > more
> > > > > > > > work
> > > > > > > > > > for
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > patch -
> > > > > > removed
> > > > > > > > > scala
> > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 3. This proposal
> introduces
> > a
> > > > new
> > > > > > type
> > > > > > > > of
> > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > >> > > > > > > > inconsistent with
> everything
> > > > else
> > > > > in
> > > > > > > the
> > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > >> > > > > > > > other marker value. You
> > could
> > > > > argue
> > > > > > > > either
> > > > > > > > > > way
> > > > > > > > > > > > but
> > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > >> > > > > > > > for consistency. For
> clients
> > > > that
> > > > > > > > > > implemented
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > >> > > > > > > > than our scala code these
> > > basic
> > > > > > > > primitives
> > > > > > > > > > are
> > > > > > > > > > > > > hard
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > patch -
> > > > > > removed
> > > > > > > > > > MaybeOf
> > > > > > > > > > > > > type
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This
> > seems
> > > > to
> > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > >> > > > > > > > brokers, topics, and
> > > > partitions. I
> > > > > > > think
> > > > > > > > > we
> > > > > > > > > > > > should
> > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > >> > > > > > > > ClusterMetadataRequest (or
> > > just
> > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > controller. Or are there
> > other
> > > > > > things
> > > > > > > we
> > > > > > > > > > could
> > > > > > > > > > > > add
> > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: I agree. Updated the
> KIP.
> > > > Let's
> > > > > > > > extends
> > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a tendency to
> try
> > > to
> > > > > > make a
> > > > > > > > lot
> > > > > > > > > > of
> > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > particular nodes. This
> adds
> > a
> > > > lot
> > > > > of
> > > > > > > > > burden
> > > > > > > > > > for
> > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > >> > > > > > > > sounds easy but each
> > discovery
> > > > can
> > > > > > > fail
> > > > > > > > in
> > > > > > > > > > many
> > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > >> > > > > > > > full state machine to do
> > > > right). I
> > > > > > > think
> > > > > > > > > we
> > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > >> > > > > > > > commands and ideally as
> many
> > > of
> > > > > the
> > > > > > > > other
> > > > > > > > > > apis
> > > > > > > > > > > > as
> > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > >> > > > > > > > brokers and just redirect
> to
> > > the
> > > > > > > > > controller
> > > > > > > > > > on
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > >> > > > > > > > there would be a general
> way
> > > to
> > > > > > > > > encapsulate
> > > > > > > > > > this
> > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a very interesting
> > > idea,
> > > > > but
> > > > > > > > seems
> > > > > > > > > > there
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > >> > > > > > > > feature (like performance
> > > > > > > > considerations,
> > > > > > > > > > how
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > >> > > > > > > > I believe this shouldn't
> be
> > a
> > > > > > blocker.
> > > > > > > > If
> > > > > > > > > > this
> > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > >> > > > > > > > point it won't affect
> Admin
> > > > > changes
> > > > > > -
> > > > > > > at
> > > > > > > > > > least
> > > > > > > > > > > > no
> > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 6. We should probably
> > > normalize
> > > > > the
> > > > > > > key
> > > > > > > > > > value
> > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > >> > > > > > > > than embedding a new
> > > formatting.
> > > > > So
> > > > > > > two
> > > > > > > > > > strings
> > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > patch -
> > > > > > > > normalized
> > > > > > > > > > > > configs
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the postcondition of
> > > these
> > > > > > APIs
> > > > > > > > that
> > > > > > > > > > the
> > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > >> > > > > > > > the command has been
> > > completed?
> > > > It
> > > > > > is
> > > > > > > a
> > > > > > > > > lot
> > > > > > > > > > more
> > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > >> > > > > > > > been completed so you know
> > > that
> > > > if
> > > > > > you
> > > > > > > > > > create a
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > it you won't get an
> > exception
> > > > > about
> > > > > > > > there
> > > > > > > > > > being
> > > > > > > > > > > > no
> > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: For long running
> requests
> > > > (like
> > > > > > > > > reassign
> > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > > >> > > > > > > > command has begun - so we
> > > don't
> > > > > > block
> > > > > > > > the
> > > > > > > > > > > > client.
> > > > > > > > > > > > > In
> > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > >> > > > > > > > topic commands, this will
> be
> > > > > > > refactored
> > > > > > > > > and
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > >> > > > > > > > immediately, since the
> > > > Controller
> > > > > > will
> > > > > > > > > serve
> > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up ticket
> > KAFKA-1777).
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe topic and list
> > > > topics
> > > > > > > > > duplicate
> > > > > > > > > > a
> > > > > > > > > > > > lot
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > >> > > > > > > > request. Is there a reason
> > to
> > > > give
> > > > > > > back
> > > > > > > > > > topics
> > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > >> > > > > > > > like if we just make the
> > > > > > > post-condition
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > topic is deleted that will
> > get
> > > > rid
> > > > > > of
> > > > > > > > the
> > > > > > > > > > need
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > patch -
> > > > > > removed
> > > > > > > > > > topics
> > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we consider
> > batching
> > > > > these
> > > > > > > > > > requests?
> > > > > > > > > > > > We
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > allow multiple operations
> to
> > > be
> > > > > > > batched.
> > > > > > > > > My
> > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > >> > > > > > > > we will get a lot of code
> > that
> > > > > does
> > > > > > > > > > something
> > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > >> > > > > > > > this code will work great
> > when
> > > > you
> > > > > > > test
> > > > > > > > > on 5
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> please
> > > > check
> > > > > > > "Topic
> > > > > > > > > > Admin
> > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 10. I think we should also
> > > > discuss
> > > > > > how
> > > > > > > > we
> > > > > > > > > > want
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > >> > > > > > > > client api for these
> > > operations.
> > > > > > > > Currently
> > > > > > > > > > > > people
> > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > >> > > > > > > > is totally sketchy. I
> think
> > we
> > > > > > > probably
> > > > > > > > > need
> > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > >> > > > > > > > that exposes
> administrative
> > > > > > > > functionality.
> > > > > > > > > > We
> > > > > > > > > > > > will
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > properly test the new
> apis,
> > I
> > > > > > suspect.
> > > > > > > > We
> > > > > > > > > > should
> > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> please
> > > > check
> > > > > > > "Admin
> > > > > > > > > > Client"
> > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 11. The other information
> > that
> > > > > would
> > > > > > > be
> > > > > > > > > > really
> > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > >> > > > > > > > information about
> > > > partitions--how
> > > > > > much
> > > > > > > > > data
> > > > > > > > > > is
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > >> > > > > > > > the segment offsets, what
> is
> > > the
> > > > > > > log-end
> > > > > > > > > > offset
> > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > >> > > > > > > > the compaction point,
> etc. I
> > > > think
> > > > > > > that
> > > > > > > > > done
> > > > > > > > > > > > right
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > successor to the very
> > awkward
> > > > > > > > > OffsetRequest
> > > > > > > > > > we
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > > >> > > > > > > > be resolved in a separate
> > KIP
> > > /
> > > > > jira
> > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 12. Generally we can do
> good
> > > > error
> > > > > > > > > handling
> > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > > >> > > > > > > > messages. I.e. generally
> the
> > > > > client
> > > > > > > has
> > > > > > > > > the
> > > > > > > > > > > > > context
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > >> > > > > > > > an error that the topic
> > > doesn't
> > > > > > exist
> > > > > > > to
> > > > > > > > > say
> > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > >> > > > > > > > than "error code 14" (or
> > > > > whatever).
> > > > > > > > Maybe
> > > > > > > > > > there
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > >> > > > > > > > this is hard? If we want
> to
> > > add
> > > > > > > > > server-side
> > > > > > > > > > > > error
> > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > >> > > > > > > > need to do this in a
> > > consistent
> > > > > way
> > > > > > > > across
> > > > > > > > > > the
> > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> please
> > > > check
> > > > > > > > > "Protocol
> > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > comprehensive,
> fine-grained
> > > list
> > > > > of
> > > > > > > > error
> > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe topic
> request:
> > it
> > > > > would
> > > > > > > be
> > > > > > > > > > great to
> > > > > > > > > > > > > go
> > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex for this
> > > > request.
> > > > > > For
> > > > > > > > > > example,
> > > > > > > > > > > > a
> > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > >> > > > > > > > the topic command is to
> list
> > > all
> > > > > > > topics
> > > > > > > > > > whose
> > > > > > > > > > > > > config
> > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex then we
> > have
> > > to
> > > > > > first
> > > > > > > > > > retrieve
> > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > >> > > > > > > > description info and then
> > > filter
> > > > > at
> > > > > > > the
> > > > > > > > > > client
> > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in create
> > > topic:
> > > > > > this
> > > > > > > is
> > > > > > > > > > related
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > >> > > > > > > > maybe we can add another
> > > > metadata
> > > > > > K-V
> > > > > > > or
> > > > > > > > > > just a
> > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > >> > > > > > > > with config K-V in create
> > > topic
> > > > > like
> > > > > > > we
> > > > > > > > > did
> > > > > > > > > > for
> > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > >> > > > > > > > field can be quite useful
> in
> > > > > storing
> > > > > > > > > > information
> > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > >> > > > > > > > who issue the create
> > command,
> > > > etc,
> > > > > > > which
> > > > > > > > > is
> > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant setting. Then
> > in
> > > > the
> > > > > > > > describe
> > > > > > > > > > topic
> > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > >> > > > > > > > on regex of the metadata
> > > field.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: As discussed it is very
> > > > > > interesting
> > > > > > > > but
> > > > > > > > > > can
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > >> > > > > > > > we have some basic
> > > functionality
> > > > > > > there.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today all the admin
> > > > operations
> > > > > > are
> > > > > > > > > > async in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > >> > > > > > > > return once it is written
> in
> > > ZK,
> > > > > and
> > > > > > > > that
> > > > > > > > > > is why
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > testUtil.waitForTopicCreated()
> > > > > > /
> > > > > > > > > verify
> > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > >> > > > > > > > request, etc. With admin
> > > > requests
> > > > > we
> > > > > > > > could
> > > > > > > > > > add a
> > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > >> > > > > > > > synchronous requests; when
> > it
> > > is
> > > > > > > turned
> > > > > > > > > on,
> > > > > > > > > > the
> > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > >> > > > > > > > until the request has been
> > > > > > completed.
> > > > > > > > And
> > > > > > > > > > for
> > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > >> > > > > > > > "token" field in the
> > response,
> > > > and
> > > > > > > then
> > > > > > > > > only
> > > > > > > > > > > > need
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > >> > > > > > > > verification request" with
> > the
> > > > > given
> > > > > > > > token
> > > > > > > > > > to
> > > > > > > > > > > > > check
> > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: I see your point. My
> idea
> > > was
> > > > > to
> > > > > > > > > provide
> > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > >> > > > > > > > long running request,
> where
> > > > > needed.
> > > > > > We
> > > > > > > > can
> > > > > > > > > > do it
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > >> > > > > > > > concern is that
> introducing
> > a
> > > > > token
> > > > > > we
> > > > > > > > > again
> > > > > > > > > > > > will
> > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > >> > > > > > > > to do similar thing
> > > introducing
> > > > > > single
> > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > > >> > > > > > > > this idea because we
> wanted
> > to
> > > > > have
> > > > > > > > schema
> > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > > > > >> > > > > > > > a) have fixed schema but
> > > > introduce
> > > > > > > each
> > > > > > > > > > time new
> > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > > > > > > > > >> > > > > > > > b) use one request for
> > > > > verification
> > > > > > > but
> > > > > > > > > > > > generalize
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine with whatever
> > > decision
> > > > > > > > community
> > > > > > > > > > come
> > > > > > > > > > > > to.
> > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > > > > > >> > > > > > > > 16. Specifically for
> > > ownership,
> > > > I
> > > > > > > think
> > > > > > > > > the
> > > > > > > > > > plan
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > >> > > > > > > > like you are describing
> ACL)
> > > via
> > > > > an
> > > > > > > > > external
> > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > >> > > > > > > > I remember KIP-11
> described
> > > > this,
> > > > > > but
> > > > > > > I
> > > > > > > > > > can't
> > > > > > > > > > > > find
> > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no problem. Not
> > sure
> > > > > though
> > > > > > > how
> > > > > > > > > we
> > > > > > > > > > are
> > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > >> > > > > > > > will be committed first
> and
> > > > > include
> > > > > > > > > changes
> > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I added this note
> to
> > > > "Open
> > > > > > > > > > Questions"
> > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at
> > 12:34
> > > > AM,
> > > > > > > Andrii
> > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > >> > > > > > > >
> andrii.biletskyi@stealth.ly
> > >
> > > > > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > Today I uploaded the
> patch
> > > > that
> > > > > > > covers
> > > > > > > > > > some of
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf
> optional
> > > > type
> > > > > > > > > > > > > > > > > >> > > > > > > > > - switched to java
> > protocol
> > > > > > > > definitions
> > > > > > > > > > > > > > > > > >> > > > > > > > > - simplified messages
> > > > > (normalized
> > > > > > > > > configs,
> > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > I also updated the KIP-4
> > > with
> > > > > > > > respective
> > > > > > > > > > > > changes
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin Operations
> > ->
> > > > > > updated
> > > > > > > > Wire
> > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove ClusterMetadata
> > ->
> > > > > > changed
> > > > > > > to
> > > > > > > > > > extend
> > > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin Client ->
> updated
> > my
> > > > > > initial
> > > > > > > > > > proposal
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > >> > > > > > > > > - Error codes ->
> proposed
> > > > > > > fine-grained
> > > > > > > > > > error
> > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > I will also send a
> > separate
> > > > > email
> > > > > > to
> > > > > > > > > > cover all
> > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at
> > 9:26
> > > > PM,
> > > > > > > Gwen
> > > > > > > > > > Shapira
> > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > >> > > > > > > > >> It actually specifies
> > > changes
> > > > > to
> > > > > > > the
> > > > > > > > > > Metadata
> > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are
> consistent
> > in
> > > > > this
> > > > > > > > regard
> > > > > > > > > > will
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at
> > > 12:21
> > > > > PM,
> > > > > > > > Gwen
> > > > > > > > > > > > Shapira
> > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > Specifically for
> > > > ownership, I
> > > > > > > think
> > > > > > > > > the
> > > > > > > > > > > > plan
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > like you are
> describing
> > > > ACL)
> > > > > > via
> > > > > > > an
> > > > > > > > > > > > external
> > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > I remember KIP-11
> > > described
> > > > > > this,
> > > > > > > > > but I
> > > > > > > > > > > > can't
> > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I think
> > KIP-4
> > > > > > focuses
> > > > > > > > on
> > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > exists from Kafka
> > > brokers,
> > > > > not
> > > > > > on
> > > > > > > > > > adding
> > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > should exist but
> > doesn't
> > > > yet?
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015
> at
> > > > 6:37
> > > > > > AM,
> > > > > > > > > > Guozhang
> > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just want to
> > elaborate a
> > > > bit
> > > > > > > more
> > > > > > > > on
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> describe-topic based
> > on
> > > > > > config /
> > > > > > > > > > metadata
> > > > > > > > > > > > in
> > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The
> > main
> > > > > > > motivation
> > > > > > > > > is
> > > > > > > > > > to
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms, which I
> > > think
> > > > is
> > > > > > > quite
> > > > > > > > > > > > important
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> architecture: today
> > > anyone
> > > > > can
> > > > > > > > > create
> > > > > > > > > > > > topics
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> there is no concept
> or
> > > > > > > "ownership"
> > > > > > > > > of
> > > > > > > > > > > > topics
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> users. For example,
> at
> > > > > > LinkedIn
> > > > > > > we
> > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some casual topic
> name
> > > > > prefix,
> > > > > > > > which
> > > > > > > > > > is a
> > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale our
> > customers.
> > > It
> > > > > > would
> > > > > > > > be
> > > > > > > > > > great
> > > > > > > > > > > > to
> > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics
> > that
> > > > is
> > > > > > > > created
> > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics
> > > whose
> > > > > > > > retention
> > > > > > > > > > time
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics
> > > whose
> > > > > > > writable
> > > > > > > > > > group
> > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> authorization),
> etc..
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One possible way to
> > > > achieve
> > > > > > this
> > > > > > > > is
> > > > > > > > > to
> > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> create-topic
> request,
> > > > whose
> > > > > > > value
> > > > > > > > > will
> > > > > > > > > > > > also
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic; then
> > > > describe-topics
> > > > > > can
> > > > > > > > > > choose to
> > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2) config K-V
> > > > > matching,
> > > > > > > 3)
> > > > > > > > > > metadata
> > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015
> at
> > > > 4:37
> > > > > > PM,
> > > > > > > > > > Guozhang
> > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for the
> > updated
> > > > > wiki.
> > > > > > A
> > > > > > > > few
> > > > > > > > > > > > comments
> > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error
> description
> > in
> > > > > > > > response: I
> > > > > > > > > > think
> > > > > > > > > > > > > if
> > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> several different
> > error
> > > > > cases
> > > > > > > > then
> > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In general
> the
> > > > > > errorCode
> > > > > > > > > > itself
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> describing the
> server
> > > > side
> > > > > > > > errors.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic
> > > > request:
> > > > > it
> > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > > great
> > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex
> for
> > > this
> > > > > > > > request.
> > > > > > > > > > For
> > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic command
> is
> > to
> > > > > list
> > > > > > > all
> > > > > > > > > > topics
> > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex
> then
> > > we
> > > > > have
> > > > > > > to
> > > > > > > > > > first
> > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> description info
> and
> > > then
> > > > > > > filter
> > > > > > > > at
> > > > > > > > > > the
> > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in
> > > create
> > > > > > topic:
> > > > > > > > > this
> > > > > > > > > > is
> > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can add
> > > another
> > > > > > > metadata
> > > > > > > > > > K-V or
> > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with config K-V in
> > > create
> > > > > > topic
> > > > > > > > > like
> > > > > > > > > > we
> > > > > > > > > > > > did
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field can be quite
> > > useful
> > > > > in
> > > > > > > > > storing
> > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue the
> create
> > > > > command,
> > > > > > > > etc,
> > > > > > > > > > which
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant
> setting.
> > > > Then
> > > > > in
> > > > > > > the
> > > > > > > > > > > > describe
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of the
> > > metadata
> > > > > > field.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all the
> > admin
> > > > > > > operations
> > > > > > > > > are
> > > > > > > > > > > > async
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> return once it is
> > > written
> > > > > in
> > > > > > > ZK,
> > > > > > > > > and
> > > > > > > > > > that
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> request, etc. With
> > > admin
> > > > > > > requests
> > > > > > > > > we
> > > > > > > > > > > > could
> > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous
> requests;
> > > > when
> > > > > it
> > > > > > > is
> > > > > > > > > > turned
> > > > > > > > > > > > on,
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until the request
> has
> > > > been
> > > > > > > > > > completed. And
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token" field in
> the
> > > > > > response,
> > > > > > > > and
> > > > > > > > > > then
> > > > > > > > > > > > > only
> > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> verification
> request"
> > > > with
> > > > > > the
> > > > > > > > > given
> > > > > > > > > > > > token
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for extending
> > > > > Metadata
> > > > > > > > > request
> > > > > > > > > > to
> > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> information, and
> then
> > > we
> > > > > can
> > > > > > > > remove
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015
> > at
> > > > > 10:23
> > > > > > > AM,
> > > > > > > > > Joel
> > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for sending
> > > that
> > > > > out
> > > > > > > Joe
> > > > > > > > -
> > > > > > > > > I
> > > > > > > > > > > > don't
> > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so if
> > notes
> > > > can
> > > > > be
> > > > > > > > sent
> > > > > > > > > > out
> > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02,
> 2015
> > > at
> > > > > > > > 09:16:13AM
> > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for
> sending
> > > > this
> > > > > > out
> > > > > > > > Joe.
> > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2,
> > 2015
> > > at
> > > > > > 6:46
> > > > > > > > AM,
> > > > > > > > > > Joe
> > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just
> sent
> > > > out a
> > > > > > > > google
> > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I
> found
> > > > > working
> > > > > > > on
> > > > > > > > a
> > > > > > > > > > KIP.
> > > > > > > > > > > > If
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me know
> and
> > > can
> > > > > > update
> > > > > > > > it,
> > > > > > > > > > np.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We should do
> > this
> > > > > every
> > > > > > > > > Tuesday
> > > > > > > > > > @
> > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to make a
> > > > google
> > > > > > > > account
> > > > > > > > > > so we
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in progress
> and
> > > > > related
> > > > > > > JIRA
> > > > > > > > > > that
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb
> 24,
> > > 2015
> > > > > at
> > > > > > > 2:59
> > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on
> > > > Google
> > > > > > > > hangouts
> > > > > > > > > > that
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> available on
> > > > youtube.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb
> 24,
> > > > 2015
> > > > > at
> > > > > > > > 11:49
> > > > > > > > > > AM,
> > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > jholoman@cloudera.com
> > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're happy
> > to
> > > > send
> > > > > > > out a
> > > > > > > > > > Webex
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions if
> > > there
> > > > > is
> > > > > > > > > > interest and
> > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb
> > 24,
> > > > > 2015
> > > > > > at
> > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try
> > to
> > > > get
> > > > > > the
> > > > > > > > > > technical
> > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there is
> > some
> > > > > > benefit
> > > > > > > > to
> > > > > > > > > > live
> > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post
> > > > > > instructions
> > > > > > > > and
> > > > > > > > > > give
> > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday
> at
> > > that
> > > > > > time
> > > > > > > > > would
> > > > > > > > > > work
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue,
> Feb
> > > 24,
> > > > > > 2015
> > > > > > > at
> > > > > > > > > > 8:18
> > > > > > > > > > > > AM,
> > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly
> > > would
> > > > be
> > > > > > > great
> > > > > > > > > > maybe
> > > > > > > > > > > > > like
> > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't
> > > mind
> > > > > > google
> > > > > > > > > > hangout
> > > > > > > > > > > > but
> > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we know
> > the
> > > > > > apache
> > > > > > > > irc
> > > > > > > > > > > > channel
> > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes?
> We
> > > can
> > > > > pull
> > > > > > > > > > transcripts
> > > > > > > > > > > > > too
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it
> > helpful
> > > > for
> > > > > > > > things.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~
> > Joestein
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue,
> > Feb
> > > > 24,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > 11:10
> > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> jay.kreps@gmail.com
> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd
> > > talked
> > > > > > about
> > > > > > > > > > doing a
> > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > generalizing
> > > > > > > that a
> > > > > > > > > > little
> > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > everyone
> > > > > > > spending a
> > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync
> up
> > > > once
> > > > > a
> > > > > > > > week.
> > > > > > > > > I
> > > > > > > > > > > > think
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> stuff,
> > > make
> > > > > > sure
> > > > > > > we
> > > > > > > > > > are on
> > > > > > > > > > > > > top
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> issues,
> > > > etc.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We
> can
> > > make
> > > > > it
> > > > > > > > > publicly
> > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> likes.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any
> > > > interest
> > > > > in
> > > > > > > > doing
> > > > > > > > > > this?
> > > > > > > > > > > > > If
> > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On
> Tue,
> > > Feb
> > > > > 24,
> > > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi
> > all,
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> I've
> > > > > updated
> > > > > > > KIP
> > > > > > > > > > page,
> > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> some
> > > > > > > >
> > > > > > > ...
> > > > > > >
> > > > > > > [Message clipped]
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
Hi, Andrii,

A few comments below.

201. I think we can evolve Cluster in a backward compatible way. We can
keep all existing methods on Cluster unchanged and add a new method like
configsForTopic(). This shouldn't add too much overhead since the configs
are returned as part of TMR.

300. ReassignPartitionRequest: Do we need this one or could we just fold it
under AlterTopicRequest?

301. CreateTopicRequest: We need to think through the auto topic creation
case. In the future, the producer client will issue CreateTopicRequest when
it identifies the topic doesn't exist. In this case, what value should the
client use for # partitions and the replication factor? One way is to
expose a new config on the producer side. However, the client may not know
how to configure this. Another way is to set these two values with -1 and
use the default values configured on the broker side (as we do today). If
we go with the latter, we need to change how we interpret the request a
bit. Currently, it expects either replicaAssignment is set or #
partitions/replication factor are set, which will no longer to true.

302. AlterTopicRequest: You included partitions in the request, but not
replicas. This seems inconsistent. If a client just wants to increase
partitions and/or replicas, but doesn't care about the actual replica
assignment, it can set those values and leave ReplicaAssignment empty. The
broker will then figure out the replica assignment.

303. AdminClient:
303.1 In the following, should we make this a batch api?
  public boolean topicExists(String topic) throws ApiException;
303.2 Do we still need verifyReassignPartitions() and
verifyPreferredReplicaLeaderElection()?
303.3 Shouldn't preferredReplicaLeaderElection() return Future<Map<String,
Errors>> as well?
303.4 Is it better to return Future<Map<String, Errors>> and Map<String,
DescribeTopicOutput> or sth like Future<TopicState> and
Future<TopicMetadata>? The latter is more consistent with our client api.

304. Command line interface:
304.1 In the following, should we specify a timeout in the blocking mode?
 bin/kafka.sh --reassign-partitions --reassignment-json-file
/user/file.json --blocking --broker-list <host : port>
304.2 Could --reassign-partitions be folded under --alter-topic?

305. Minor editorial stuff:
305.1 In the following, VerifyReassignParitionRequest is no longer valid.
"Status of the ReassignPartitionRequest can be checked with
VerifyReassignParitionRequest."
305.2 In the following, there is no optional field in our Schema definition
protocol.
"The schema consist of one optional field "

Thanks,

Jun


On Mon, Apr 6, 2015 at 8:20 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Hi all,
>
> Let me list remaining issues for tomorrow call.
>
> 210. ListTopicsRequest - do we need it?
> I agreed with Jun, it looks reasonable people would want
> to have lightweight request to get all topics. Also this may
> be related with the next item.
>
> 201. We need to include topic-level configuration somewhere -
> DescribeTopicRequest vs TopicMetadataRequest_V1.
> During the last call, I think, we agreed to go with TMR_V1.
> But now, when I have looked a bit at this approach I see some difficulties.
> Currently the TopicMetadataResponse is essentially
> org.apache.kafka.common.Cluster.
> We use Cluster instance to hold TopicMetadata in network client.
> Now, if we evolve TMR to version V1 to include topic-level configs
> it will break Cluster model (once we migrate clients). Also taking into
> account MetadataRequest is used in NetworkClient
> (= Producer & Consumer) are we okay with receiving some amount
> of irrelevant data (topic-level configs) which has nothing to do
> with cluster metadata? Can this affect clients performance?
> I think this should be discussed separately again.
>
> Also, I believe we are now more or less close to agree on Admin RQ/RP
> schema. Can we also talk about AdminClient API which is a part of
> public interfaces for these changes? I updated KIP to reflect current
> Wire Protocol schema, but I think we need to look carefully at it,
> especially
> the way we will handle async semantics of the Admin requests.
>
> Thanks,
> Andrii Biletskyi
>
>
>
> On Fri, Apr 3, 2015 at 8:14 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > There is one more item that we need to discuss.
> >
> > 210. Do we need a separate ListTopic request that just returns a list of
> > all topics. Currently, we piggyback this functionality in TMR. If the
> topic
> > list is empty in TMR, we return all topics and their metadata. The
> tradeoff
> > is the following. If we piggyback on TMR, there is one fewer request that
> > we have to deal with. On the other hand, it does mean that the response
> > size can be large (e.g., if you have 10K topics and a 1KB metadata per
> > topic, the response can be 10MB). There are probably cases (e.g., in
> > supporting wildcard topics in the new java consumer) that a client just
> > needs to see the full topic list, but not the metadata for every topic.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Mar 24, 2015 at 2:01 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > 201. I think it's probably easier to consolidate on DTR. If we evolve
> > TMR,
> > > the producer will be using the latest version of TMR. However, the
> latest
> > > version of TMR won't trigger auto topic creation. This means that we
> need
> > > to change the producer to use CreateTopicRequest. If we consolidate on
> > DTR.
> > > We can fix the producer logic later.
> > >
> > > 205. Also, there was a remaining item on how to get the controller
> info.
> > > We can either add the controller id to the response of DTR or create a
> > new
> > > cluster metadata request/response. The cluster meta response will then
> > > include the detailed information about every broker and the
> controller. I
> > > think the latter is probably better. For example, when we add the
> > > multi-port support (for security), the cluster meta response can
> include
> > > all ports for a broker. In DTR, we only need to return one broker port
> to
> > > the client.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Tue, Mar 24, 2015 at 12:26 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > >> Hi all,
> > >>
> > >> A short summary of our discussion:
> > >>
> > >> 201. DescribeTopicRequest(DTR) vs TopicMetadataRequest(TMR)
> > >>
> > >> *Issue*: Topic description should include topic-level configuration,
> > >> other information is available in TMR - leader, ISR, AR.
> > >> So now there are two options - either add full-fledged new DTR
> > >> that will include TMR + all needed information (which will be used
> then
> > >> instead TMR) or extend TMR to include topic-level configuration.
> > >>
> > >> *Resolution*: it was decided to vote on it: either a) *extend TMR* or
> b)
> > >> *add
> > >> DTR*.
> > >> I'd go with a) - we will have to roll out TMR_V1 but it's better than
> > >> having
> > >> "dead" requests if we add DTR and leave TMR unused in future versions
> > >>
> > >> 202. CreateTopicRequest sync/async semantics
> > >>
> > >> *Issue*: What should be a post-condition of Create/Alter/Delete
> > >> requests - the command has been completed or command has been
> > >> initiated?
> > >> Even though it looks reasonable that users would typically expect to
> > >> have topic created upon CreateTopicRequest the corner case is that
> > >> topic can be deleted right after creation, so received
> > CreateTopicResponse
> > >> cannot be a 100% guarantee topic is created and available, even if
> > >> CreateTopicRequest is synchronous. Also, making request blocking on
> > >> server may be a hard thing to do right now (check Jun's email from Mar
> > >> 21).
> > >>
> > >> *Resolution*: for now it makes sense to start with async semantics for
> > >> Create/Alter/Delete.
> > >>
> > >> 203. VerifyReassignPartition vs leverage DescribeTopicRequest
> > >>
> > >> *Issue*: DescribeTopicRequest (or TopicMetadaRequest) should be
> enough,
> > >> since controller doesn't remove partitions from
> > admin/reassign_partitions
> > >> until
> > >> the assignment actually matches target assignment so partition
> > >> reassignment
> > >> cannot fail for specific partition in the current implementation.
> > >> Everything else
> > >> ("Completed" and "In Progress" state) can be checked with assigned
> > >> partitions
> > >> field from DescribeTopicRequest.
> > >>
> > >> *Resolution*: remove VerifyReassignPartitionsRequest
> > >>
> > >> 204. Also it was discussed that after we finalize question 201 and
> port
> > >> java protocol definition (KAFKA-1927) I will split patch to separate
> > >> pieces,
> > >> first will cover Wire Protocol changes, since we have a common vision
> on
> > >> it.
> > >>
> > >> Please let me know your thoughts on q. 201!
> > >>
> > >> Thanks,
> > >> Andrii Biletskyi
> > >>
> > >>
> > >>
> > >>
> > >> On Tue, Mar 24, 2015 at 7:57 PM, Neha Narkhede <ne...@confluent.io>
> > wrote:
> > >>
> > >> > I'd like to resurface the discussion of sync vs async topic
> creation.
> > >> > Possibly, we can discuss that in today's KIP meeting. I am worried
> > about
> > >> > pushing the burden of handling async topic creation on the clients.
> > >> This is
> > >> > one of the most frequently asked questions on the mailing list and I
> > >> hope
> > >> > we find a way to address it as we separate the metadata request and
> > >> topic
> > >> > creation. Since topic creation doesn't take that long, there should
> > be a
> > >> > really easy way for clients to express their preference for a sync
> > topic
> > >> > creation request where the receipt of a response suggests completion
> > of
> > >> > topic creation.
> > >> >
> > >> > The behavior that I'm expecting is that if I'm a client and I choose
> > to
> > >> > create a topic synchronously, after I receive a response, my first
> > send
> > >> > request for that topic should *not* fail with some error.
> > >> >
> > >> > On Tue, Mar 24, 2015 at 8:02 AM, Jun Rao <ju...@confluent.io> wrote:
> > >> >
> > >> > > Andrii,
> > >> > >
> > >> > >
> > >> > > 111.4 Yes, we can discuss this in the meeting today.
> > >> > >
> > >> > > 112. The question is when will a PartitionReassignment fail.
> > >> Currently,
> > >> > it
> > >> > > can fail if the input is incorrect (e.g, invalid
> partition/replica).
> > >> > > However, this can be detected when the ReassignPartitionRequest is
> > >> > issued.
> > >> > > Once the process of partition reassignment is started, it will not
> > >> fail.
> > >> > > The controller just waits until the process completes. We can
> > improve
> > >> the
> > >> > > status reporting of partition reassignment. However, I am not sure
> > if
> > >> > that
> > >> > > needs to be tied to this KIP.
> > >> > >
> > >> > > 115.1 Yes, that makes sense. I misunderstood it. I thought
> > >> > > PreferredReplicaLeaderElectionInProgress and
> > >> ReassignPartitionsInProgress
> > >> > > are used in the status check.
> > >> > >
> > >> > > Thanks,
> > >> > >
> > >> > > Jun
> > >> > >
> > >> > >
> > >> > > On Tue, Mar 24, 2015 at 3:46 AM, Andrii Biletskyi <
> > >> > > andrii.biletskyi@stealth.ly> wrote:
> > >> > >
> > >> > > > Jun,
> > >> > > >
> > >> > > > Thanks for such an accurate review!
> > >> > > >
> > >> > > > Most of your remarks I didn't fix in the previous change because
> > >> > > > I thought we'd finalize them today, during the call. Anyway,
> I've
> > >> > > > fixed them now.
> > >> > > > My comments to ensure I covered everything:
> > >> > > >
> > >> > > > 110. Fixed. Removed global error code.
> > >> > > >
> > >> > > > 111.1-3. Removed redundant TopicName, Partitions and
> > >> ReplicationFactor
> > >> > > > 111.4 Yes, now we are in the situation when
> TopicMetadataResponse
> > >> > > > contains everything we need except topic level configs. I'm
> > >> > > > okay with both solutions, probably makes sense to discuss and
> pick
> > >> > > > the preferable one.
> > >> > > >
> > >> > > > 112. Need to think about it a bit more. Currently we may
> > distinguish
> > >> > > > 3 states of specific partition being reassigned:
> > >> > > > a) Completed - absent in /reassign_partitions zk path and
> > >> > DescribeTopic/
> > >> > > > TopicMetadata shows expected assignment
> > >> > > > 2) In progress - present in /reassign_partitions
> > >> > > > 3) Failed - absent in /reassign_partitions but
> > >> > > DescribeTopic/TopicMetadata
> > >> > > > replicas field doesn't correspond expected
> > >> > > > Now, if use only DescribeTopic to check reassignment status -
> how
> > >> can
> > >> > we
> > >> > > > distinguish states "In Progress" and "Failed"?
> > >> > > > Also, there are some concerns that current reassignment status
> > >> provided
> > >> > > > by ReaasignPartitionsCommand (and the same was intended in
> KIP-4)
> > is
> > >> > > > very spurious, non-informative. Probably it makes sense to add
> > >> > additional
> > >> > > > data there (thus separate request may be useful), I'm
> considering
> > >> > > different
> > >> > > > options now - btw, comments here are highly appreciated!
> > >> > > >
> > >> > > > 113-114. Okay, made it compliant with topic commands. About "The
> > >> error
> > >> > > code
> > >> > > > will
> > >> > > > then be per topic" - just a note: currently
> > >> (ReassignPartitionsCommand)
> > >> > > > validation
> > >> > > > is done for the whole reassignment string, so in case of errors
> > for
> > >> > > > specific
> > >> > > > partitions the reassignment is NOT started at all. Both options
> > look
> > >> > fine
> > >> > > > for me,
> > >> > > > it's just the logic will be slightly changed.
> > >> > > >
> > >> > > > 115.1 Removed NotControllerReceivedAdminCommand. But why should
> we
> > >> > > > remove PreferredReplicaLeaderElectionInProgress and
> > >> > > > ReassignPartitionsInProgress?
> > >> > > > On Reassign/Preferred..Request we create admin zk path, we can
> of
> > >> > course
> > >> > > > update the path, but wouldn't it be safer if we simply refuse to
> > >> start
> > >> > > new
> > >> > > > reassignment in case one is in progress?
> > >> > > >
> > >> > > > 115.2 Yes, paragraph Protocol Errors covers only errors that are
> > >> > proposed
> > >> > > > to be added. I didn't want to list all errors that are to be
> added
> > >> > since
> > >> > > > anyway
> > >> > > > the final version will be clear after the development is
> > completed.
> > >> > > >
> > >> > > > 100. Fixed, normalized json string in Create/Alter.
> > >> > > >
> > >> > > > Thanks,
> > >> > > > Andrii Biletskyi
> > >> > > >
> > >> > > > On Tue, Mar 24, 2015 at 12:58 AM, Jun Rao <ju...@confluent.io>
> > wrote:
> > >> > > >
> > >> > > > > Andrii,
> > >> > > > >
> > >> > > > > I looked at the latest wiki for this KIP. I have a few more
> > >> comments.
> > >> > > > >
> > >> > > > > 110. In CreateTopicResponse, AlterTopicResponse,
> > >> DeleteTopicResponse
> > >> > > and
> > >> > > > > DescribeTopicResponse, we probably don't need the global error
> > >> code.
> > >> > > Our
> > >> > > > > current convention is to just use the same global error code
> in
> > >> each
> > >> > > > topic.
> > >> > > > > This simplifies the error checking on the client side.
> > >> > > > >
> > >> > > > > 111. DescribeTopicResponse:
> > >> > > > > 111.1 Our protocol definition doesn't support optional fields.
> > >> So, we
> > >> > > > can't
> > >> > > > > make Leader an optional field.
> > >> > > > > 111.2 Do we really need the field Partitions and
> > >> ReplicationFactor in
> > >> > > > > TopicConfigDetails?
> > >> > > > > The former is basically the size of the TopicPartitionDetails
> > >> array
> > >> > and
> > >> > > > the
> > >> > > > > latter can be found from the size of the Replica array.
> > >> > > > > 111.3 Do we need TopicName in TopicDescription since it's
> > already
> > >> in
> > >> > > > > DescribeTopicResponse?
> > >> > > > > 111.4 Finally, DescribeTopicResponse is very similar to
> > >> > > > > TopicMetadataResponse. So, it probably makes sense just to
> keep
> > >> one
> > >> > of
> > >> > > > them
> > >> > > > > in the future. Should we just use
> DescribeTopicRequest/Response
> > to
> > >> > > > replace
> > >> > > > > TopicMetadataRequest/Response in the producer/consumer client
> in
> > >> the
> > >> > > > future
> > >> > > > > (since DescribeTopicRequest doesn't trigger auto topic
> > creation)?
> > >> If
> > >> > > so,
> > >> > > > we
> > >> > > > > will need to add the broker list in DescribeTopicResponse.
> > >> > > > >
> > >> > > > > 112. Thinking about this a bit more, I don't think we need a
> > >> separate
> > >> > > > > VerifyReassignPartitionRequest/Response.
> > >> > > > > We can just use DescribeTopic to get the assigned replicas and
> > >> check
> > >> > if
> > >> > > > > they match the target replica assignment in the client. The
> > >> > controller
> > >> > > > > propagates the metadata change after the reassignment
> completes
> > >> for
> > >> > > each
> > >> > > > > partition.
> > >> > > > >
> > >> > > > > 113. ReassignPartitionRequest: For consistency, we probably
> want
> > >> to
> > >> > > nest
> > >> > > > > the partition data under topic. So instead of
> > >> > > > >   ReassignPartitionRequest => [Topic PartitionId [ReplicaId]]
> > >> > > > > we can have sth like
> > >> > > > >   ReassignPartitionRequest => [Topic [PartitionId
> [ReplicaId]]]
> > >> > > > > The error code will then be per topic.
> > >> > > > >
> > >> > > > > 114. PreferredReplicaLeaderElectionRequest: Same as the above.
> > >> > Instead
> > >> > > of
> > >> > > > >   PreferredReplicaLeaderElectionRequest => [Topic PartitionId]
> > >> > > > > we can have
> > >> > > > >   PreferredReplicaLeaderElectionRequest => [Topic
> [PartitionId]]
> > >> > > > > Again, the error code will be per topic.
> > >> > > > >
> > >> > > > > 115. ErrorCode:
> > >> > > > > 115.1 NotControllerReceivedAdminRequest,
> > >> > > > > PreferredReplicaLeaderElectionInProgress
> > >> > > > > and ReassignPartitionsInProgress are likely not needed any
> more.
> > >> > > > > 115.2 We probably need to add UnknownTopicOrPartitionCode
> since
> > >> it's
> > >> > a
> > >> > > > > valid error code for some of the requests (e.g. describe,
> alter,
> > >> etc)
> > >> > > > when
> > >> > > > > a non-existing topic is specified.
> > >> > > > >
> > >> > > > >
> > >> > > > > Previous comments.
> > >> > > > > 100. ReplicaAssignment in CreateTopicRequest and
> > AlterTopicRequest
> > >> > are
> > >> > > > > still a json string. We need to flatten them.
> > >> > > > >
> > >> > > > >
> > >> > > > > Thanks,
> > >> > > > >
> > >> > > > > Jun
> > >> > > > >
> > >> > > > >
> > >> > > > > On Sat, Mar 21, 2015 at 2:03 AM, Andrii Biletskyi <
> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > >
> > >> > > > > > Guozhang,
> > >> > > > > >
> > >> > > > > > I'm not sure I understand how can we use those tools in CLI.
> > >> > > > > > First of all, those are *Test*Utils, e.g.
> > >> > > waitUntilMetadataIsPropagated
> > >> > > > > > (which might be very useful for us) requires
> > >> > servers:Seq[KafkaServer]
> > >> > > > > > as an argument:
> > >> > > > > >
> > >> > > > > > def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer],
> > >> topic:
> > >> > > > > String,
> > >> > > > > > partition: Int, timeout: Long = 5000L): Int
> > >> > > > > >
> > >> > > > > > how can we have access it at runtime in one of the brokers?
> > >> > > > > >
> > >> > > > > > Secondly, how can user directly call these tools if
> zookeeper
> > >> might
> > >> > > be
> > >> > > > > > not accessible (which is used in the tools, right?) at all -
> > >> e.g.
> > >> > > > behind
> > >> > > > > > the VPC
> > >> > > > > > in AWS. Furthermore, I think with KIP-4 we are trying to
> > create
> > >> an
> > >> > > > > > abstraction and
> > >> > > > > > a single point of zookeeper interactions, to eliminate
> direct
> > >> calls
> > >> > > to
> > >> > > > ZK
> > >> > > > > > either to get
> > >> > > > > > some cluster information or to change something.
> > >> > > > > >
> > >> > > > > > Thanks,
> > >> > > > > > Andrii Biletskyi
> > >> > > > > >
> > >> > > > > > On Sat, Mar 21, 2015 at 3:37 AM, Guozhang Wang <
> > >> wangguoz@gmail.com
> > >> > >
> > >> > > > > wrote:
> > >> > > > > >
> > >> > > > > > > Andrii,
> > >> > > > > > >
> > >> > > > > > > Actually the checking logic Jun mentioned is already
> > >> implemented
> > >> > as
> > >> > > > > > > TestUtils.waitUntilXXX (LeaderIsElected,
> > MetadataIsPropagated,
> > >> > > > etc...)
> > >> > > > > I
> > >> > > > > > > think we can extend these functions as CLI tools like
> > >> > TopicCommand
> > >> > > so
> > >> > > > > > that
> > >> > > > > > > users re-implementing such endpoint can directly call
> > >> something
> > >> > > like
> > >> > > > > > > java.tools.WaitUntilXXX (of course this requires them to
> > have
> > >> > javac
> > >> > > > > > > installed, which should be a reasonable requirement?)
> > >> > > > > > >
> > >> > > > > > > Guozhang
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
> > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > >
> > >> > > > > > > > Jun,
> > >> > > > > > > >
> > >> > > > > > > > Not that I was saying we need to make requests blocking
> on
> > >> > > server,
> > >> > > > > > > > it was just to emphasize that with async requests a
> client
> > >> > > > > > > implementations
> > >> > > > > > > > may be a little bit more than just issue request - get
> the
> > >> > > > response.
> > >> > > > > > > > Thanks for the explanation, I understand now that we can
> > go
> > >> > with
> > >> > > > > agreed
> > >> > > > > > > > solution though it may not be perfect.
> > >> > > > > > > > I believe this was one of the last controversial
> questions
> > >> from
> > >> > > the
> > >> > > > > > list.
> > >> > > > > > > >
> > >> > > > > > > > Thanks,
> > >> > > > > > > > Andrii Biletskyi
> > >> > > > > > > >
> > >> > > > > > > > On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <
> > jun@confluent.io
> > >> >
> > >> > > > wrote:
> > >> > > > > > > >
> > >> > > > > > > > > Andrii,
> > >> > > > > > > > >
> > >> > > > > > > > > A few points.
> > >> > > > > > > > >
> > >> > > > > > > > > 1. Create/Alter can typically complete quickly. So,
> it's
> > >> > > possible
> > >> > > > > to
> > >> > > > > > > make
> > >> > > > > > > > > the request block until it's completed. However,
> > >> currently,
> > >> > > doing
> > >> > > > > > this
> > >> > > > > > > at
> > >> > > > > > > > > the broker is a bit involved. To make Create block, we
> > >> will
> > >> > > need
> > >> > > > to
> > >> > > > > > add
> > >> > > > > > > > > some callbacks in KafkaController. This is possible.
> > >> However,
> > >> > > the
> > >> > > > > > > > > controller logic currently is pretty completed. It
> would
> > >> > > probably
> > >> > > > > be
> > >> > > > > > > > better
> > >> > > > > > > > > if we clean it up first before adding more complexity
> to
> > >> it.
> > >> > > > Alter
> > >> > > > > is
> > >> > > > > > > > even
> > >> > > > > > > > > trickier. Adding partition is currently handled
> through
> > >> > > > > > > KafkaController.
> > >> > > > > > > > So
> > >> > > > > > > > > it can be dealt with in a similar way. However, Alter
> > >> config
> > >> > is
> > >> > > > > done
> > >> > > > > > > > > completely differently. It doesn't go through the
> > >> controller.
> > >> > > > > > Instead,
> > >> > > > > > > > each
> > >> > > > > > > > > broker listens to ZooKeeper directly. So, it's not
> clear
> > >> if
> > >> > > there
> > >> > > > > is
> > >> > > > > > an
> > >> > > > > > > > > easy way on the broker to figure out whether a config
> is
> > >> > > applied
> > >> > > > on
> > >> > > > > > > every
> > >> > > > > > > > > broker.
> > >> > > > > > > > >
> > >> > > > > > > > > 2. Delete can potentially take long if a replica to be
> > >> > deleted
> > >> > > is
> > >> > > > > > > > offline.
> > >> > > > > > > > > PreferredLeader/PartitionReassign can also take long.
> > So,
> > >> we
> > >> > > > can't
> > >> > > > > > > really
> > >> > > > > > > > > make those requests block on the broker.
> > >> > > > > > > > >
> > >> > > > > > > > > As you can see, at this moment it's not easy to make
> all
> > >> > admin
> > >> > > > > > requests
> > >> > > > > > > > > block on the broker. So, if we want the blocking
> feature
> > >> in
> > >> > the
> > >> > > > > admin
> > >> > > > > > > > > utility in the short term, doing the completion check
> at
> > >> the
> > >> > > > admin
> > >> > > > > > > client
> > >> > > > > > > > > is probably an easier route, even though it may not be
> > >> ideal.
> > >> > > > > > > > >
> > >> > > > > > > > > Thanks,
> > >> > > > > > > > >
> > >> > > > > > > > > Jun
> > >> > > > > > > > >
> > >> > > > > > > > > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> > >> > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > > > >
> > >> > > > > > > > > > Jun,
> > >> > > > > > > > > >
> > >> > > > > > > > > > I see your point. But wouldn't that lead to a "fat"
> > >> client
> > >> > > > > > > > > implementations?
> > >> > > > > > > > > > Suppose someone would like to implement client for
> > Admin
> > >> > Wire
> > >> > > > > > > protocol.
> > >> > > > > > > > > > Not only people will have to code quite complicated
> > >> logic
> > >> > > like
> > >> > > > > > "send
> > >> > > > > > > > > > describe
> > >> > > > > > > > > > request to each broker" (again state machin?) but it
> > >> will
> > >> > > also
> > >> > > > > mean
> > >> > > > > > > > > people
> > >> > > > > > > > > > must understand internal kafka logic related to
> topic
> > >> > storage
> > >> > > > and
> > >> > > > > > how
> > >> > > > > > > > > > information is propageted from the controller to
> > >> brokers.
> > >> > > > > > > > > > I see this like a dilemma between having a concise
> > Wire
> > >> > > > Protocol
> > >> > > > > > and
> > >> > > > > > > > > > self-sufficient API to make client implementations
> > >> simple.
> > >> > > > > > > > > > I don't have a win-win solution though.
> > >> > > > > > > > > >
> > >> > > > > > > > > > Thanks,
> > >> > > > > > > > > > Andrii Biletskyi
> > >> > > > > > > > > >
> > >> > > > > > > > > >
> > >> > > > > > > > > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <
> > >> > jun@confluent.io>
> > >> > > > > > wrote:
> > >> > > > > > > > > >
> > >> > > > > > > > > > > For 1), 2) and 3), blocking would probably mean
> that
> > >> the
> > >> > > new
> > >> > > > > > > metadata
> > >> > > > > > > > > is
> > >> > > > > > > > > > > propagated to every broker. To achieve that, the
> > >> client
> > >> > can
> > >> > > > > keep
> > >> > > > > > > > > issuing
> > >> > > > > > > > > > > the describe topic request to every broker until
> it
> > >> sees
> > >> > > the
> > >> > > > > new
> > >> > > > > > > > > metadata
> > >> > > > > > > > > > > in the response.
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > Thanks,
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > Jun
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii
> Biletskyi <
> > >> > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > > Hm, actually the ticket you linked, Guozhang,
> > >> brings as
> > >> > > > back
> > >> > > > > > > > > > > > to the problem what should be considered a
> > >> > post-condition
> > >> > > > for
> > >> > > > > > > > > > > > each of the admin commands.
> > >> > > > > > > > > > > > In my understanding:
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > 1) CreateTopic - broker created
> > >> /brokers/topics/<topic>
> > >> > > > > > > > > > > > (Not the controller picked up changes from zk
> and
> > >> > > > broadcasted
> > >> > > > > > > > > > > > LeaderAndIsr and UpdateMetadata)
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > 2) AlterTopic - same as 1) - broker changed
> > >> assignment
> > >> > > data
> > >> > > > > > > > > > > > in zookeeper or created admin path for topic
> > config
> > >> > > change
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > 3) DeleteTopic - admin path /admin/delete_topics
> > is
> > >> > > created
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > 4) ReassignPartitions and PreferredReplica -
> > >> > > corresponding
> > >> > > > > > admin
> > >> > > > > > > > > > > > path is created
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Now what can be considered a completed operation
> > >> from
> > >> > the
> > >> > > > > > > client's
> > >> > > > > > > > > > > > perspective?
> > >> > > > > > > > > > > > 1) Topic is created once corresponding data is
> in
> > zk
> > >> > > > > > > > > > > > (I remember there were some thoughts that it'd
> be
> > >> good
> > >> > to
> > >> > > > > > > consider
> > >> > > > > > > > > > > > topic created once all replicas receive
> > information
> > >> > about
> > >> > > > it
> > >> > > > > > and
> > >> > > > > > > > thus
> > >> > > > > > > > > > > > clients can produce/consume from it, but as was
> > >> > discussed
> > >> > > > > this
> > >> > > > > > > > seems
> > >> > > > > > > > > > > > to be a hard thing to do)
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > 2) Probably same as 1), so right after
> AlterTopic
> > is
> > >> > > issued
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > 3) The topic has been removed from
> /brokers/topics
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > 4) ReassignPartitions and PrefferedReplica were
> > >> > discussed
> > >> > > > > > > earlier -
> > >> > > > > > > > > > > > in short the former is completed once partition
> > >> state
> > >> > > info
> > >> > > > in
> > >> > > > > > zk
> > >> > > > > > > > > > matches
> > >> > > > > > > > > > > > reassignment request and admin path is empty,
> the
> > >> > latter
> > >> > > -
> > >> > > > > once
> > >> > > > > > > > data
> > >> > > > > > > > > > > > in zk shows that head of assignned replicas of
> the
> > >> > > > partition
> > >> > > > > > and
> > >> > > > > > > > > leader
> > >> > > > > > > > > > > > is the same replica
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Thoughts?
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > Andrii Biletskyi
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <
> > >> > > > > > > wangguoz@gmail.com
> > >> > > > > > > > >
> > >> > > > > > > > > > > wrote:
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > > I think while loop is fine for supporting
> > >> blocking,
> > >> > > just
> > >> > > > > that
> > >> > > > > > > we
> > >> > > > > > > > > need
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > add back off to avoid bombarding brokers with
> > >> > > > DescribeTopic
> > >> > > > > > > > > requests.
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > Also I have linked KAFKA-1125
> > >> > > > > > > > > > > > > <
> > https://issues.apache.org/jira/browse/KAFKA-1125
> > >> >
> > >> > to
> > >> > > > your
> > >> > > > > > > > > proposal,
> > >> > > > > > > > > > > and
> > >> > > > > > > > > > > > > when KAFKA-1694 is done this ticket can also
> be
> > >> > closed.
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > Guozhang
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii
> > Biletskyi
> > >> <
> > >> > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > Great.
> > >> > > > > > > > > > > > > > I want to elaborate this a bit more, to see
> we
> > >> are
> > >> > on
> > >> > > > the
> > >> > > > > > > same
> > >> > > > > > > > > page
> > >> > > > > > > > > > > > > > concerning the client code.
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > So with all topic commands being async a
> > client
> > >> > > > > > (AdminClient
> > >> > > > > > > in
> > >> > > > > > > > > our
> > >> > > > > > > > > > > > > > case or any other other client people would
> > >> like to
> > >> > > > > > > implement)
> > >> > > > > > > > to
> > >> > > > > > > > > > > > support
> > >> > > > > > > > > > > > > > a blocking operation (which seems to be a
> > >> natural
> > >> > > > > use-case
> > >> > > > > > > e.g.
> > >> > > > > > > > > for
> > >> > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > creation): would have to do:
> > >> > > > > > > > > > > > > > 1. issue CreateTopicRequest
> > >> > > > > > > > > > > > > > 2. if successful, in a "while" loop send
> > >> > > > > > DescribeTopicRequest
> > >> > > > > > > > and
> > >> > > > > > > > > > > > > > break the loop once all topics are returned
> in
> > >> > > response
> > >> > > > > (or
> > >> > > > > > > > upon
> > >> > > > > > > > > > > > > timeout).
> > >> > > > > > > > > > > > > > 3. if unsuccessful throw exception
> > >> > > > > > > > > > > > > > Would it be okay?
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > > > Andrii Biletskyi
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <
> > >> > > > > jun@confluent.io
> > >> > > > > > >
> > >> > > > > > > > > wrote:
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > Andrii,
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > I think you are right. It seems that only
> > >> > > > > > > ReassignPartitions
> > >> > > > > > > > > > needs
> > >> > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > separate verification request.
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > Jun
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii
> > >> > Biletskyi <
> > >> > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > Guys,
> > >> > > > > > > > > > > > > > > > I like this idea too. Let's stick with
> > that.
> > >> > I'll
> > >> > > > > > update
> > >> > > > > > > > KIP
> > >> > > > > > > > > > > > > > accordingly.
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > I was also thinking we can avoid adding
> > >> > dedicated
> > >> > > > > > status
> > >> > > > > > > > > check
> > >> > > > > > > > > > > > > > > > requests for topic commands. - We have
> > >> > everything
> > >> > > > in
> > >> > > > > > > > > > > DescribeTopic
> > >> > > > > > > > > > > > > > > > for that! E.g.:
> > >> > > > > > > > > > > > > > > > User issued CreateTopic - to check the
> > >> status
> > >> > > > client
> > >> > > > > > > sends
> > >> > > > > > > > > > > > > > DescribeTopic
> > >> > > > > > > > > > > > > > > > and checks whether is something returned
> > for
> > >> > that
> > >> > > > > > topic.
> > >> > > > > > > > The
> > >> > > > > > > > > > same
> > >> > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > alteration, deletion.
> > >> > > > > > > > > > > > > > > > Btw, PreferredReplica status can be also
> > >> > checked
> > >> > > > with
> > >> > > > > > > > > > > > > > > DescribeTopicRequest
> > >> > > > > > > > > > > > > > > > (head of assigned replicas list ==
> > leader).
> > >> > > > > > > > > > > > > > > > For ReassignPartitions as discussed
> we'll
> > >> need
> > >> > to
> > >> > > > > have
> > >> > > > > > a
> > >> > > > > > > > > > separate
> > >> > > > > > > > > > > > > > > Verify...
> > >> > > > > > > > > > > > > > > > request.
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > > > > > Andrii Biletskyi
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM,
> Guozhang
> > >> Wang
> > >> > <
> > >> > > > > > > > > > > wangguoz@gmail.com
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > +1 on broker writing to ZK for async
> > >> > handling.
> > >> > > I
> > >> > > > > was
> > >> > > > > > > > > thinking
> > >> > > > > > > > > > > > that
> > >> > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > end state the admin requests would be
> > >> > > eventually
> > >> > > > > sent
> > >> > > > > > > to
> > >> > > > > > > > > > > > controller
> > >> > > > > > > > > > > > > > > > either
> > >> > > > > > > > > > > > > > > > > through re-routing or clients
> > discovering
> > >> > them,
> > >> > > > > > instead
> > >> > > > > > > > of
> > >> > > > > > > > > > > > letting
> > >> > > > > > > > > > > > > > > > > controller listen on ZK admin path.
> But
> > >> > > thinking
> > >> > > > > > about
> > >> > > > > > > > it a
> > >> > > > > > > > > > > > second
> > >> > > > > > > > > > > > > > > time,
> > >> > > > > > > > > > > > > > > > I
> > >> > > > > > > > > > > > > > > > > think it is actually simpler to let
> > >> > controller
> > >> > > > > manage
> > >> > > > > > > > > > > > > > > > > incoming queued-up admin requests
> > through
> > >> ZK.
> > >> > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > Guozhang
> > >> > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel
> > >> Koshy <
> > >> > > > > > > > > > > jjkoshy.w@gmail.com
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > +1 as well. I think it helps to keep
> > the
> > >> > > > > rerouting
> > >> > > > > > > > > approach
> > >> > > > > > > > > > > > > > > orthogonal
> > >> > > > > > > > > > > > > > > > > > to this KIP.
> > >> > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM
> > >> -0700,
> > >> > Jay
> > >> > > > > Kreps
> > >> > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > I'm +1 on Jun's suggestion as long
> > as
> > >> it
> > >> > > can
> > >> > > > > work
> > >> > > > > > > for
> > >> > > > > > > > > all
> > >> > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > requests.
> > >> > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM,
> Jun
> > >> Rao
> > >> > <
> > >> > > > > > > > > > jun@confluent.io
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > Andrii,
> > >> > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > I think we agreed on the
> > following.
> > >> > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > (a) Admin requests can be sent
> to
> > >> and
> > >> > > > handled
> > >> > > > > > by
> > >> > > > > > > > any
> > >> > > > > > > > > > > > broker.
> > >> > > > > > > > > > > > > > > > > > > > (b) Admin requests are processed
> > >> > > > > > asynchronously,
> > >> > > > > > > at
> > >> > > > > > > > > > least
> > >> > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > now.
> > >> > > > > > > > > > > > > > > > > > That is,
> > >> > > > > > > > > > > > > > > > > > > > when the client gets a response,
> > it
> > >> > just
> > >> > > > > means
> > >> > > > > > > that
> > >> > > > > > > > > the
> > >> > > > > > > > > > > > > request
> > >> > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > initiated, but not necessarily
> > >> > completed.
> > >> > > > > Then,
> > >> > > > > > > > it's
> > >> > > > > > > > > up
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > client
> > >> > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > issue another request to check
> the
> > >> > status
> > >> > > > for
> > >> > > > > > > > > > completion.
> > >> > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > To support (a), we were thinking
> > of
> > >> > doing
> > >> > > > > > request
> > >> > > > > > > > > > > > forwarding
> > >> > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > controller (utilizing
> KAFKA-1912).
> > >> I am
> > >> > > > > making
> > >> > > > > > an
> > >> > > > > > > > > > > > alternative
> > >> > > > > > > > > > > > > > > > > proposal.
> > >> > > > > > > > > > > > > > > > > > > > Basically, the broker can just
> > >> write to
> > >> > > > > > ZooKeeper
> > >> > > > > > > > to
> > >> > > > > > > > > > > inform
> > >> > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > controller
> > >> > > > > > > > > > > > > > > > > > > > about the request. For example,
> to
> > >> > handle
> > >> > > > > > > > > > > > > > partitionReassignment,
> > >> > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > broker
> > >> > > > > > > > > > > > > > > > > > > > will just write the requested
> > >> > partitions
> > >> > > to
> > >> > > > > > > > > > > > > > > > > /admin/reassign_partitions
> > >> > > > > > > > > > > > > > > > > > > > (like what AdminUtils currently
> > >> does)
> > >> > and
> > >> > > > > then
> > >> > > > > > > > send a
> > >> > > > > > > > > > > > > response
> > >> > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > client. This shouldn't take long
> > and
> > >> > the
> > >> > > > > > > > > implementation
> > >> > > > > > > > > > > > will
> > >> > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > simpler
> > >> > > > > > > > > > > > > > > > > > > > than forwarding the requests to
> > the
> > >> > > > > controller
> > >> > > > > > > > > through
> > >> > > > > > > > > > > RPC.
> > >> > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > Jun
> > >> > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM,
> > >> Andrii
> > >> > > > > > > Biletskyi <
> > >> > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > wrote:
> > >> > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > Jun,
> > >> > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > I might be wrong but didn't we
> > >> agree
> > >> > we
> > >> > > > > will
> > >> > > > > > > let
> > >> > > > > > > > > any
> > >> > > > > > > > > > > > broker
> > >> > > > > > > > > > > > > > > from
> > >> > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > cluster handle *long-running*
> > >> admin
> > >> > > > > requests
> > >> > > > > > > (at
> > >> > > > > > > > > this
> > >> > > > > > > > > > > > time
> > >> > > > > > > > > > > > > > > > > > > > preferredReplica
> > >> > > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > reassignPartitions), via zk
> > admin
> > >> > path.
> > >> > > > > Thus
> > >> > > > > > > > > > > CreateTopics
> > >> > > > > > > > > > > > > etc
> > >> > > > > > > > > > > > > > > > > should
> > >> > > > > > > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > > > > > sent
> > >> > > > > > > > > > > > > > > > > > > > > only to the controller.
> > >> > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > >> > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55
> > PM,
> > >> Jun
> > >> > > > Rao <
> > >> > > > > > > > > > > > > jun@confluent.io>
> > >> > > > > > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > Joel, Andril,
> > >> > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > I think we agreed that those
> > >> admin
> > >> > > > > requests
> > >> > > > > > > can
> > >> > > > > > > > > be
> > >> > > > > > > > > > > > issued
> > >> > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > any
> > >> > > > > > > > > > > > > > > > > > > > broker.
> > >> > > > > > > > > > > > > > > > > > > > > > Because of that, there
> doesn't
> > >> seem
> > >> > > to
> > >> > > > > be a
> > >> > > > > > > > > strong
> > >> > > > > > > > > > > need
> > >> > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > know
> > >> > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > controller. So, perhaps we
> can
> > >> > > proceed
> > >> > > > by
> > >> > > > > > not
> > >> > > > > > > > > > making
> > >> > > > > > > > > > > > any
> > >> > > > > > > > > > > > > > > change
> > >> > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > format of TMR right now.
> When
> > we
> > >> > > start
> > >> > > > > > using
> > >> > > > > > > > > create
> > >> > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > request
> > >> > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > producer, we will need a new
> > >> > version
> > >> > > of
> > >> > > > > TMR
> > >> > > > > > > > that
> > >> > > > > > > > > > > > doesn't
> > >> > > > > > > > > > > > > > > > trigger
> > >> > > > > > > > > > > > > > > > > > auto
> > >> > > > > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > creation. But that can be
> done
> > >> > later.
> > >> > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > As a first cut
> > implementation, I
> > >> > > think
> > >> > > > > the
> > >> > > > > > > > broker
> > >> > > > > > > > > > can
> > >> > > > > > > > > > > > > just
> > >> > > > > > > > > > > > > > > > write
> > >> > > > > > > > > > > > > > > > > > to ZK
> > >> > > > > > > > > > > > > > > > > > > > > > directly for
> > >> > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > >
> > >> > > > > > >
> > >> > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > >> > > > > > > > > > > > > > > > > > > > > > requests, instead of
> > forwarding
> > >> > them
> > >> > > to
> > >> > > > > the
> > >> > > > > > > > > > > controller.
> > >> > > > > > > > > > > > > > This
> > >> > > > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > > > > > simplify
> > >> > > > > > > > > > > > > > > > > > > > > > the implementation on the
> > broker
> > >> > > side.
> > >> > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > Jun
> > >> > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at
> 11:58
> > >> AM,
> > >> > > Joel
> > >> > > > > > Koshy
> > >> > > > > > > <
> > >> > > > > > > > > > > > > > > > > jjkoshy.w@gmail.com>
> > >> > > > > > > > > > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > (Thanks Andrii for the
> > >> summary)
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > For (1) yes we will circle
> > >> back
> > >> > on
> > >> > > > that
> > >> > > > > > > > shortly
> > >> > > > > > > > > > > after
> > >> > > > > > > > > > > > > > > syncing
> > >> > > > > > > > > > > > > > > > > up
> > >> > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > person. I think it is
> close
> > to
> > >> > > > getting
> > >> > > > > > > > > committed
> > >> > > > > > > > > > > > > although
> > >> > > > > > > > > > > > > > > > > > development
> > >> > > > > > > > > > > > > > > > > > > > > > > for KAFKA-1927 can
> probably
> > >> begin
> > >> > > > > without
> > >> > > > > > > it.
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > There is one more item we
> > >> covered
> > >> > > at
> > >> > > > > the
> > >> > > > > > > > > hangout.
> > >> > > > > > > > > > > > i.e.,
> > >> > > > > > > > > > > > > > > > whether
> > >> > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > want to add the
> coordinator
> > to
> > >> > the
> > >> > > > > topic
> > >> > > > > > > > > metadata
> > >> > > > > > > > > > > > > > response
> > >> > > > > > > > > > > > > > > or
> > >> > > > > > > > > > > > > > > > > > provide
> > >> > > > > > > > > > > > > > > > > > > > > > > a clearer
> > >> ClusterMetadataRequest.
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > There are two reasons I
> > think
> > >> we
> > >> > > > should
> > >> > > > > > try
> > >> > > > > > > > and
> > >> > > > > > > > > > > avoid
> > >> > > > > > > > > > > > > > > adding
> > >> > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > field:
> > >> > > > > > > > > > > > > > > > > > > > > > > - It is irrelevant to
> topic
> > >> > > metadata
> > >> > > > > > > > > > > > > > > > > > > > > > > - If we finally do request
> > >> > > rerouting
> > >> > > > in
> > >> > > > > > > Kafka
> > >> > > > > > > > > > then
> > >> > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > field
> > >> > > > > > > > > > > > > > > > > > would
> > >> > > > > > > > > > > > > > > > > > > > add
> > >> > > > > > > > > > > > > > > > > > > > > > >   little to no value. (It
> > >> still
> > >> > > helps
> > >> > > > > to
> > >> > > > > > > > have a
> > >> > > > > > > > > > > > > separate
> > >> > > > > > > > > > > > > > > > > > > > > > >   ClusterMetadataRequest
> to
> > >> query
> > >> > > for
> > >> > > > > > > > > > cluster-wide
> > >> > > > > > > > > > > > > > > > information
> > >> > > > > > > > > > > > > > > > > > such
> > >> > > > > > > > > > > > > > > > > > > > as
> > >> > > > > > > > > > > > > > > > > > > > > > >   'which broker is the
> > >> > controller?'
> > >> > > > as
> > >> > > > > > Joe
> > >> > > > > > > > > > > > mentioned.)
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > I think it would be
> cleaner
> > to
> > >> > have
> > >> > > > an
> > >> > > > > > > > explicit
> > >> > > > > > > > > > > > > > > > > > > > ClusterMetadataRequest
> > >> > > > > > > > > > > > > > > > > > > > > > > that you can send to any
> > >> broker
> > >> > in
> > >> > > > > order
> > >> > > > > > to
> > >> > > > > > > > > > obtain
> > >> > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > controller
> > >> > > > > > > > > > > > > > > > > > > > (and
> > >> > > > > > > > > > > > > > > > > > > > > > > in the future possibly
> other
> > >> > > > > cluster-wide
> > >> > > > > > > > > > > > > information). I
> > >> > > > > > > > > > > > > > > > think
> > >> > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > main argument against
> doing
> > >> this
> > >> > > and
> > >> > > > > > > instead
> > >> > > > > > > > > > adding
> > >> > > > > > > > > > > > it
> > >> > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > metadata response was
> > >> > convenience -
> > >> > > > > i.e.,
> > >> > > > > > > you
> > >> > > > > > > > > > don't
> > >> > > > > > > > > > > > > have
> > >> > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > discover
> > >> > > > > > > > > > > > > > > > > > > > > > > the controller in advance.
> > >> > > However, I
> > >> > > > > > don't
> > >> > > > > > > > see
> > >> > > > > > > > > > > much
> > >> > > > > > > > > > > > > > actual
> > >> > > > > > > > > > > > > > > > > > > > > > > benefit/convenience in
> this
> > >> and
> > >> > in
> > >> > > > fact
> > >> > > > > > > think
> > >> > > > > > > > > it
> > >> > > > > > > > > > > is a
> > >> > > > > > > > > > > > > > > > > non-issue.
> > >> > > > > > > > > > > > > > > > > > Let
> > >> > > > > > > > > > > > > > > > > > > > > > > me know if I'm overlooking
> > >> > > something
> > >> > > > > > here.
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > As an example, say we need
> > to
> > >> > > > initiate
> > >> > > > > > > > > partition
> > >> > > > > > > > > > > > > > > reassignment
> > >> > > > > > > > > > > > > > > > > by
> > >> > > > > > > > > > > > > > > > > > > > > > > issuing the new
> > >> > > > > ReassignPartitionsRequest
> > >> > > > > > > to
> > >> > > > > > > > > the
> > >> > > > > > > > > > > > > > controller
> > >> > > > > > > > > > > > > > > > > > (assume
> > >> > > > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > already have the desired
> > >> manual
> > >> > > > > partition
> > >> > > > > > > > > > > > assignment).
> > >> > > > > > > > > > > > > > If
> > >> > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > are to
> > >> > > > > > > > > > > > > > > > > > > > > > > augment topic metadata
> > >> response
> > >> > > then
> > >> > > > > the
> > >> > > > > > > flow
> > >> > > > > > > > > be
> > >> > > > > > > > > > > > > > something
> > >> > > > > > > > > > > > > > > > like
> > >> > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > :
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > - Issue topic metadata
> > >> request to
> > >> > > any
> > >> > > > > > > broker
> > >> > > > > > > > > (and
> > >> > > > > > > > > > > > > > discover
> > >> > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > >   controller
> > >> > > > > > > > > > > > > > > > > > > > > > > - Connect to controller if
> > >> > required
> > >> > > > > > (i.e.,
> > >> > > > > > > if
> > >> > > > > > > > > the
> > >> > > > > > > > > > > > > broker
> > >> > > > > > > > > > > > > > > > above
> > >> > > > > > > > > > > > > > > > > !=
> > >> > > > > > > > > > > > > > > > > > > > > > >   controller)
> > >> > > > > > > > > > > > > > > > > > > > > > > - Issue the partition
> > >> > reassignment
> > >> > > > > > request
> > >> > > > > > > to
> > >> > > > > > > > > the
> > >> > > > > > > > > > > > > > > controller.
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > With an explicit cluster
> > >> metadata
> > >> > > > > request
> > >> > > > > > > it
> > >> > > > > > > > > > would
> > >> > > > > > > > > > > > be:
> > >> > > > > > > > > > > > > > > > > > > > > > > - Issue cluster metadata
> > >> request
> > >> > to
> > >> > > > any
> > >> > > > > > > > broker
> > >> > > > > > > > > > > > > > > > > > > > > > > - Connect to controller if
> > >> > required
> > >> > > > > > (i.e.,
> > >> > > > > > > if
> > >> > > > > > > > > the
> > >> > > > > > > > > > > > > broker
> > >> > > > > > > > > > > > > > > > above
> > >> > > > > > > > > > > > > > > > > !=
> > >> > > > > > > > > > > > > > > > > > > > > > >   controller)
> > >> > > > > > > > > > > > > > > > > > > > > > > - Issue the partition
> > >> > reassignment
> > >> > > > > > request
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > So it seems to add little
> > >> > practical
> > >> > > > > value
> > >> > > > > > > and
> > >> > > > > > > > > > > bloats
> > >> > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > metadata
> > >> > > > > > > > > > > > > > > > > > > > > > > response with an
> irrelevant
> > >> > detail.
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > The other angle to this is
> > the
> > >> > > > > following
> > >> > > > > > -
> > >> > > > > > > is
> > >> > > > > > > > > it
> > >> > > > > > > > > > a
> > >> > > > > > > > > > > > > matter
> > >> > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > naming?
> > >> > > > > > > > > > > > > > > > > > > > > > > Should we just rename
> topic
> > >> > > metadata
> > >> > > > > > > > > > > request/response
> > >> > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > just
> > >> > > > > > > > > > > > > > > > > > > > > > > MetadataRequest/Response
> and
> > >> add
> > >> > > > > cluster
> > >> > > > > > > > > metadata
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > it?
> > >> > > > > > > > > > > > > > By
> > >> > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > same
> > >> > > > > > > > > > > > > > > > > > > > > > > token should we also allow
> > >> > querying
> > >> > > > for
> > >> > > > > > the
> > >> > > > > > > > > > > consumer
> > >> > > > > > > > > > > > > > > > > coordinator
> > >> > > > > > > > > > > > > > > > > > (and
> > >> > > > > > > > > > > > > > > > > > > > > > > in future transaction
> > >> > coordinator)
> > >> > > as
> > >> > > > > > well?
> > >> > > > > > > > > This
> > >> > > > > > > > > > > > leads
> > >> > > > > > > > > > > > > > to a
> > >> > > > > > > > > > > > > > > > > > bloated
> > >> > > > > > > > > > > > > > > > > > > > > > > request which isn't very
> > >> > appealing
> > >> > > > and
> > >> > > > > > > > > altogether
> > >> > > > > > > > > > > > > > > confusing.
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > Joel
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at
> > >> > 09:34:12AM
> > >> > > > > -0700,
> > >> > > > > > > Jun
> > >> > > > > > > > > Rao
> > >> > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > Andri,
> > >> > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > Thanks for the summary.
> > >> > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > 1. I just realized that
> in
> > >> > order
> > >> > > to
> > >> > > > > > start
> > >> > > > > > > > > > working
> > >> > > > > > > > > > > > on
> > >> > > > > > > > > > > > > > > > > > KAFKA-1927, we
> > >> > > > > > > > > > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > > > > > > > > need to merge the
> changes
> > to
> > >> > > > > > > > > > OffsetCommitRequest
> > >> > > > > > > > > > > > > (from
> > >> > > > > > > > > > > > > > > > 0.8.2)
> > >> > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > trunk.
> > >> > > > > > > > > > > > > > > > > > > > > > > > This is planned to be
> done
> > >> as
> > >> > > part
> > >> > > > of
> > >> > > > > > > > > > KAFKA-1634.
> > >> > > > > > > > > > > > So,
> > >> > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > > need
> > >> > > > > > > > > > > > > > > > > > > > > > > Guozhang
> > >> > > > > > > > > > > > > > > > > > > > > > > > and Joel's help to wrap
> > this
> > >> > up.
> > >> > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > 2. Thinking about this a
> > bit
> > >> > > more,
> > >> > > > if
> > >> > > > > > the
> > >> > > > > > > > > > > semantic
> > >> > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > those
> > >> > > > > > > > > > > > > > > > > > "write"
> > >> > > > > > > > > > > > > > > > > > > > > > > > requests is async (i.e.,
> > >> after
> > >> > > the
> > >> > > > > > client
> > >> > > > > > > > > gets
> > >> > > > > > > > > > a
> > >> > > > > > > > > > > > > > > response,
> > >> > > > > > > > > > > > > > > > it
> > >> > > > > > > > > > > > > > > > > > just
> > >> > > > > > > > > > > > > > > > > > > > > > means
> > >> > > > > > > > > > > > > > > > > > > > > > > > that the operation is
> > >> > initiated,
> > >> > > > but
> > >> > > > > > not
> > >> > > > > > > > > > > > necessarily
> > >> > > > > > > > > > > > > > > > > > completed), we
> > >> > > > > > > > > > > > > > > > > > > > > > don't
> > >> > > > > > > > > > > > > > > > > > > > > > > > really need to forward
> the
> > >> > > requests
> > >> > > > > to
> > >> > > > > > > the
> > >> > > > > > > > > > > > > controller.
> > >> > > > > > > > > > > > > > > > > > Instead, the
> > >> > > > > > > > > > > > > > > > > > > > > > > > receiving broker can
> just
> > >> write
> > >> > > the
> > >> > > > > > > > operation
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > ZK
> > >> > > > > > > > > > > > > as
> > >> > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > admin
> > >> > > > > > > > > > > > > > > > > > > > > > command
> > >> > > > > > > > > > > > > > > > > > > > > > > > line tool previously
> does.
> > >> This
> > >> > > > will
> > >> > > > > > > > simplify
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > implementation.
> > >> > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > 8. There is another
> > >> > > implementation
> > >> > > > > > detail
> > >> > > > > > > > for
> > >> > > > > > > > > > > > > describe
> > >> > > > > > > > > > > > > > > > topic.
> > >> > > > > > > > > > > > > > > > > > > > > Ideally,
> > >> > > > > > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > want to read the topic
> > >> config
> > >> > > from
> > >> > > > > the
> > >> > > > > > > > broker
> > >> > > > > > > > > > > > cache,
> > >> > > > > > > > > > > > > > > > instead
> > >> > > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > > > > > > ZooKeeper.
> > >> > > > > > > > > > > > > > > > > > > > > > > > Currently, every broker
> > >> reads
> > >> > the
> > >> > > > > > > > topic-level
> > >> > > > > > > > > > > > config
> > >> > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > all
> > >> > > > > > > > > > > > > > > > > > > > topics.
> > >> > > > > > > > > > > > > > > > > > > > > > > > However, it ignores
> those
> > >> for
> > >> > > > topics
> > >> > > > > > not
> > >> > > > > > > > > hosted
> > >> > > > > > > > > > > on
> > >> > > > > > > > > > > > > > > itself.
> > >> > > > > > > > > > > > > > > > > So,
> > >> > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > may
> > >> > > > > > > > > > > > > > > > > > > > > > > need
> > >> > > > > > > > > > > > > > > > > > > > > > > > to change
> > >> TopicConfigManager a
> > >> > > bit
> > >> > > > so
> > >> > > > > > > that
> > >> > > > > > > > it
> > >> > > > > > > > > > > > caches
> > >> > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > configs
> > >> > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > all
> > >> > > > > > > > > > > > > > > > > > > > > > > > topics.
> > >> > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > Jun
> > >> > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at
> > 1:13
> > >> > PM,
> > >> > > > > Andrii
> > >> > > > > > > > > > > Biletskyi <
> > >> > > > > > > > > > > > > > > > > > > > > > > >
> > andrii.biletskyi@stealth.ly
> > >> >
> > >> > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > Guys,
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > Thanks for a great
> > >> > discussion!
> > >> > > > > > > > > > > > > > > > > > > > > > > > > Here are the actions
> > >> points:
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > 1. Q: Get rid of all
> > scala
> > >> > > > requests
> > >> > > > > > > > > objects,
> > >> > > > > > > > > > > use
> > >> > > > > > > > > > > > > java
> > >> > > > > > > > > > > > > > > > > > protocol
> > >> > > > > > > > > > > > > > > > > > > > > > > definitions.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >     A: Gwen kindly
> took
> > >> that
> > >> > > > > > > > (KAFKA-1927).
> > >> > > > > > > > > > It's
> > >> > > > > > > > > > > > > > > important
> > >> > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > speed
> > >> > > > > > > > > > > > > > > > > > > > > up
> > >> > > > > > > > > > > > > > > > > > > > > > > > > review procedure
> > >> > > > > > > > > > > > > > > > > > > > > > > > >          there since
> > this
> > >> > > ticket
> > >> > > > > > blocks
> > >> > > > > > > > > other
> > >> > > > > > > > > > > > > > important
> > >> > > > > > > > > > > > > > > > > > changes.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > 2. Q: Generic
> re-reroute
> > >> > > facility
> > >> > > > > vs
> > >> > > > > > > > client
> > >> > > > > > > > > > > > > > maintaining
> > >> > > > > > > > > > > > > > > > > > cluster
> > >> > > > > > > > > > > > > > > > > > > > > > state.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >     A: Jay has added
> > >> pseudo
> > >> > > code
> > >> > > > to
> > >> > > > > > > > > > KAFKA-1912
> > >> > > > > > > > > > > -
> > >> > > > > > > > > > > > > need
> > >> > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > consider
> > >> > > > > > > > > > > > > > > > > > > > > > > whether
> > >> > > > > > > > > > > > > > > > > > > > > > > > > this will be
> > >> > > > > > > > > > > > > > > > > > > > > > > > >         easy to
> > implement
> > >> as
> > >> > a
> > >> > > > > > > > server-side
> > >> > > > > > > > > > > > feature
> > >> > > > > > > > > > > > > > > > > (comments
> > >> > > > > > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > > > > > > welcomed!).
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > 3. Q: Controller field
> > in
> > >> > wire
> > >> > > > > > > protocol.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >     A: This might be
> > >> useful
> > >> > for
> > >> > > > > > > clients,
> > >> > > > > > > > > add
> > >> > > > > > > > > > > this
> > >> > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > TopicMetadataResponse
> > >> > > > > > > > > > > > > > > > > > > > > > > > > (already in KIP).
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic
> > >> > creation
> > >> > > > > from
> > >> > > > > > > TMR.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >     A: I will add
> > >> proposed by
> > >> > > Jun
> > >> > > > > > > > solution
> > >> > > > > > > > > > > (using
> > >> > > > > > > > > > > > > > > > clientId
> > >> > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > that)
> > >> > > > > > > > > > > > > > > > > > > > > > > to the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > KIP.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > 5. Q: Bumping new
> > >> versions of
> > >> > > TMR
> > >> > > > > vs
> > >> > > > > > > > > grabbing
> > >> > > > > > > > > > > all
> > >> > > > > > > > > > > > > > > > protocol
> > >> > > > > > > > > > > > > > > > > > > > changes
> > >> > > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > one
> > >> > > > > > > > > > > > > > > > > > > > > > > > > version.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >     A: It was decided
> to
> > >> try
> > >> > to
> > >> > > > > > gather
> > >> > > > > > > > all
> > >> > > > > > > > > > > > changes
> > >> > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > protocol
> > >> > > > > > > > > > > > > > > > > > > > > > (before
> > >> > > > > > > > > > > > > > > > > > > > > > > > > release).
> > >> > > > > > > > > > > > > > > > > > > > > > > > >         In case of TMR
> > it
> > >> > worth
> > >> > > > > > > checking:
> > >> > > > > > > > > > > > > KAFKA-2020
> > >> > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > KIP-13
> > >> > > > > > > > > > > > > > > > > > > > > > > (quotas)
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > 6. Q: JSON lib is
> needed
> > >> to
> > >> > > > > > deserialize
> > >> > > > > > > > > > user's
> > >> > > > > > > > > > > > > input
> > >> > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > CLI
> > >> > > > > > > > > > > > > > > > > > tool.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >     A: Use jackson for
> > >> that,
> > >> > > > /tools
> > >> > > > > > > > project
> > >> > > > > > > > > > is
> > >> > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > separate
> > >> > > > > > > > > > > > > > > > > > jar so
> > >> > > > > > > > > > > > > > > > > > > > > > > shouldn't
> > >> > > > > > > > > > > > > > > > > > > > > > > > > be a big deal.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > 7.  Q:
> > >> > VerifyReassingPartitions
> > >> > > > vs
> > >> > > > > > > > generic
> > >> > > > > > > > > > > status
> > >> > > > > > > > > > > > > > check
> > >> > > > > > > > > > > > > > > > > > command.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >      A: For
> long-running
> > >> > > requests
> > >> > > > > > like
> > >> > > > > > > > > > reassign
> > >> > > > > > > > > > > > > > > > partitions
> > >> > > > > > > > > > > > > > > > > > > > > *progress*
> > >> > > > > > > > > > > > > > > > > > > > > > > check
> > >> > > > > > > > > > > > > > > > > > > > > > > > > request is useful,
> > >> > > > > > > > > > > > > > > > > > > > > > > > >          it makes
> sense
> > to
> > >> > > > > introduce
> > >> > > > > > > it.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > >  Please add, correct
> me
> > >> if I
> > >> > > > missed
> > >> > > > > > > > > > something.
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015
> at
> > >> 6:20
> > >> > > PM,
> > >> > > > > > Andrii
> > >> > > > > > > > > > > > Biletskyi <
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> andrii.biletskyi@stealth.ly>
> > >> > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > Joel,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > You are right, I
> > removed
> > >> > > > > > > > ClusterMetadata
> > >> > > > > > > > > > > > because
> > >> > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > have
> > >> > > > > > > > > > > > > > > > > > > > > partially
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > what we need in
> > >> > > TopicMetadata.
> > >> > > > > > Also,
> > >> > > > > > > as
> > >> > > > > > > > > Jay
> > >> > > > > > > > > > > > > pointed
> > >> > > > > > > > > > > > > > > out
> > >> > > > > > > > > > > > > > > > > > > > earlier,
> > >> > > > > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > would like to have
> > >> > > "orthogonal"
> > >> > > > > > API,
> > >> > > > > > > > but
> > >> > > > > > > > > at
> > >> > > > > > > > > > > the
> > >> > > > > > > > > > > > > > same
> > >> > > > > > > > > > > > > > > > time
> > >> > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > need
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > to be backward
> > >> compatible.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > But I like your idea
> > and
> > >> > even
> > >> > > > > have
> > >> > > > > > > some
> > >> > > > > > > > > > other
> > >> > > > > > > > > > > > > > > arguments
> > >> > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > option:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > There is also
> > >> > > > > DescribeTopicRequest
> > >> > > > > > > > which
> > >> > > > > > > > > > was
> > >> > > > > > > > > > > > > > proposed
> > >> > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > KIP,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > it returns topic
> > >> configs,
> > >> > > > > > partitions,
> > >> > > > > > > > > > > > replication
> > >> > > > > > > > > > > > > > > > factor
> > >> > > > > > > > > > > > > > > > > > plus
> > >> > > > > > > > > > > > > > > > > > > > > > > partition
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > ISR, ASR,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > leader replica. The
> > >> later
> > >> > > part
> > >> > > > is
> > >> > > > > > > > really
> > >> > > > > > > > > > > > already
> > >> > > > > > > > > > > > > > > there
> > >> > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >
> TopicMetadataRequest.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > So again we'll have
> to
> > >> add
> > >> > > > stuff
> > >> > > > > to
> > >> > > > > > > > TMR,
> > >> > > > > > > > > > not
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > duplicate
> > >> > > > > > > > > > > > > > > > > > some
> > >> > > > > > > > > > > > > > > > > > > > > info
> > >> > > > > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > newly added
> requests.
> > >> > > However,
> > >> > > > > this
> > >> > > > > > > way
> > >> > > > > > > > > > we'll
> > >> > > > > > > > > > > > end
> > >> > > > > > > > > > > > > > up
> > >> > > > > > > > > > > > > > > > with
> > >> > > > > > > > > > > > > > > > > > > > > "monster"
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > request which
> returns
> > >> > cluster
> > >> > > > > > > metadata,
> > >> > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > replication
> > >> > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > config
> > >> > > > > > > > > > > > > > > > > > > > > > > info
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > plus partition
> > >> replication
> > >> > > > data.
> > >> > > > > > > Seems
> > >> > > > > > > > > > > logical
> > >> > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > split
> > >> > > > > > > > > > > > > > > > > > TMR to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > - ClusterMetadata
> > >> (brokers
> > >> > +
> > >> > > > > > > > controller,
> > >> > > > > > > > > > > maybe
> > >> > > > > > > > > > > > > smth
> > >> > > > > > > > > > > > > > > > else)
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > - TopicMetadata
> (topic
> > >> > info +
> > >> > > > > > > partition
> > >> > > > > > > > > > > > details)
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > But since current
> TMR
> > is
> > >> > > > involved
> > >> > > > > > in
> > >> > > > > > > > lots
> > >> > > > > > > > > > of
> > >> > > > > > > > > > > > > places
> > >> > > > > > > > > > > > > > > > > > (including
> > >> > > > > > > > > > > > > > > > > > > > > > > network
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > client,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > as I understand)
> this
> > >> might
> > >> > > be
> > >> > > > > very
> > >> > > > > > > > > serious
> > >> > > > > > > > > > > > > change
> > >> > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > it
> > >> > > > > > > > > > > > > > > > > > > > > probably
> > >> > > > > > > > > > > > > > > > > > > > > > > makes
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > sense to stick with
> > >> current
> > >> > > > > > approach.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015
> > at
> > >> > 5:29
> > >> > > > PM,
> > >> > > > > > Joel
> > >> > > > > > > > > > Koshy <
> > >> > > > > > > > > > > > > > > > > > > > jjkoshy.w@gmail.com
> > >> > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> I may be missing
> some
> > >> > > context
> > >> > > > > but
> > >> > > > > > > > > > hopefully
> > >> > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > > also be
> > >> > > > > > > > > > > > > > > > > > > > > > > covered
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> today: I thought
> the
> > >> > earlier
> > >> > > > > > > proposal
> > >> > > > > > > > > > where
> > >> > > > > > > > > > > > > there
> > >> > > > > > > > > > > > > > > was
> > >> > > > > > > > > > > > > > > > an
> > >> > > > > > > > > > > > > > > > > > > > > explicit
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> ClusterMetadata
> > request
> > >> > was
> > >> > > > > > clearer
> > >> > > > > > > > and
> > >> > > > > > > > > > > > > explicit.
> > >> > > > > > > > > > > > > > > > During
> > >> > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > course
> > >> > > > > > > > > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> this thread I think
> > the
> > >> > > > > conclusion
> > >> > > > > > > was
> > >> > > > > > > > > > that
> > >> > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > main
> > >> > > > > > > > > > > > > > > > > need
> > >> > > > > > > > > > > > > > > > > > was
> > >> > > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> controller
> > information
> > >> and
> > >> > > > that
> > >> > > > > > can
> > >> > > > > > > be
> > >> > > > > > > > > > > rolled
> > >> > > > > > > > > > > > > into
> > >> > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > metadata
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> response but that
> > >> seems a
> > >> > > bit
> > >> > > > > > > > irrelevant
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > metadata.
> > >> > > > > > > > > > > > > > > > > > > > > FWIW I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> think the full
> > >> broker-list
> > >> > > is
> > >> > > > > also
> > >> > > > > > > > > > > irrelevant
> > >> > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > metadata,
> > >> > > > > > > > > > > > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> it is already there
> > >> and in
> > >> > > > use.
> > >> > > > > I
> > >> > > > > > > > think
> > >> > > > > > > > > > > there
> > >> > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > still
> > >> > > > > > > > > > > > > > > > > > room
> > >> > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > an
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> explicit
> > >> ClusterMetadata
> > >> > > > request
> > >> > > > > > > since
> > >> > > > > > > > > > there
> > >> > > > > > > > > > > > may
> > >> > > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > other
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> cluster-level
> > >> information
> > >> > > that
> > >> > > > > we
> > >> > > > > > > may
> > >> > > > > > > > > want
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > add
> > >> > > > > > > > > > > > > > > over
> > >> > > > > > > > > > > > > > > > > > time
> > >> > > > > > > > > > > > > > > > > > > > (and
> > >> > > > > > > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> have nothing to do
> > with
> > >> > > topic
> > >> > > > > > > > metadata).
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> On Tue, Mar 17,
> 2015
> > at
> > >> > > > > 02:45:30PM
> > >> > > > > > > > > +0200,
> > >> > > > > > > > > > > > Andrii
> > >> > > > > > > > > > > > > > > > > Biletskyi
> > >> > > > > > > > > > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > Jun,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > 101. Okay, if you
> > say
> > >> > that
> > >> > > > > such
> > >> > > > > > > use
> > >> > > > > > > > > case
> > >> > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > important. I
> > >> > > > > > > > > > > > > > > > > > > > also
> > >> > > > > > > > > > > > > > > > > > > > > > > think
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > using clientId
> for
> > >> these
> > >> > > > > > purposes
> > >> > > > > > > is
> > >> > > > > > > > > > fine
> > >> > > > > > > > > > > -
> > >> > > > > > > > > > > > if
> > >> > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > already
> > >> > > > > > > > > > > > > > > > > > > > have
> > >> > > > > > > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> field
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > as part of all
> Wire
> > >> > > protocol
> > >> > > > > > > > messages,
> > >> > > > > > > > > > why
> > >> > > > > > > > > > > > not
> > >> > > > > > > > > > > > > > use
> > >> > > > > > > > > > > > > > > > > that.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > I will update
> KIP-4
> > >> page
> > >> > > if
> > >> > > > > > nobody
> > >> > > > > > > > has
> > >> > > > > > > > > > > other
> > >> > > > > > > > > > > > > > ideas
> > >> > > > > > > > > > > > > > > > > > (which
> > >> > > > > > > > > > > > > > > > > > > > may
> > >> > > > > > > > > > > > > > > > > > > > > > > come up
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > during the call
> > >> today).
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll
> > >> update
> > >> > > the
> > >> > > > > KIP
> > >> > > > > > > > > > > > accordingly.
> > >> > > > > > > > > > > > > I
> > >> > > > > > > > > > > > > > > > think
> > >> > > > > > > > > > > > > > > > > > we can
> > >> > > > > > > > > > > > > > > > > > > > > add
> > >> > > > > > > > > > > > > > > > > > > > > > > new,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > fine-grained
> error
> > >> codes
> > >> > > if
> > >> > > > > some
> > >> > > > > > > > error
> > >> > > > > > > > > > > code
> > >> > > > > > > > > > > > > > > received
> > >> > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > specific
> > >> > > > > > > > > > > > > > > > > > > > > > > case
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > won't give enough
> > >> > context
> > >> > > to
> > >> > > > > > > return
> > >> > > > > > > > a
> > >> > > > > > > > > > > > > > descriptive
> > >> > > > > > > > > > > > > > > > > error
> > >> > > > > > > > > > > > > > > > > > > > > message
> > >> > > > > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> user.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > Look forward to
> > >> > discussing
> > >> > > > all
> > >> > > > > > > > > > outstanding
> > >> > > > > > > > > > > > > > issues
> > >> > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > detail
> > >> > > > > > > > > > > > > > > > > > > > > > today
> > >> > > > > > > > > > > > > > > > > > > > > > > > > during
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > the call.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16,
> > 2015
> > >> at
> > >> > > > 10:59
> > >> > > > > > PM,
> > >> > > > > > > > Jun
> > >> > > > > > > > > > Rao
> > >> > > > > > > > > > > <
> > >> > > > > > > > > > > > > > > > > > jun@confluent.io
> > >> > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 101. There may
> > be a
> > >> > use
> > >> > > > case
> > >> > > > > > > where
> > >> > > > > > > > > you
> > >> > > > > > > > > > > > only
> > >> > > > > > > > > > > > > > want
> > >> > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > topics
> > >> > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> created
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > manually by
> > admins.
> > >> > > > > Currently,
> > >> > > > > > > you
> > >> > > > > > > > > can
> > >> > > > > > > > > > > do
> > >> > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > by
> > >> > > > > > > > > > > > > > > > > > > > disabling
> > >> > > > > > > > > > > > > > > > > > > > > > auto
> > >> > > > > > > > > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > creation and
> > issue
> > >> > topic
> > >> > > > > > > creation
> > >> > > > > > > > > from
> > >> > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > TopicCommand.
> > >> > > > > > > > > > > > > > > > > > > > If
> > >> > > > > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> disable auto
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > topic creation
> > >> > > completely
> > >> > > > on
> > >> > > > > > the
> > >> > > > > > > > > > broker
> > >> > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > don't
> > >> > > > > > > > > > > > > > > > > > have a
> > >> > > > > > > > > > > > > > > > > > > > way
> > >> > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> distinguish
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > between topic
> > >> creation
> > >> > > > > > requests
> > >> > > > > > > > from
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > > > regular
> > >> > > > > > > > > > > > > > > > > > clients
> > >> > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> admin, we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > can't support
> > >> manual
> > >> > > topic
> > >> > > > > > > > creation
> > >> > > > > > > > > > any
> > >> > > > > > > > > > > > > more.
> > >> > > > > > > > > > > > > > I
> > >> > > > > > > > > > > > > > > > was
> > >> > > > > > > > > > > > > > > > > > > > thinking
> > >> > > > > > > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> another
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > way of
> > >> distinguishing
> > >> > > the
> > >> > > > > > > clients
> > >> > > > > > > > > > making
> > >> > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > creation
> > >> > > > > > > > > > > > > > > > > > > > > > > requests
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > using clientId.
> > For
> > >> > > > example,
> > >> > > > > > the
> > >> > > > > > > > > admin
> > >> > > > > > > > > > > > tool
> > >> > > > > > > > > > > > > > can
> > >> > > > > > > > > > > > > > > > set
> > >> > > > > > > > > > > > > > > > > > it to
> > >> > > > > > > > > > > > > > > > > > > > > > > something
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> like
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin and the
> > >> broker
> > >> > can
> > >> > > > > treat
> > >> > > > > > > > that
> > >> > > > > > > > > > > > clientId
> > >> > > > > > > > > > > > > > > > > > specially.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Also, there is
> a
> > >> > related
> > >> > > > > > > > discussion
> > >> > > > > > > > > in
> > >> > > > > > > > > > > > > > > KAFKA-2020.
> > >> > > > > > > > > > > > > > > > > > > > > Currently,
> > >> > > > > > > > > > > > > > > > > > > > > > > we do
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > following in
> > >> > > > > > > > TopicMetadataResponse:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 1. If leader is
> > not
> > >> > > > > available,
> > >> > > > > > > we
> > >> > > > > > > > > set
> > >> > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > partition
> > >> > > > > > > > > > > > > > > > > > level
> > >> > > > > > > > > > > > > > > > > > > > > > error
> > >> > > > > > > > > > > > > > > > > > > > > > > code
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > LeaderNotAvailable.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 2. If a
> > non-leader
> > >> > > replica
> > >> > > > > is
> > >> > > > > > > not
> > >> > > > > > > > > > > > available,
> > >> > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > take
> > >> > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > replica
> > >> > > > > > > > > > > > > > > > > > > > > > > > > out
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > the assigned
> > >> replica
> > >> > > list
> > >> > > > > and
> > >> > > > > > > isr
> > >> > > > > > > > in
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > > > > response.
> > >> > > > > > > > > > > > > > > > > As
> > >> > > > > > > > > > > > > > > > > > an
> > >> > > > > > > > > > > > > > > > > > > > > > > indication
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > doing that, we
> > set
> > >> the
> > >> > > > > > partition
> > >> > > > > > > > > level
> > >> > > > > > > > > > > > error
> > >> > > > > > > > > > > > > > > code
> > >> > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> ReplicaNotAvailable.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > This has a few
> > >> > problems.
> > >> > > > > > First,
> > >> > > > > > > > > > > > > > > > ReplicaNotAvailable
> > >> > > > > > > > > > > > > > > > > > > > probably
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> shouldn't be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > an error, at
> > least
> > >> for
> > >> > > the
> > >> > > > > > > normal
> > >> > > > > > > > > > > > > > > > producer/consumer
> > >> > > > > > > > > > > > > > > > > > > > clients
> > >> > > > > > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > just
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> want
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to find out the
> > >> > leader.
> > >> > > > > > Second,
> > >> > > > > > > it
> > >> > > > > > > > > can
> > >> > > > > > > > > > > > > happen
> > >> > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > both
> > >> > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > leader
> > >> > > > > > > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > another replica
> > are
> > >> > not
> > >> > > > > > > available
> > >> > > > > > > > at
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > > same
> > >> > > > > > > > > > > > > > > > time.
> > >> > > > > > > > > > > > > > > > > > There
> > >> > > > > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > no
> > >> > > > > > > > > > > > > > > > > > > > > > > > > error
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> code
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to indicate
> both.
> > >> > Third,
> > >> > > > > even
> > >> > > > > > > if a
> > >> > > > > > > > > > > replica
> > >> > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > not
> > >> > > > > > > > > > > > > > > > > > > > available,
> > >> > > > > > > > > > > > > > > > > > > > > > > it's
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> still
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > useful to
> return
> > >> its
> > >> > > > replica
> > >> > > > > > id
> > >> > > > > > > > > since
> > >> > > > > > > > > > > some
> > >> > > > > > > > > > > > > > > clients
> > >> > > > > > > > > > > > > > > > > > (e.g.
> > >> > > > > > > > > > > > > > > > > > > > > admin
> > >> > > > > > > > > > > > > > > > > > > > > > > tool)
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> may
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > still make use
> of
> > >> it.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > One way to
> > address
> > >> > this
> > >> > > > > issue
> > >> > > > > > is
> > >> > > > > > > > to
> > >> > > > > > > > > > > always
> > >> > > > > > > > > > > > > > > return
> > >> > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > replica
> > >> > > > > > > > > > > > > > > > > > > > > > > id for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > leader,
> assigned
> > >> > > replicas,
> > >> > > > > and
> > >> > > > > > > isr
> > >> > > > > > > > > > > > > regardless
> > >> > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > whether
> > >> > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> corresponding
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > broker is live
> or
> > >> not.
> > >> > > > Since
> > >> > > > > > we
> > >> > > > > > > > also
> > >> > > > > > > > > > > > return
> > >> > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > list
> > >> > > > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > > > live
> > >> > > > > > > > > > > > > > > > > > > > > > > > > brokers,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > client can
> figure
> > >> out
> > >> > > > > whether
> > >> > > > > > a
> > >> > > > > > > > > leader
> > >> > > > > > > > > > > or
> > >> > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > replica
> > >> > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > live
> > >> > > > > > > > > > > > > > > > > > > > > or
> > >> > > > > > > > > > > > > > > > > > > > > > > not
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> and act
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > accordingly.
> This
> > >> way,
> > >> > > we
> > >> > > > > > don't
> > >> > > > > > > > need
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > > set
> > >> > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > partition
> > >> > > > > > > > > > > > > > > > > > > > > > level
> > >> > > > > > > > > > > > > > > > > > > > > > > > > error
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> code
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > when the leader
> > or
> > >> a
> > >> > > > replica
> > >> > > > > > is
> > >> > > > > > > > not
> > >> > > > > > > > > > > > > available.
> > >> > > > > > > > > > > > > > > > This
> > >> > > > > > > > > > > > > > > > > > > > doesn't
> > >> > > > > > > > > > > > > > > > > > > > > > > change
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> the wire
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > protocol, but
> > does
> > >> > > change
> > >> > > > > the
> > >> > > > > > > > > > semantics.
> > >> > > > > > > > > > > > > Since
> > >> > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > > evolving
> > >> > > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> protocol
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > of
> > >> > TopicMetadataRequest
> > >> > > > > here,
> > >> > > > > > we
> > >> > > > > > > > can
> > >> > > > > > > > > > > > > > potentially
> > >> > > > > > > > > > > > > > > > > > piggyback
> > >> > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > change.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 102.1 For those
> > >> types
> > >> > of
> > >> > > > > > errors
> > >> > > > > > > > due
> > >> > > > > > > > > to
> > >> > > > > > > > > > > > > invalid
> > >> > > > > > > > > > > > > > > > > input,
> > >> > > > > > > > > > > > > > > > > > > > > > shouldn't
> > >> > > > > > > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > guard it at
> > >> parameter
> > >> > > > > > validation
> > >> > > > > > > > > time
> > >> > > > > > > > > > > and
> > >> > > > > > > > > > > > > > throw
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> > >> InvalidArgumentException
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > without even
> > >> sending
> > >> > the
> > >> > > > > > request
> > >> > > > > > > > to
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > > > broker?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Jun
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16,
> > >> 2015
> > >> > at
> > >> > > > > 10:37
> > >> > > > > > > AM,
> > >> > > > > > > > > > Andrii
> > >> > > > > > > > > > > > > > > > Biletskyi <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > andrii.biletskyi@stealth.ly
> > >> > > > > >
> > >> > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Jun,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Answering
> your
> > >> > > > questions:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 101. If I
> > >> understand
> > >> > > you
> > >> > > > > > > > > correctly,
> > >> > > > > > > > > > > you
> > >> > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > saying
> > >> > > > > > > > > > > > > > > > > > > > future
> > >> > > > > > > > > > > > > > > > > > > > > > > producer
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > versions
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > (which
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will be
> ported
> > to
> > >> > > > TMR_V1)
> > >> > > > > > > won't
> > >> > > > > > > > be
> > >> > > > > > > > > > > able
> > >> > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > automatically
> > >> > > > > > > > > > > > > > > > > > > > > > > create
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> topic (if
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> unconditionally
> > >> > remove
> > >> > > > > topic
> > >> > > > > > > > > > creation
> > >> > > > > > > > > > > > from
> > >> > > > > > > > > > > > > > > > there).
> > >> > > > > > > > > > > > > > > > > > But
> > >> > > > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > need to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > preserve
> logic.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Ok, about
> your
> > >> > > proposal:
> > >> > > > > I'm
> > >> > > > > > > > not a
> > >> > > > > > > > > > big
> > >> > > > > > > > > > > > fan
> > >> > > > > > > > > > > > > > > too,
> > >> > > > > > > > > > > > > > > > > > when it
> > >> > > > > > > > > > > > > > > > > > > > > > comes
> > >> > > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> differentiating
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > clients
> > directly
> > >> in
> > >> > > > > protocol
> > >> > > > > > > > > schema.
> > >> > > > > > > > > > > And
> > >> > > > > > > > > > > > > > also
> > >> > > > > > > > > > > > > > > > I'm
> > >> > > > > > > > > > > > > > > > > > not
> > >> > > > > > > > > > > > > > > > > > > > > sure I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> understand
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > at
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > all why
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > >> > > > auto.create.topics.enable
> > >> > > > > > is a
> > >> > > > > > > > > > server
> > >> > > > > > > > > > > > side
> > >> > > > > > > > > > > > > > > > > > > > configuration.
> > >> > > > > > > > > > > > > > > > > > > > > > Can
> > >> > > > > > > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > deprecate
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > this setting
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in future
> > >> versions,
> > >> > > add
> > >> > > > > this
> > >> > > > > > > > > setting
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > producer
> > >> > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > based
> > >> > > > > > > > > > > > > > > > > > > > > > on
> > >> > > > > > > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> upon
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > receiving
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > UnknownTopic
> > >> create
> > >> > > > topic
> > >> > > > > > > > > explicitly
> > >> > > > > > > > > > > by
> > >> > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > separate
> > >> > > > > > > > > > > > > > > > > > > > > producer
> > >> > > > > > > > > > > > > > > > > > > > > > > call
> > >> > > > > > > > > > > > > > > > > > > > > > > > > via
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > adminClient?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm,
> yes.
> > >> It's
> > >> > > > > because
> > >> > > > > > > we
> > >> > > > > > > > > want
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > support
> > >> > > > > > > > > > > > > > > > > > batching
> > >> > > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > at
> > >> > > > > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> same
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > time we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > want to give
> > >> > > descriptive
> > >> > > > > > error
> > >> > > > > > > > > > > messages
> > >> > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > clients.
> > >> > > > > > > > > > > > > > > > > > > > Since
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> AdminClient
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > holds the
> > context
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to construct
> > such
> > >> > > > messages
> > >> > > > > > > (e.g.
> > >> > > > > > > > > > > > > AdminClient
> > >> > > > > > > > > > > > > > > > layer
> > >> > > > > > > > > > > > > > > > > > can
> > >> > > > > > > > > > > > > > > > > > > > > know
> > >> > > > > > > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > >> InvalidArgumentsCode
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > means two
> > cases:
> > >> > > either
> > >> > > > > > > invalid
> > >> > > > > > > > > > > number -
> > >> > > > > > > > > > > > > > e.g.
> > >> > > > > > > > > > > > > > > > -1;
> > >> > > > > > > > > > > > > > > > > or
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> replication-factor
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > was
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > provided
> while
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > partitions
> > >> argument
> > >> > > > > wasn't)
> > >> > > > > > -
> > >> > > > > > > I
> > >> > > > > > > > > > > wrapped
> > >> > > > > > > > > > > > > > > > responses
> > >> > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > Exceptions.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> But I'm
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > open to any
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > other ideas,
> > this
> > >> > was
> > >> > > > just
> > >> > > > > > > > initial
> > >> > > > > > > > > > > > > version.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I
> > >> agree.
> > >> > > > I'll
> > >> > > > > > > change
> > >> > > > > > > > > > that
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > probably
> > >> > > > > > > > > > > > > > > > > > some
> > >> > > > > > > > > > > > > > > > > > > > > other
> > >> > > > > > > > > > > > > > > > > > > > > > > dto.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Andrii
> > Biletskyi
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar
> 13,
> > >> 2015
> > >> > > at
> > >> > > > > 7:16
> > >> > > > > > > PM,
> > >> > > > > > > > > Jun
> > >> > > > > > > > > > > > Rao <
> > >> > > > > > > > > > > > > > > > > > > > > jun@confluent.io>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 101. That's
> > >> what I
> > >> > > was
> > >> > > > > > > > thinking
> > >> > > > > > > > > > too,
> > >> > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > it
> > >> > > > > > > > > > > > > > > > may
> > >> > > > > > > > > > > > > > > > > > not be
> > >> > > > > > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> simple. In
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > TopicMetadataRequest_V1,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we can let
> it
> > >> not
> > >> > > > > trigger
> > >> > > > > > > auto
> > >> > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > creation.
> > >> > > > > > > > > > > > > > > > > > Then,
> > >> > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> producer
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > side,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > if it gets
> an
> > >> > > > > > > > > > UnknownTopicException,
> > >> > > > > > > > > > > > it
> > >> > > > > > > > > > > > > > can
> > >> > > > > > > > > > > > > > > > > > explicitly
> > >> > > > > > > > > > > > > > > > > > > > > > > issue a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> createTopicRequest
> > >> > > for
> > >> > > > > > auto
> > >> > > > > > > > > topic
> > >> > > > > > > > > > > > > > creation.
> > >> > > > > > > > > > > > > > > On
> > >> > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > consumer
> > >> > > > > > > > > > > > > > > > > > > > > > > > > side,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > will
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > never issue
> > >> > > > > > > > createTopicRequest.
> > >> > > > > > > > > > This
> > >> > > > > > > > > > > > > works
> > >> > > > > > > > > > > > > > > > when
> > >> > > > > > > > > > > > > > > > > > auto
> > >> > > > > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> creation is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > enabled on
> > the
> > >> > > broker
> > >> > > > > > side.
> > >> > > > > > > > > > > However, I
> > >> > > > > > > > > > > > > am
> > >> > > > > > > > > > > > > > > not
> > >> > > > > > > > > > > > > > > > > > sure how
> > >> > > > > > > > > > > > > > > > > > > > > > > things
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> will work
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > when auto
> > topic
> > >> > > > creation
> > >> > > > > > is
> > >> > > > > > > > > > disabled
> > >> > > > > > > > > > > > on
> > >> > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > broker
> > >> > > > > > > > > > > > > > > > > > > > side.
> > >> > > > > > > > > > > > > > > > > > > > > > In
> > >> > > > > > > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> case,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > want to
> have
> > a
> > >> way
> > >> > > to
> > >> > > > > > > manually
> > >> > > > > > > > > > > create
> > >> > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > topic,
> > >> > > > > > > > > > > > > > > > > > > > > potentially
> > >> > > > > > > > > > > > > > > > > > > > > > > > > through
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > commands.
> > >> However,
> > >> > > > then
> > >> > > > > we
> > >> > > > > > > > need
> > >> > > > > > > > > a
> > >> > > > > > > > > > > way
> > >> > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > distinguish
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> createTopicRequest
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > issued from
> > the
> > >> > > > producer
> > >> > > > > > > > clients
> > >> > > > > > > > > > and
> > >> > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > admin
> > >> > > > > > > > > > > > > > > > > > tools.
> > >> > > > > > > > > > > > > > > > > > > > > May
> > >> > > > > > > > > > > > > > > > > > > > > > > be we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > new field
> in
> > >> > > > > > > > createTopicRequest
> > >> > > > > > > > > > and
> > >> > > > > > > > > > > > set
> > >> > > > > > > > > > > > > it
> > >> > > > > > > > > > > > > > > > > > differently
> > >> > > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> producer
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > client and
> > the
> > >> > admin
> > >> > > > > > client.
> > >> > > > > > > > > > > However,
> > >> > > > > > > > > > > > I
> > >> > > > > > > > > > > > > am
> > >> > > > > > > > > > > > > > > not
> > >> > > > > > > > > > > > > > > > > > sure if
> > >> > > > > > > > > > > > > > > > > > > > > > > that's
> > >> > > > > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> best
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > approach.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 2. Yes,
> > >> > refactoring
> > >> > > > > > existing
> > >> > > > > > > > > > > requests
> > >> > > > > > > > > > > > > is a
> > >> > > > > > > > > > > > > > > > > > non-trivial
> > >> > > > > > > > > > > > > > > > > > > > > > > amount of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> work.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > posted some
> > >> > comments
> > >> > > > in
> > >> > > > > > > > > > KAFKA-1927.
> > >> > > > > > > > > > > We
> > >> > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > > probably
> > >> > > > > > > > > > > > > > > > > > > > > have
> > >> > > > > > > > > > > > > > > > > > > > > > > to fix
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > first,
> before
> > >> > adding
> > >> > > > the
> > >> > > > > > new
> > >> > > > > > > > > logic
> > >> > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > KAFKA-1694.
> > >> > > > > > > > > > > > > > > > > > > > > > > Otherwise, the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > changes
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > will be too
> > >> big.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102. About
> > the
> > >> > > > > > AdminClient:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.1.
> It's a
> > >> bit
> > >> > > > weird
> > >> > > > > > that
> > >> > > > > > > > we
> > >> > > > > > > > > > > return
> > >> > > > > > > > > > > > > > > > exception
> > >> > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > api. It
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> seems
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we should
> > >> either
> > >> > > > return
> > >> > > > > > > error
> > >> > > > > > > > > code
> > >> > > > > > > > > > > or
> > >> > > > > > > > > > > > > > throw
> > >> > > > > > > > > > > > > > > an
> > >> > > > > > > > > > > > > > > > > > > > exception
> > >> > > > > > > > > > > > > > > > > > > > > > > when
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> getting
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > response
> > state.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.2. We
> > >> probably
> > >> > > > > > shouldn't
> > >> > > > > > > > > > > > explicitly
> > >> > > > > > > > > > > > > > use
> > >> > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > request
> > >> > > > > > > > > > > > > > > > > > > > > > > object
> > >> > > > > > > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > api.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Not every
> > >> request
> > >> > > > > > evolution
> > >> > > > > > > > > > requires
> > >> > > > > > > > > > > > an
> > >> > > > > > > > > > > > > > api
> > >> > > > > > > > > > > > > > > > > > change.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Jun
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar
> > 13,
> > >> > 2015
> > >> > > > at
> > >> > > > > > 4:08
> > >> > > > > > > > AM,
> > >> > > > > > > > > > > > Andrii
> > >> > > > > > > > > > > > > > > > > Biletskyi
> > >> > > > > > > > > > > > > > > > > > <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > > > andrii.biletskyi@stealth.ly
> > >> > > > > > > >
> > >> > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks
> for
> > >> you
> > >> > > > > comments.
> > >> > > > > > > > > Answers
> > >> > > > > > > > > > > > > inline:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 100.
> There
> > >> are a
> > >> > > few
> > >> > > > > > > fields
> > >> > > > > > > > > such
> > >> > > > > > > > > > > as
> > >> > > > > > > > > > > > > > > > > > > > ReplicaAssignment,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > ReassignPartitionRequest,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > >> > > > > > PartitionsSerialized
> > >> > > > > > > > > that
> > >> > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > represented
> > >> > > > > > > > > > > > > > > > > > as a
> > >> > > > > > > > > > > > > > > > > > > > > > > string,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> composite
> > >> > > > structures
> > >> > > > > > in
> > >> > > > > > > > > json.
> > >> > > > > > > > > > > > Could
> > >> > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > flatten
> > >> > > > > > > > > > > > > > > > > > > > them
> > >> > > > > > > > > > > > > > > > > > > > > > out
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> directly in
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> protocol
> > >> > > > definition
> > >> > > > > as
> > >> > > > > > > > > > > > > arrays/records?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, now
> > with
> > >> > > Admin
> > >> > > > > > Client
> > >> > > > > > > > > this
> > >> > > > > > > > > > > > looks
> > >> > > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > bit
> > >> > > > > > > > > > > > > > > > > > weird.
> > >> > > > > > > > > > > > > > > > > > > > My
> > >> > > > > > > > > > > > > > > > > > > > > > > initial
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > motivation
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > was:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > ReassignPartitionCommand
> > >> > > > > > > > > accepts
> > >> > > > > > > > > > > > input
> > >> > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > json,
> > >> > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > want
> > >> > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> remain
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > tools'
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> interfaces
> > >> > > > unchanged,
> > >> > > > > > > where
> > >> > > > > > > > > > > > possible.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > If we
> port
> > >> it to
> > >> > > > > > > > deserialized
> > >> > > > > > > > > > > > format,
> > >> > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > CLI
> > >> > > > > > > > > > > > > > > > > > (/tools
> > >> > > > > > > > > > > > > > > > > > > > > > > project)
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> we will
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > have
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to add
> some
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json
> > library
> > >> > since
> > >> > > > > > /tools
> > >> > > > > > > is
> > >> > > > > > > > > > > written
> > >> > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > java
> > >> > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > we'll
> > >> > > > > > > > > > > > > > > > > > > > > > > need to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > deserialize
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json file
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > provided
> > by a
> > >> > > user.
> > >> > > > > Can
> > >> > > > > > we
> > >> > > > > > > > > > quickly
> > >> > > > > > > > > > > > > agree
> > >> > > > > > > > > > > > > > > on
> > >> > > > > > > > > > > > > > > > > what
> > >> > > > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > library
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> should
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > (Jackson,
> > >> GSON,
> > >> > > > > > whatever)?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> > >> > > > > > > > TopicMetadataRequest
> > >> > > > > > > > > > v1
> > >> > > > > > > > > > > > > still
> > >> > > > > > > > > > > > > > > > > trigger
> > >> > > > > > > > > > > > > > > > > > auto
> > >> > > > > > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> creation?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > This
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> be a
> > >> bit
> > >> > > > weird
> > >> > > > > > now
> > >> > > > > > > > that
> > >> > > > > > > > > > we
> > >> > > > > > > > > > > > > have a
> > >> > > > > > > > > > > > > > > > > > separate
> > >> > > > > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> creation api.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you
> > thought
> > >> > > about
> > >> > > > > how
> > >> > > > > > > the
> > >> > > > > > > > > new
> > >> > > > > > > > > > > > > > > > > > createTopicRequest
> > >> > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > TopicMetadataRequest
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will
> > be
> > >> > used
> > >> > > in
> > >> > > > > the
> > >> > > > > > > > > > > > > > producer/consumer
> > >> > > > > > > > > > > > > > > > > > client,
> > >> > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > addition
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools?
> > For
> > >> > > > example,
> > >> > > > > > > > ideally,
> > >> > > > > > > > > > we
> > >> > > > > > > > > > > > > don't
> > >> > > > > > > > > > > > > > > want
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> TopicMetadataRequest
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > from
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> consumer
> > to
> > >> > > > trigger
> > >> > > > > > auto
> > >> > > > > > > > > topic
> > >> > > > > > > > > > > > > > creation.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > I agree,
> > this
> > >> > > > strange
> > >> > > > > > > logic
> > >> > > > > > > > > > should
> > >> > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > fixed.
> > >> > > > > > > > > > > > > > > > > > I'm not
> > >> > > > > > > > > > > > > > > > > > > > > > > confident
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> in
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka
> part
> > so
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > correct
> me
> > if
> > >> > I'm
> > >> > > > > wrong,
> > >> > > > > > > but
> > >> > > > > > > > > it
> > >> > > > > > > > > > > > > doesn't
> > >> > > > > > > > > > > > > > > look
> > >> > > > > > > > > > > > > > > > > > like a
> > >> > > > > > > > > > > > > > > > > > > > > hard
> > >> > > > > > > > > > > > > > > > > > > > > > > thing
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > do, I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > think we
> > can
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > leverage
> > >> > > AdminClient
> > >> > > > > for
> > >> > > > > > > > that
> > >> > > > > > > > > in
> > >> > > > > > > > > > > > > > Producer
> > >> > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > unconditionally
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> remove
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > creation
> > from
> > >> > the
> > >> > > > > > > > > > > > > > TopicMetadataRequest_V1.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 2. I
> think
> > >> Jay
> > >> > > meant
> > >> > > > > > > getting
> > >> > > > > > > > > rid
> > >> > > > > > > > > > > of
> > >> > > > > > > > > > > > > > scala
> > >> > > > > > > > > > > > > > > > > > classes
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > >> > > > > > > > > HeartbeatRequestAndHeader
> > >> > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> > >> > HeartbeatResponseAndHeader.
> > >> > > We
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > did
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a
> > >> stop-gap
> > >> > > > thing
> > >> > > > > > when
> > >> > > > > > > > > > adding
> > >> > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > new
> > >> > > > > > > > > > > > > > > > > > requests
> > >> > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> consumers.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> However,
> > >> the
> > >> > > long
> > >> > > > > term
> > >> > > > > > > > plan
> > >> > > > > > > > > is
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > get
> > >> > > > > > > > > > > > > > > rid
> > >> > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > all
> > >> > > > > > > > > > > > > > > > > > > > > those
> > >> > > > > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> > >> > > > > request/response
> > >> > > > > > in
> > >> > > > > > > > the
> > >> > > > > > > > > > > > client.
> > >> > > > > > > > > > > > > > > Since
> > >> > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > KIP
> > >> > > > > > > > > > > > > > > > > > > > > > > proposes
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > significant
> > >> > > number
> > >> > > > > of
> > >> > > > > > > new
> > >> > > > > > > > > > > > requests,
> > >> > > > > > > > > > > > > > > > perhaps
> > >> > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > should
> > >> > > > > > > > > > > > > > > > > > > > > > > bite
> > >> > > > > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean
> up
> > >> the
> > >> > > > > existing
> > >> > > > > > > > scala
> > >> > > > > > > > > > > > requests
> > >> > > > > > > > > > > > > > > first
> > >> > > > > > > > > > > > > > > > > > before
> > >> > > > > > > > > > > > > > > > > > > > > > > adding new
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> ones?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes,
> looks
> > >> like
> > >> > I
> > >> > > > > > > > > misunderstood
> > >> > > > > > > > > > > the
> > >> > > > > > > > > > > > > > point
> > >> > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> ...RequestAndHeader.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Okay, I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > will
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > rework
> > that.
> > >> The
> > >> > > > only
> > >> > > > > > > thing
> > >> > > > > > > > is
> > >> > > > > > > > > > > that
> > >> > > > > > > > > > > > I
> > >> > > > > > > > > > > > > > > don't
> > >> > > > > > > > > > > > > > > > > see
> > >> > > > > > > > > > > > > > > > > > any
> > >> > > > > > > > > > > > > > > > > > > > > > > example
> > >> > > > > > > > > > > > > > > > > > > > > > > > > how
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > was
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > done
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > for at
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > least one
> > >> > existing
> > >> > > > > > > protocol
> > >> > > > > > > > > > > message.
> > >> > > > > > > > > > > > > > Thus,
> > >> > > > > > > > > > > > > > > > as
> > >> > > > > > > > > > > > > > > > > I
> > >> > > > > > > > > > > > > > > > > > > > > > > understand, I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> have to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > think
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > how we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are going
> > to
> > >> do
> > >> > > it.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Re
> porting
> > >> all
> > >> > > > > existing
> > >> > > > > > > > RQ/RP
> > >> > > > > > > > > in
> > >> > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > patch.
> > >> > > > > > > > > > > > > > > > > > Sounds
> > >> > > > > > > > > > > > > > > > > > > > > > > > > reasonable,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> but
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > if
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > it's
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > an
> > >> *obligatory*
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> requirement
> > >> to
> > >> > > have
> > >> > > > > > Admin
> > >> > > > > > > > KIP
> > >> > > > > > > > > > > done,
> > >> > > > > > > > > > > > > I'm
> > >> > > > > > > > > > > > > > > > afraid
> > >> > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > can
> > >> > > > > > > > > > > > > > > > > > > > > > > be a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> serious
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > blocker
> for
> > >> us.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > There are
> > 13
> > >> > > > protocol
> > >> > > > > > > > messages
> > >> > > > > > > > > > and
> > >> > > > > > > > > > > > all
> > >> > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > would
> > >> > > > > > > > > > > > > > > > > > > > > > require
> > >> > > > > > > > > > > > > > > > > > > > > > > not
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> only
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > unit
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > tests but
> > >> quite
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > intensive
> > >> manual
> > >> > > > > > testing,
> > >> > > > > > > > no?
> > >> > > > > > > > > > I'm
> > >> > > > > > > > > > > > > afraid
> > >> > > > > > > > > > > > > > > I'm
> > >> > > > > > > > > > > > > > > > > > not the
> > >> > > > > > > > > > > > > > > > > > > > > > > right guy
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > cover
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > pretty
> much
> > >> all
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka
> core
> > >> > > internals
> > >> > > > > :).
> > >> > > > > > > Let
> > >> > > > > > > > > me
> > >> > > > > > > > > > > know
> > >> > > > > > > > > > > > > > your
> > >> > > > > > > > > > > > > > > > > > thoughts
> > >> > > > > > > > > > > > > > > > > > > > on
> > >> > > > > > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> item. Btw
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > there
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is a
> ticket
> > >> to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > follow-up
> > >> this
> > >> > > > issue (
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > >
> > >> https://issues.apache.org/jira/browse/KAFKA-2006
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > ).
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Andrii
> > >> Biletskyi
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > On Fri,
> Mar
> > >> 13,
> > >> > > 2015
> > >> > > > > at
> > >> > > > > > > 6:40
> > >> > > > > > > > > AM,
> > >> > > > > > > > > > > Jun
> > >> > > > > > > > > > > > > > Rao <
> > >> > > > > > > > > > > > > > > > > > > > > > > jun@confluent.io>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > A few
> > more
> > >> > > > comments.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 100.
> > There
> > >> > are a
> > >> > > > few
> > >> > > > > > > > fields
> > >> > > > > > > > > > such
> > >> > > > > > > > > > > > as
> > >> > > > > > > > > > > > > > > > > > > > > ReplicaAssignment,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > ReassignPartitionRequest,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > >> > > > > > PartitionsSerialized
> > >> > > > > > > > > that
> > >> > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > represented
> > >> > > > > > > > > > > > > > > > > > as a
> > >> > > > > > > > > > > > > > > > > > > > > > > string,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> composite
> > >> > > > structures
> > >> > > > > > in
> > >> > > > > > > > > json.
> > >> > > > > > > > > > > > Could
> > >> > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > flatten
> > >> > > > > > > > > > > > > > > > > > > > them
> > >> > > > > > > > > > > > > > > > > > > > > > out
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> directly in
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> protocol
> > >> > > > definition
> > >> > > > > as
> > >> > > > > > > > > > > > > arrays/records?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 101.
> Does
> > >> > > > > > > > > TopicMetadataRequest
> > >> > > > > > > > > > > v1
> > >> > > > > > > > > > > > > > still
> > >> > > > > > > > > > > > > > > > > > trigger
> > >> > > > > > > > > > > > > > > > > > > > auto
> > >> > > > > > > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > creation?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > This
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> be a
> > >> bit
> > >> > > > weird
> > >> > > > > > now
> > >> > > > > > > > that
> > >> > > > > > > > > > we
> > >> > > > > > > > > > > > > have a
> > >> > > > > > > > > > > > > > > > > > separate
> > >> > > > > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> creation api.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you
> > thought
> > >> > > about
> > >> > > > > how
> > >> > > > > > > the
> > >> > > > > > > > > new
> > >> > > > > > > > > > > > > > > > > > createTopicRequest
> > >> > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > TopicMetadataRequest
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will
> > be
> > >> > used
> > >> > > in
> > >> > > > > the
> > >> > > > > > > > > > > > > > producer/consumer
> > >> > > > > > > > > > > > > > > > > > client,
> > >> > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > addition
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools?
> > For
> > >> > > > example,
> > >> > > > > > > > ideally,
> > >> > > > > > > > > > we
> > >> > > > > > > > > > > > > don't
> > >> > > > > > > > > > > > > > > want
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> TopicMetadataRequest
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > from
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> consumer
> > to
> > >> > > > trigger
> > >> > > > > > auto
> > >> > > > > > > > > topic
> > >> > > > > > > > > > > > > > creation.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 2. I
> > think
> > >> Jay
> > >> > > > meant
> > >> > > > > > > > getting
> > >> > > > > > > > > > rid
> > >> > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > scala
> > >> > > > > > > > > > > > > > > > > > classes
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > >> > > > > > > > > HeartbeatRequestAndHeader
> > >> > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> > >> > HeartbeatResponseAndHeader.
> > >> > > We
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > did
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a
> > >> stop-gap
> > >> > > > thing
> > >> > > > > > when
> > >> > > > > > > > > > adding
> > >> > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > new
> > >> > > > > > > > > > > > > > > > > > requests
> > >> > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> consumers.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> However,
> > >> the
> > >> > > long
> > >> > > > > term
> > >> > > > > > > > plan
> > >> > > > > > > > > is
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > get
> > >> > > > > > > > > > > > > > > rid
> > >> > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > all
> > >> > > > > > > > > > > > > > > > > > > > > those
> > >> > > > > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> > >> > > > > request/response
> > >> > > > > > in
> > >> > > > > > > > the
> > >> > > > > > > > > > > > client.
> > >> > > > > > > > > > > > > > > Since
> > >> > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > KIP
> > >> > > > > > > > > > > > > > > > > > > > > > > proposes
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > significant
> > >> > > number
> > >> > > > > of
> > >> > > > > > > new
> > >> > > > > > > > > > > > requests,
> > >> > > > > > > > > > > > > > > > perhaps
> > >> > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > should
> > >> > > > > > > > > > > > > > > > > > > > > > > bite
> > >> > > > > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean
> up
> > >> the
> > >> > > > > existing
> > >> > > > > > > > scala
> > >> > > > > > > > > > > > requests
> > >> > > > > > > > > > > > > > > first
> > >> > > > > > > > > > > > > > > > > > before
> > >> > > > > > > > > > > > > > > > > > > > > > > adding new
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> ones?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > On Thu,
> > Mar
> > >> > 12,
> > >> > > > 2015
> > >> > > > > > at
> > >> > > > > > > > 3:37
> > >> > > > > > > > > > PM,
> > >> > > > > > > > > > > > > > Andrii
> > >> > > > > > > > > > > > > > > > > > Biletskyi
> > >> > > > > > > > > > > > > > > > > > > > <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > andrii.biletskyi@stealth.ly
> > >> > > > > > > > > >
> > >> > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > As
> said
> > >> > above
> > >> > > -
> > >> > > > I
> > >> > > > > > list
> > >> > > > > > > > > again
> > >> > > > > > > > > > > all
> > >> > > > > > > > > > > > > > > > comments
> > >> > > > > > > > > > > > > > > > > > from
> > >> > > > > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > thread
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> so we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can
> see
> > >> > what's
> > >> > > > > left
> > >> > > > > > > and
> > >> > > > > > > > > > > finalize
> > >> > > > > > > > > > > > > all
> > >> > > > > > > > > > > > > > > > > pending
> > >> > > > > > > > > > > > > > > > > > > > > issues.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > Comments
> > >> > from
> > >> > > > Jay:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 1.
> This
> > >> is
> > >> > > much
> > >> > > > > > needed
> > >> > > > > > > > > > > > > > functionality,
> > >> > > > > > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > > > > > there
> > >> > > > > > > > > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > > > > > > > > lot
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> of the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > so
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > let's
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> really
> > >> think
> > >> > > > these
> > >> > > > > > > > > protocols
> > >> > > > > > > > > > > > > > through.
> > >> > > > > > > > > > > > > > > We
> > >> > > > > > > > > > > > > > > > > > really
> > >> > > > > > > > > > > > > > > > > > > > > want
> > >> > > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > end
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> up
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > with a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > set
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > of
> well
> > >> > > > > thought-out,
> > >> > > > > > > > > > > orthoganol
> > >> > > > > > > > > > > > > > apis.
> > >> > > > > > > > > > > > > > > > For
> > >> > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > reason
> > >> > > > > > > > > > > > > > > > > > > > > > > I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> think it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > really
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> important to
> > >> > > > think
> > >> > > > > > > > through
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > end
> > >> > > > > > > > > > > > > > > state
> > >> > > > > > > > > > > > > > > > > > even if
> > >> > > > > > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> includes
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > APIs
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > won't
> > >> > > implement
> > >> > > > in
> > >> > > > > > the
> > >> > > > > > > > > first
> > >> > > > > > > > > > > > > phase.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> > >> > Definitely
> > >> > > > > behind
> > >> > > > > > > > this.
> > >> > > > > > > > > > > Would
> > >> > > > > > > > > > > > > > > > > appreciate
> > >> > > > > > > > > > > > > > > > > > if
> > >> > > > > > > > > > > > > > > > > > > > > there
> > >> > > > > > > > > > > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> concrete
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> comments
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > how
> > this
> > >> can
> > >> > > be
> > >> > > > > > > > improved.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 2.
> > Let's
> > >> > > please
> > >> > > > > > please
> > >> > > > > > > > > > please
> > >> > > > > > > > > > > > wait
> > >> > > > > > > > > > > > > > > until
> > >> > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > have
> > >> > > > > > > > > > > > > > > > > > > > > > > switched
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > server
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > over
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to
> the
> > >> new
> > >> > > java
> > >> > > > > > > protocol
> > >> > > > > > > > > > > > > > definitions.
> > >> > > > > > > > > > > > > > > If
> > >> > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > add
> > >> > > > > > > > > > > > > > > > > > > > > > upteen
> > >> > > > > > > > > > > > > > > > > > > > > > > > > more
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> ad
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > hoc
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > scala
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> objects
> > >> that
> > >> > > is
> > >> > > > > just
> > >> > > > > > > > > > > generating
> > >> > > > > > > > > > > > > more
> > >> > > > > > > > > > > > > > > > work
> > >> > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> conversion we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > know
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > have
> to
> > >> do.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> > Fixed
> > >> in
> > >> > > the
> > >> > > > > > latest
> > >> > > > > > > > > > patch -
> > >> > > > > > > > > > > > > > removed
> > >> > > > > > > > > > > > > > > > > scala
> > >> > > > > > > > > > > > > > > > > > > > > > protocol
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> classes.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 3.
> This
> > >> > > proposal
> > >> > > > > > > > > introduces
> > >> > > > > > > > > > a
> > >> > > > > > > > > > > > new
> > >> > > > > > > > > > > > > > type
> > >> > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > > > optional
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> parameter.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > This
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> inconsistent
> > >> > > > with
> > >> > > > > > > > > everything
> > >> > > > > > > > > > > > else
> > >> > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > protocol
> > >> > > > > > > > > > > > > > > > > > > > > > > where we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> use -1
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > or
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > some
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > other
> > >> marker
> > >> > > > > value.
> > >> > > > > > > You
> > >> > > > > > > > > > could
> > >> > > > > > > > > > > > > argue
> > >> > > > > > > > > > > > > > > > either
> > >> > > > > > > > > > > > > > > > > > way
> > >> > > > > > > > > > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > > > > > > > > > > let's
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> stick
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > with
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > for
> > >> > > consistency.
> > >> > > > > For
> > >> > > > > > > > > clients
> > >> > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > implemented
> > >> > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > protocol
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> in a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > better
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > way
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
> > our
> > >> > scala
> > >> > > > > code
> > >> > > > > > > > these
> > >> > > > > > > > > > > basic
> > >> > > > > > > > > > > > > > > > primitives
> > >> > > > > > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > > hard
> > >> > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> change.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> > Fixed
> > >> in
> > >> > > the
> > >> > > > > > latest
> > >> > > > > > > > > > patch -
> > >> > > > > > > > > > > > > > removed
> > >> > > > > > > > > > > > > > > > > > MaybeOf
> > >> > > > > > > > > > > > > > > > > > > > > type
> > >> > > > > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> changed
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > protocol
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> accordingly.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 4.
> > >> > > > > ClusterMetadata:
> > >> > > > > > > This
> > >> > > > > > > > > > seems
> > >> > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > duplicate
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> TopicMetadataRequest
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > which
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > brokers,
> > >> > > topics,
> > >> > > > > and
> > >> > > > > > > > > > > > partitions. I
> > >> > > > > > > > > > > > > > > think
> > >> > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > should
> > >> > > > > > > > > > > > > > > > > > > > > > > rename
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > request
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > ClusterMetadataRequest
> > >> > > > > > > > (or
> > >> > > > > > > > > > > just
> > >> > > > > > > > > > > > > > > > > > MetadataRequest)
> > >> > > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> include the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > id
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> controller.
> > >> > Or
> > >> > > > are
> > >> > > > > > > there
> > >> > > > > > > > > > other
> > >> > > > > > > > > > > > > > things
> > >> > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > could
> > >> > > > > > > > > > > > > > > > > > > > add
> > >> > > > > > > > > > > > > > > > > > > > > > > here?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I
> > >> agree.
> > >> > > > > Updated
> > >> > > > > > > the
> > >> > > > > > > > > KIP.
> > >> > > > > > > > > > > > Let's
> > >> > > > > > > > > > > > > > > > extends
> > >> > > > > > > > > > > > > > > > > > > > > > > TopicMetadata
> > >> > > > > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > version 2
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> include
> > >> > > > > controller.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We
> > >> have a
> > >> > > > > > tendency
> > >> > > > > > > to
> > >> > > > > > > > > try
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > > make a
> > >> > > > > > > > > > > > > > > > lot
> > >> > > > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > > > > > requests
> > >> > > > > > > > > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > only
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > go
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> particular
> > >> > > > nodes.
> > >> > > > > > This
> > >> > > > > > > > > adds
> > >> > > > > > > > > > a
> > >> > > > > > > > > > > > lot
> > >> > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > burden
> > >> > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > client
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > implementations
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > (it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> sounds
> > >> easy
> > >> > > but
> > >> > > > > each
> > >> > > > > > > > > > discovery
> > >> > > > > > > > > > > > can
> > >> > > > > > > > > > > > > > > fail
> > >> > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > many
> > >> > > > > > > > > > > > > > > > > > > > > > parts
> > >> > > > > > > > > > > > > > > > > > > > > > > so
> > >> > > > > > > > > > > > > > > > > > > > > > > > > it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> ends
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > up
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > being a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > full
> > >> state
> > >> > > > machine
> > >> > > > > > to
> > >> > > > > > > do
> > >> > > > > > > > > > > > right). I
> > >> > > > > > > > > > > > > > > think
> > >> > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > should
> > >> > > > > > > > > > > > > > > > > > > > > > > > > consider
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > making
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > admin
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > commands
> > >> and
> > >> > > > > ideally
> > >> > > > > > > as
> > >> > > > > > > > > many
> > >> > > > > > > > > > > of
> > >> > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > other
> > >> > > > > > > > > > > > > > > > > > apis
> > >> > > > > > > > > > > > > > > > > > > > as
> > >> > > > > > > > > > > > > > > > > > > > > > > possible
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > available
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > on
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > all
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> brokers
> > >> and
> > >> > > just
> > >> > > > > > > > redirect
> > >> > > > > > > > > to
> > >> > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > controller
> > >> > > > > > > > > > > > > > > > > > on
> > >> > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > broker
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> side.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > there
> > >> would
> > >> > > be a
> > >> > > > > > > general
> > >> > > > > > > > > way
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > encapsulate
> > >> > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > re-routing
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > behavior.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> > It's a
> > >> > very
> > >> > > > > > > > interesting
> > >> > > > > > > > > > > idea,
> > >> > > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > > > seems
> > >> > > > > > > > > > > > > > > > > > there
> > >> > > > > > > > > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > > > > some
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > concerns
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > about
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> feature
> > >> > (like
> > >> > > > > > > > performance
> > >> > > > > > > > > > > > > > > > considerations,
> > >> > > > > > > > > > > > > > > > > > how
> > >> > > > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > complicate
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > server
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I
> > believe
> > >> > this
> > >> > > > > > > shouldn't
> > >> > > > > > > > > be
> > >> > > > > > > > > > a
> > >> > > > > > > > > > > > > > blocker.
> > >> > > > > > > > > > > > > > > > If
> > >> > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > feature is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > implemented
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > at
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > some
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point
> > it
> > >> > won't
> > >> > > > > > affect
> > >> > > > > > > > > Admin
> > >> > > > > > > > > > > > > changes
> > >> > > > > > > > > > > > > > -
> > >> > > > > > > > > > > > > > > at
> > >> > > > > > > > > > > > > > > > > > least
> > >> > > > > > > > > > > > > > > > > > > > no
> > >> > > > > > > > > > > > > > > > > > > > > > > changes
> > >> > > > > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > public
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > API
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will
> be
> > >> > > > required.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We
> > >> should
> > >> > > > > > probably
> > >> > > > > > > > > > > normalize
> > >> > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > key
> > >> > > > > > > > > > > > > > > > > > value
> > >> > > > > > > > > > > > > > > > > > > > > pairs
> > >> > > > > > > > > > > > > > > > > > > > > > > used
> > >> > > > > > > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > configs
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
> > >> > > embedding a
> > >> > > > > new
> > >> > > > > > > > > > > formatting.
> > >> > > > > > > > > > > > > So
> > >> > > > > > > > > > > > > > > two
> > >> > > > > > > > > > > > > > > > > > strings
> > >> > > > > > > > > > > > > > > > > > > > > > rather
> > >> > > > > > > > > > > > > > > > > > > > > > > > > than
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> one
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > with
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > an
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > internal
> > >> > > equals
> > >> > > > > > sign.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> > Fixed
> > >> in
> > >> > > the
> > >> > > > > > latest
> > >> > > > > > > > > > patch -
> > >> > > > > > > > > > > > > > > > normalized
> > >> > > > > > > > > > > > > > > > > > > > configs
> > >> > > > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> changed
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > protocol
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> accordingly.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is
> > the
> > >> > > > > > > postcondition
> > >> > > > > > > > of
> > >> > > > > > > > > > > these
> > >> > > > > > > > > > > > > > APIs
> > >> > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > command has
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> begun
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > or
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > >> command
> > >> > > has
> > >> > > > > been
> > >> > > > > > > > > > > completed?
> > >> > > > > > > > > > > > It
> > >> > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > > lot
> > >> > > > > > > > > > > > > > > > > > more
> > >> > > > > > > > > > > > > > > > > > > > > > > usable if
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > been
> > >> > completed
> > >> > > > so
> > >> > > > > > you
> > >> > > > > > > > know
> > >> > > > > > > > > > > that
> > >> > > > > > > > > > > > if
> > >> > > > > > > > > > > > > > you
> > >> > > > > > > > > > > > > > > > > > create a
> > >> > > > > > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> then
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > publish
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it
> you
> > >> won't
> > >> > > get
> > >> > > > > an
> > >> > > > > > > > > > exception
> > >> > > > > > > > > > > > > about
> > >> > > > > > > > > > > > > > > > there
> > >> > > > > > > > > > > > > > > > > > being
> > >> > > > > > > > > > > > > > > > > > > > no
> > >> > > > > > > > > > > > > > > > > > > > > > > such
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> topic.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> For
> > >> long
> > >> > > > > running
> > >> > > > > > > > > requests
> > >> > > > > > > > > > > > (like
> > >> > > > > > > > > > > > > > > > > reassign
> > >> > > > > > > > > > > > > > > > > > > > > > > partitions) -
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > post
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> condition is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> command
> > >> has
> > >> > > > begun
> > >> > > > > -
> > >> > > > > > so
> > >> > > > > > > > we
> > >> > > > > > > > > > > don't
> > >> > > > > > > > > > > > > > block
> > >> > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > client.
> > >> > > > > > > > > > > > > > > > > > > > > In
> > >> > > > > > > > > > > > > > > > > > > > > > > case
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> of your
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> example -
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
> > >> > > commands,
> > >> > > > > this
> > >> > > > > > > > will
> > >> > > > > > > > > be
> > >> > > > > > > > > > > > > > > refactored
> > >> > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > commands
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> will
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> executed
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> immediately,
> > >> > > > since
> > >> > > > > > the
> > >> > > > > > > > > > > > Controller
> > >> > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > serve
> > >> > > > > > > > > > > > > > > > > > > > Admin
> > >> > > > > > > > > > > > > > > > > > > > > > > > > requests
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> (follow-up
> > >> > > > ticket
> > >> > > > > > > > > > KAFKA-1777).
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 8.
> > >> Describe
> > >> > > > topic
> > >> > > > > > and
> > >> > > > > > > > list
> > >> > > > > > > > > > > > topics
> > >> > > > > > > > > > > > > > > > > duplicate
> > >> > > > > > > > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > > > > > lot
> > >> > > > > > > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > > > > > > stuff
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> in the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> metadata
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > request.
> > >> Is
> > >> > > > there
> > >> > > > > a
> > >> > > > > > > > reason
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > > give
> > >> > > > > > > > > > > > > > > back
> > >> > > > > > > > > > > > > > > > > > topics
> > >> > > > > > > > > > > > > > > > > > > > > > marked
> > >> > > > > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > deletion? I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > feel
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> if
> > >> we
> > >> > > just
> > >> > > > > make
> > >> > > > > > > the
> > >> > > > > > > > > > > > > > > post-condition
> > >> > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > delete
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> command be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
> > is
> > >> > > deleted
> > >> > > > > > that
> > >> > > > > > > > will
> > >> > > > > > > > > > get
> > >> > > > > > > > > > > > rid
> > >> > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > need
> > >> > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> right?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > And
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be
> much
> > >> more
> > >> > > > > > > intuitive.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> > Fixed
> > >> in
> > >> > > the
> > >> > > > > > latest
> > >> > > > > > > > > > patch -
> > >> > > > > > > > > > > > > > removed
> > >> > > > > > > > > > > > > > > > > > topics
> > >> > > > > > > > > > > > > > > > > > > > > marked
> > >> > > > > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> deletion
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > ListTopicsRequest.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 9.
> > >> Should we
> > >> > > > > > consider
> > >> > > > > > > > > > batching
> > >> > > > > > > > > > > > > these
> > >> > > > > > > > > > > > > > > > > > requests?
> > >> > > > > > > > > > > > > > > > > > > > We
> > >> > > > > > > > > > > > > > > > > > > > > > have
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> generally
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > tried
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > allow
> > >> > multiple
> > >> > > > > > > > operations
> > >> > > > > > > > > to
> > >> > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > batched.
> > >> > > > > > > > > > > > > > > > > My
> > >> > > > > > > > > > > > > > > > > > > > > > suspicion
> > >> > > > > > > > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > without
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we
> will
> > >> get
> > >> > a
> > >> > > > lot
> > >> > > > > of
> > >> > > > > > > > code
> > >> > > > > > > > > > that
> > >> > > > > > > > > > > > > does
> > >> > > > > > > > > > > > > > > > > > something
> > >> > > > > > > > > > > > > > > > > > > > like
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > for(topic:
> > >> > > > > > > > > > > > > > > adminClient.listTopics())
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > >  adminClient.describeTopic(topic)
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > code
> > >> > will
> > >> > > > > work
> > >> > > > > > > > great
> > >> > > > > > > > > > when
> > >> > > > > > > > > > > > you
> > >> > > > > > > > > > > > > > > test
> > >> > > > > > > > > > > > > > > > > on 5
> > >> > > > > > > > > > > > > > > > > > > > > topics
> > >> > > > > > > > > > > > > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > > > > > > > > > > > > not
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> do as
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > well
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > if
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > you
> > have
> > >> > 50k.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> > >> Updated
> > >> > the
> > >> > > > > KIP -
> > >> > > > > > > > > please
> > >> > > > > > > > > > > > check
> > >> > > > > > > > > > > > > > > "Topic
> > >> > > > > > > > > > > > > > > > > > Admin
> > >> > > > > > > > > > > > > > > > > > > > > > Schema"
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> section.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I
> > >> think
> > >> > we
> > >> > > > > > should
> > >> > > > > > > > also
> > >> > > > > > > > > > > > discuss
> > >> > > > > > > > > > > > > > how
> > >> > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > want
> > >> > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > expose a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> programmatic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> client
> > >> api
> > >> > for
> > >> > > > > these
> > >> > > > > > > > > > > operations.
> > >> > > > > > > > > > > > > > > > Currently
> > >> > > > > > > > > > > > > > > > > > > > people
> > >> > > > > > > > > > > > > > > > > > > > > > > rely on
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > which
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > is
> > >> totally
> > >> > > > > sketchy.
> > >> > > > > > I
> > >> > > > > > > > > think
> > >> > > > > > > > > > we
> > >> > > > > > > > > > > > > > > probably
> > >> > > > > > > > > > > > > > > > > need
> > >> > > > > > > > > > > > > > > > > > > > > another
> > >> > > > > > > > > > > > > > > > > > > > > > > > > client
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> under
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> clients/
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > that
> > >> exposes
> > >> > > > > > > > > administrative
> > >> > > > > > > > > > > > > > > > functionality.
> > >> > > > > > > > > > > > > > > > > > We
> > >> > > > > > > > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > > > > > > > need
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> this just
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > properly
> > >> > test
> > >> > > > the
> > >> > > > > > new
> > >> > > > > > > > > apis,
> > >> > > > > > > > > > I
> > >> > > > > > > > > > > > > > suspect.
> > >> > > > > > > > > > > > > > > > We
> > >> > > > > > > > > > > > > > > > > > should
> > >> > > > > > > > > > > > > > > > > > > > > > > figure
> > >> > > > > > > > > > > > > > > > > > > > > > > > > out
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > API.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> > >> Updated
> > >> > the
> > >> > > > > KIP -
> > >> > > > > > > > > please
> > >> > > > > > > > > > > > check
> > >> > > > > > > > > > > > > > > "Admin
> > >> > > > > > > > > > > > > > > > > > Client"
> > >> > > > > > > > > > > > > > > > > > > > > > > section
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> with an
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > initial
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > API
> > >> > proposal.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 11.
> The
> > >> > other
> > >> > > > > > > > information
> > >> > > > > > > > > > that
> > >> > > > > > > > > > > > > would
> > >> > > > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > > really
> > >> > > > > > > > > > > > > > > > > > > > > > useful
> > >> > > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > get
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > would
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> information
> > >> > > > about
> > >> > > > > > > > > > > > partitions--how
> > >> > > > > > > > > > > > > > much
> > >> > > > > > > > > > > > > > > > > data
> > >> > > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> partition,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > what
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > >> segment
> > >> > > > > offsets,
> > >> > > > > > > > what
> > >> > > > > > > > > is
> > >> > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > log-end
> > >> > > > > > > > > > > > > > > > > > offset
> > >> > > > > > > > > > > > > > > > > > > > > > (i.e.
> > >> > > > > > > > > > > > > > > > > > > > > > > last
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > offset),
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > what
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > >> > compaction
> > >> > > > > > point,
> > >> > > > > > > > > etc. I
> > >> > > > > > > > > > > > think
> > >> > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > done
> > >> > > > > > > > > > > > > > > > > > > > right
> > >> > > > > > > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> would be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> successor to
> > >> > > the
> > >> > > > > > very
> > >> > > > > > > > > > awkward
> > >> > > > > > > > > > > > > > > > > OffsetRequest
> > >> > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > have
> > >> > > > > > > > > > > > > > > > > > > > > > > today.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I
> > >> removed
> > >> > > > > > > > > > > > > > > ConsumerGroupOffsetsRequest
> > >> > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > latest
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> patch. I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > believe
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > >> should
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be
> > >> resolved
> > >> > > in a
> > >> > > > > > > > separate
> > >> > > > > > > > > > KIP
> > >> > > > > > > > > > > /
> > >> > > > > > > > > > > > > jira
> > >> > > > > > > > > > > > > > > > > ticket.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 12.
> > >> > Generally
> > >> > > we
> > >> > > > > can
> > >> > > > > > > do
> > >> > > > > > > > > good
> > >> > > > > > > > > > > > error
> > >> > > > > > > > > > > > > > > > > handling
> > >> > > > > > > > > > > > > > > > > > > > > without
> > >> > > > > > > > > > > > > > > > > > > > > > > > > needing
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > custom
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> server-side
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > messages.
> > >> > I.e.
> > >> > > > > > > generally
> > >> > > > > > > > > the
> > >> > > > > > > > > > > > > client
> > >> > > > > > > > > > > > > > > has
> > >> > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > context
> > >> > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > know
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > if
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > got
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > an
> > error
> > >> > that
> > >> > > > the
> > >> > > > > > > topic
> > >> > > > > > > > > > > doesn't
> > >> > > > > > > > > > > > > > exist
> > >> > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > say
> > >> > > > > > > > > > > > > > > > > > > > > "Topic
> > >> > > > > > > > > > > > > > > > > > > > > > X
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> doesn't
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > exist"
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
> > >> "error
> > >> > > code
> > >> > > > > 14"
> > >> > > > > > > (or
> > >> > > > > > > > > > > > > whatever).
> > >> > > > > > > > > > > > > > > > Maybe
> > >> > > > > > > > > > > > > > > > > > there
> > >> > > > > > > > > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > > > > > > specific
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > cases
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > where
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> is
> > >> > hard?
> > >> > > If
> > >> > > > > we
> > >> > > > > > > want
> > >> > > > > > > > > to
> > >> > > > > > > > > > > add
> > >> > > > > > > > > > > > > > > > > server-side
> > >> > > > > > > > > > > > > > > > > > > > error
> > >> > > > > > > > > > > > > > > > > > > > > > > messages
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > really
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > do
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > need
> to
> > >> do
> > >> > > this
> > >> > > > > in a
> > >> > > > > > > > > > > consistent
> > >> > > > > > > > > > > > > way
> > >> > > > > > > > > > > > > > > > across
> > >> > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > protocol.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> > >> Updated
> > >> > the
> > >> > > > > KIP -
> > >> > > > > > > > > please
> > >> > > > > > > > > > > > check
> > >> > > > > > > > > > > > > > > > > "Protocol
> > >> > > > > > > > > > > > > > > > > > > > > Errors"
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> section. I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > added
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > comprehensive,
> > >> > > > > > > > > fine-grained
> > >> > > > > > > > > > > list
> > >> > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > error
> > >> > > > > > > > > > > > > > > > > > codes.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > Comments
> > >> > from
> > >> > > > > > > Guozhang:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 13.
> > >> Describe
> > >> > > > topic
> > >> > > > > > > > > request:
> > >> > > > > > > > > > it
> > >> > > > > > > > > > > > > would
> > >> > > > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > > great to
> > >> > > > > > > > > > > > > > > > > > > > > go
> > >> > > > > > > > > > > > > > > > > > > > > > > beyond
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > batching
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
> > >> name
> > >> > > regex
> > >> > > > > for
> > >> > > > > > > > this
> > >> > > > > > > > > > > > request.
> > >> > > > > > > > > > > > > > For
> > >> > > > > > > > > > > > > > > > > > example,
> > >> > > > > > > > > > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > > > > > > > very
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> common use
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > case
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > topic
> > >> > > > command
> > >> > > > > is
> > >> > > > > > > to
> > >> > > > > > > > > list
> > >> > > > > > > > > > > all
> > >> > > > > > > > > > > > > > > topics
> > >> > > > > > > > > > > > > > > > > > whose
> > >> > > > > > > > > > > > > > > > > > > > > config
> > >> > > > > > > > > > > > > > > > > > > > > > > A's
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> value is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > B.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > With
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
> > >> name
> > >> > > regex
> > >> > > > > > then
> > >> > > > > > > we
> > >> > > > > > > > > > have
> > >> > > > > > > > > > > to
> > >> > > > > > > > > > > > > > first
> > >> > > > > > > > > > > > > > > > > > retrieve
> > >> > > > > > > > > > > > > > > > > > > > > > > __all__
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> topics's
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> description
> > >> > > info
> > >> > > > > and
> > >> > > > > > > > then
> > >> > > > > > > > > > > filter
> > >> > > > > > > > > > > > > at
> > >> > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > client
> > >> > > > > > > > > > > > > > > > > > > > > end,
> > >> > > > > > > > > > > > > > > > > > > > > > > which
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> will
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > huge
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> burden
> > on
> > >> > ZK.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 14.
> > >> Config
> > >> > > K-Vs
> > >> > > > in
> > >> > > > > > > > create
> > >> > > > > > > > > > > topic:
> > >> > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > related
> > >> > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> previous
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > point;
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > maybe
> > we
> > >> can
> > >> > > add
> > >> > > > > > > another
> > >> > > > > > > > > > > > metadata
> > >> > > > > > > > > > > > > > K-V
> > >> > > > > > > > > > > > > > > or
> > >> > > > > > > > > > > > > > > > > > just a
> > >> > > > > > > > > > > > > > > > > > > > > > > metadata
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> string
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > along
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > side
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > with
> > >> config
> > >> > > K-V
> > >> > > > in
> > >> > > > > > > > create
> > >> > > > > > > > > > > topic
> > >> > > > > > > > > > > > > like
> > >> > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > did
> > >> > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > offset
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> commit
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > request.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > This
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > field
> > >> can be
> > >> > > > quite
> > >> > > > > > > > useful
> > >> > > > > > > > > in
> > >> > > > > > > > > > > > > storing
> > >> > > > > > > > > > > > > > > > > > information
> > >> > > > > > > > > > > > > > > > > > > > > > like
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> "owner" of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > who
> > issue
> > >> > the
> > >> > > > > create
> > >> > > > > > > > > > command,
> > >> > > > > > > > > > > > etc,
> > >> > > > > > > > > > > > > > > which
> > >> > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > quite
> > >> > > > > > > > > > > > > > > > > > > > > > > > > important
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> for a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> multi-tenant
> > >> > > > > > setting.
> > >> > > > > > > > Then
> > >> > > > > > > > > > in
> > >> > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > describe
> > >> > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > request
> > >> > > > > > > > > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > also
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > batch
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > on
> > regex
> > >> of
> > >> > > the
> > >> > > > > > > metadata
> > >> > > > > > > > > > > field.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: As
> > >> > > discussed
> > >> > > > it
> > >> > > > > > is
> > >> > > > > > > > very
> > >> > > > > > > > > > > > > > interesting
> > >> > > > > > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > > > > > can
> > >> > > > > > > > > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> implemented
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > later
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > after
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we
> have
> > >> some
> > >> > > > basic
> > >> > > > > > > > > > > functionality
> > >> > > > > > > > > > > > > > > there.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 15.
> > Today
> > >> > all
> > >> > > > the
> > >> > > > > > > admin
> > >> > > > > > > > > > > > operations
> > >> > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > async in
> > >> > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > sense
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> return
> > >> once
> > >> > it
> > >> > > > is
> > >> > > > > > > > written
> > >> > > > > > > > > in
> > >> > > > > > > > > > > ZK,
> > >> > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > is why
> > >> > > > > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > need
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> extra
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> verification
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > >> > > > > > > > > > > > > testUtil.waitForTopicCreated()
> > >> > > > > > > > > > > > > > /
> > >> > > > > > > > > > > > > > > > > verify
> > >> > > > > > > > > > > > > > > > > > > > > > partition
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reassignment
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > request,
> > >> > etc.
> > >> > > > With
> > >> > > > > > > admin
> > >> > > > > > > > > > > > requests
> > >> > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > could
> > >> > > > > > > > > > > > > > > > > > add a
> > >> > > > > > > > > > > > > > > > > > > > > > flag
> > >> > > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> enable /
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > disable
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> synchronous
> > >> > > > > > requests;
> > >> > > > > > > > when
> > >> > > > > > > > > > it
> > >> > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > turned
> > >> > > > > > > > > > > > > > > > > on,
> > >> > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > response
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> will not
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > return
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > until
> > the
> > >> > > > request
> > >> > > > > > has
> > >> > > > > > > > been
> > >> > > > > > > > > > > > > > completed.
> > >> > > > > > > > > > > > > > > > And
> > >> > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > async
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> requests we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > can
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > add a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> "token"
> > >> > field
> > >> > > in
> > >> > > > > the
> > >> > > > > > > > > > response,
> > >> > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > then
> > >> > > > > > > > > > > > > > > > > only
> > >> > > > > > > > > > > > > > > > > > > > need
> > >> > > > > > > > > > > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > general
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > "admin
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> verification
> > >> > > > > > request"
> > >> > > > > > > > with
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > > given
> > >> > > > > > > > > > > > > > > > token
> > >> > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > check
> > >> > > > > > > > > > > > > > > > > > > > > > > if the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> async
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > request
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > has
> > been
> > >> > > > > completed.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I
> > see
> > >> > your
> > >> > > > > point.
> > >> > > > > > > My
> > >> > > > > > > > > idea
> > >> > > > > > > > > > > was
> > >> > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > provide
> > >> > > > > > > > > > > > > > > > > > > > > specific
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > Verify...Request
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > per
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > each
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long
> > >> running
> > >> > > > > > request,
> > >> > > > > > > > > where
> > >> > > > > > > > > > > > > needed.
> > >> > > > > > > > > > > > > > We
> > >> > > > > > > > > > > > > > > > can
> > >> > > > > > > > > > > > > > > > > > do it
> > >> > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > way
> > >> > > > > > > > > > > > > > > > > > > > > > > > > you
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > suggest.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > The
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > only
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> concern
> > >> is
> > >> > > that
> > >> > > > > > > > > introducing
> > >> > > > > > > > > > a
> > >> > > > > > > > > > > > > token
> > >> > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > again
> > >> > > > > > > > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > > > > > > make
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> schema
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> "dynamic".
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > We
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> wanted
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to do
> > >> > similar
> > >> > > > > thing
> > >> > > > > > > > > > > introducing
> > >> > > > > > > > > > > > > > single
> > >> > > > > > > > > > > > > > > > > > > > > AdminRequest
> > >> > > > > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > all
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> commands
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > but
> > >> rejected
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > idea
> > >> > > > because
> > >> > > > > we
> > >> > > > > > > > > wanted
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > > > have
> > >> > > > > > > > > > > > > > > > schema
> > >> > > > > > > > > > > > > > > > > > > > > defined.
> > >> > > > > > > > > > > > > > > > > > > > > > So
> > >> > > > > > > > > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > more a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> choice
> > >> > > between:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > a)
> have
> > >> > fixed
> > >> > > > > schema
> > >> > > > > > > but
> > >> > > > > > > > > > > > introduce
> > >> > > > > > > > > > > > > > > each
> > >> > > > > > > > > > > > > > > > > > time new
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> Verify...Request
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> long-running
> > >> > > > > > requests
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > b)
> use
> > >> one
> > >> > > > request
> > >> > > > > > for
> > >> > > > > > > > > > > > > verification
> > >> > > > > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > > > > > > > generalize
> > >> > > > > > > > > > > > > > > > > > > > > > it
> > >> > > > > > > > > > > > > > > > > > > > > > > with
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> token
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I'm
> > fine
> > >> > with
> > >> > > > > > whatever
> > >> > > > > > > > > > > decision
> > >> > > > > > > > > > > > > > > > community
> > >> > > > > > > > > > > > > > > > > > come
> > >> > > > > > > > > > > > > > > > > > > > to.
> > >> > > > > > > > > > > > > > > > > > > > > > > Just
> > >> > > > > > > > > > > > > > > > > > > > > > > > > let
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> me
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > know
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > your
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > thoughts.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> Comment
> > >> from
> > >> > > > Gwen:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 16.
> > >> > > Specifically
> > >> > > > > for
> > >> > > > > > > > > > > ownership,
> > >> > > > > > > > > > > > I
> > >> > > > > > > > > > > > > > > think
> > >> > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > plan
> > >> > > > > > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > to add
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> ACL
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sounds
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > you
> > >> are
> > >> > > > > > > describing
> > >> > > > > > > > > ACL)
> > >> > > > > > > > > > > via
> > >> > > > > > > > > > > > > an
> > >> > > > > > > > > > > > > > > > > external
> > >> > > > > > > > > > > > > > > > > > > > > system
> > >> > > > > > > > > > > > > > > > > > > > > > > > > (Argus,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Sentry).
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I
> > >> remember
> > >> > > > KIP-11
> > >> > > > > > > > > described
> > >> > > > > > > > > > > > this,
> > >> > > > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > > I
> > >> > > > > > > > > > > > > > > > > > can't
> > >> > > > > > > > > > > > > > > > > > > > find
> > >> > > > > > > > > > > > > > > > > > > > > > > the KIP
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> any
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > longer.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> > Okay,
> > >> no
> > >> > > > > problem.
> > >> > > > > > > Not
> > >> > > > > > > > > > sure
> > >> > > > > > > > > > > > > though
> > >> > > > > > > > > > > > > > > how
> > >> > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > > > going
> > >> > > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> handle
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > it.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Wait
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > which
> > KIP
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will
> be
> > >> > > > committed
> > >> > > > > > > first
> > >> > > > > > > > > and
> > >> > > > > > > > > > > > > include
> > >> > > > > > > > > > > > > > > > > changes
> > >> > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadata from
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > later
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > Anyway, I
> > >> > > added
> > >> > > > > this
> > >> > > > > > > > note
> > >> > > > > > > > > to
> > >> > > > > > > > > > > > "Open
> > >> > > > > > > > > > > > > > > > > > Questions"
> > >> > > > > > > > > > > > > > > > > > > > > > section
> > >> > > > > > > > > > > > > > > > > > > > > > > so
> > >> > > > > > > > > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> don't
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > miss
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> piece.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> Andrii
> > >> > > Biletskyi
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > On
> Fri,
> > >> Mar
> > >> > > 13,
> > >> > > > > 2015
> > >> > > > > > > at
> > >> > > > > > > > > > 12:34
> > >> > > > > > > > > > > > AM,
> > >> > > > > > > > > > > > > > > Andrii
> > >> > > > > > > > > > > > > > > > > > > > > Biletskyi <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > andrii.biletskyi@stealth.ly
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi
> > all,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > Today I
> > >> > > > uploaded
> > >> > > > > > the
> > >> > > > > > > > > patch
> > >> > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > covers
> > >> > > > > > > > > > > > > > > > > > some of
> > >> > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> discussed
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > agreed
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > items:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> > >> removed
> > >> > > > > MaybeOf
> > >> > > > > > > > > optional
> > >> > > > > > > > > > > > type
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> > >> switched
> > >> > > to
> > >> > > > > java
> > >> > > > > > > > > > protocol
> > >> > > > > > > > > > > > > > > > definitions
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> > >> > simplified
> > >> > > > > > > messages
> > >> > > > > > > > > > > > > (normalized
> > >> > > > > > > > > > > > > > > > > configs,
> > >> > > > > > > > > > > > > > > > > > > > > removed
> > >> > > > > > > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> marked
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > >> deletion)
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I
> > also
> > >> > > updated
> > >> > > > > the
> > >> > > > > > > > KIP-4
> > >> > > > > > > > > > > with
> > >> > > > > > > > > > > > > > > > respective
> > >> > > > > > > > > > > > > > > > > > > > changes
> > >> > > > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> wrote down
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > my
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > >> proposal
> > >> > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > pending
> > >> > > items:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> > Batch
> > >> > > Admin
> > >> > > > > > > > Operations
> > >> > > > > > > > > > ->
> > >> > > > > > > > > > > > > > updated
> > >> > > > > > > > > > > > > > > > Wire
> > >> > > > > > > > > > > > > > > > > > > > > Protocol
> > >> > > > > > > > > > > > > > > > > > > > > > > schema
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > proposal
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> > >> Remove
> > >> > > > > > > > ClusterMetadata
> > >> > > > > > > > > > ->
> > >> > > > > > > > > > > > > > changed
> > >> > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > extend
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > >> TopicMetadataRequest
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> > Admin
> > >> > > Client
> > >> > > > > ->
> > >> > > > > > > > > updated
> > >> > > > > > > > > > my
> > >> > > > > > > > > > > > > > initial
> > >> > > > > > > > > > > > > > > > > > proposal
> > >> > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > reflect
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > batching
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> > Error
> > >> > > codes
> > >> > > > ->
> > >> > > > > > > > > proposed
> > >> > > > > > > > > > > > > > > fine-grained
> > >> > > > > > > > > > > > > > > > > > error
> > >> > > > > > > > > > > > > > > > > > > > > code
> > >> > > > > > > > > > > > > > > > > > > > > > > > > instead
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > >> > > > > AdminRequestFailed
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I
> > will
> > >> > also
> > >> > > > > send a
> > >> > > > > > > > > > separate
> > >> > > > > > > > > > > > > email
> > >> > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > cover all
> > >> > > > > > > > > > > > > > > > > > > > > > > comments
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> from
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > Thanks,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > Andrii
> > >> > > > Biletskyi
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > On
> > Thu,
> > >> > Mar
> > >> > > > 12,
> > >> > > > > > 2015
> > >> > > > > > > > at
> > >> > > > > > > > > > 9:26
> > >> > > > > > > > > > > > PM,
> > >> > > > > > > > > > > > > > > Gwen
> > >> > > > > > > > > > > > > > > > > > Shapira
> > >> > > > > > > > > > > > > > > > > > > > <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > gshapira@cloudera.com
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > Found
> > >> > > KIP-11
> > >> > > > (
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > >
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> It
> > >> > actually
> > >> > > > > > > specifies
> > >> > > > > > > > > > > changes
> > >> > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > Metadata
> > >> > > > > > > > > > > > > > > > > > > > > > > protocol,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> so
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > making
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sure
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> both
> > >> KIPs
> > >> > > are
> > >> > > > > > > > > consistent
> > >> > > > > > > > > > in
> > >> > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > regard
> > >> > > > > > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > good.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> On
> > >> Thu,
> > >> > Mar
> > >> > > > 12,
> > >> > > > > > > 2015
> > >> > > > > > > > at
> > >> > > > > > > > > > > 12:21
> > >> > > > > > > > > > > > > PM,
> > >> > > > > > > > > > > > > > > > Gwen
> > >> > > > > > > > > > > > > > > > > > > > Shapira
> > >> > > > > > > > > > > > > > > > > > > > > <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > gshapira@cloudera.com
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > >> > > > Specifically
> > >> > > > > > for
> > >> > > > > > > > > > > > ownership, I
> > >> > > > > > > > > > > > > > > think
> > >> > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > plan
> > >> > > > > > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > add
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> ACL
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > like
> > >> > you
> > >> > > > are
> > >> > > > > > > > > describing
> > >> > > > > > > > > > > > ACL)
> > >> > > > > > > > > > > > > > via
> > >> > > > > > > > > > > > > > > an
> > >> > > > > > > > > > > > > > > > > > > > external
> > >> > > > > > > > > > > > > > > > > > > > > > > system
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> (Argus,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> I
> > >> > > remember
> > >> > > > > > KIP-11
> > >> > > > > > > > > > > described
> > >> > > > > > > > > > > > > > this,
> > >> > > > > > > > > > > > > > > > > but I
> > >> > > > > > > > > > > > > > > > > > > > can't
> > >> > > > > > > > > > > > > > > > > > > > > > > find
> > >> > > > > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> KIP
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > any
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > >> > > > Regardless, I
> > >> > > > > > > think
> > >> > > > > > > > > > KIP-4
> > >> > > > > > > > > > > > > > focuses
> > >> > > > > > > > > > > > > > > > on
> > >> > > > > > > > > > > > > > > > > > > > getting
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> information
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > already
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > >> exists
> > >> > > from
> > >> > > > > > Kafka
> > >> > > > > > > > > > > brokers,
> > >> > > > > > > > > > > > > not
> > >> > > > > > > > > > > > > > on
> > >> > > > > > > > > > > > > > > > > > adding
> > >> > > > > > > > > > > > > > > > > > > > > > > information
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > perhaps
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > >> should
> > >> > > > exist
> > >> > > > > > but
> > >> > > > > > > > > > doesn't
> > >> > > > > > > > > > > > yet?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > Gwen
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> On
> > >> Thu,
> > >> > > Mar
> > >> > > > > 12,
> > >> > > > > > > > 2015
> > >> > > > > > > > > at
> > >> > > > > > > > > > > > 6:37
> > >> > > > > > > > > > > > > > AM,
> > >> > > > > > > > > > > > > > > > > > Guozhang
> > >> > > > > > > > > > > > > > > > > > > > > Wang
> > >> > > > > > > > > > > > > > > > > > > > > > <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > wangguoz@gmail.com>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> Folks,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> Just
> > >> > > want
> > >> > > > to
> > >> > > > > > > > > > elaborate a
> > >> > > > > > > > > > > > bit
> > >> > > > > > > > > > > > > > > more
> > >> > > > > > > > > > > > > > > > on
> > >> > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > create-topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > metadata
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> batching
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > describe-topic
> > >> > > > > > > > based
> > >> > > > > > > > > > on
> > >> > > > > > > > > > > > > > config /
> > >> > > > > > > > > > > > > > > > > > metadata
> > >> > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > my
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> previous
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > email
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > as
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> work
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > on
> > >> > > > > KAFKA-1694.
> > >> > > > > > > The
> > >> > > > > > > > > > main
> > >> > > > > > > > > > > > > > > motivation
> > >> > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > have
> > >> > > > > > > > > > > > > > > > > > > > > > > some
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> sort of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > management
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > mechanisms,
> > >> > > > > > > which
> > >> > > > > > > > I
> > >> > > > > > > > > > > think
> > >> > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > quite
> > >> > > > > > > > > > > > > > > > > > > > important
> > >> > > > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > /
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > architecture:
> > >> > > > > > > > today
> > >> > > > > > > > > > > anyone
> > >> > > > > > > > > > > > > can
> > >> > > > > > > > > > > > > > > > > create
> > >> > > > > > > > > > > > > > > > > > > > topics
> > >> > > > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> shared
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Kafka
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> cluster,
> > >> > > but
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> there
> > >> > is
> > >> > > > no
> > >> > > > > > > > concept
> > >> > > > > > > > > or
> > >> > > > > > > > > > > > > > > "ownership"
> > >> > > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > > > topics
> > >> > > > > > > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > created
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > by
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> different
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> users.
> > >> > > For
> > >> > > > > > > > example,
> > >> > > > > > > > > at
> > >> > > > > > > > > > > > > > LinkedIn
> > >> > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > basically
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> distinguish
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > owners
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> via
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> some
> > >> > > > casual
> > >> > > > > > > topic
> > >> > > > > > > > > name
> > >> > > > > > > > > > > > > prefix,
> > >> > > > > > > > > > > > > > > > which
> > >> > > > > > > > > > > > > > > > > > is a
> > >> > > > > > > > > > > > > > > > > > > > > bit
> > >> > > > > > > > > > > > > > > > > > > > > > > > > awkward
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > does
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > we
> > >> > scale
> > >> > > > our
> > >> > > > > > > > > > customers.
> > >> > > > > > > > > > > It
> > >> > > > > > > > > > > > > > would
> > >> > > > > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > > great
> > >> > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > use
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > describe-topics
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > such
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> as:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > Describe
> > >> > > > all
> > >> > > > > > > > topics
> > >> > > > > > > > > > that
> > >> > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > created
> > >> > > > > > > > > > > > > > > > > > by me.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > Describe
> > >> > > > all
> > >> > > > > > > > topics
> > >> > > > > > > > > > > whose
> > >> > > > > > > > > > > > > > > > retention
> > >> > > > > > > > > > > > > > > > > > time
> > >> > > > > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > overriden
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> to X.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > Describe
> > >> > > > all
> > >> > > > > > > > topics
> > >> > > > > > > > > > > whose
> > >> > > > > > > > > > > > > > > writable
> > >> > > > > > > > > > > > > > > > > > group
> > >> > > > > > > > > > > > > > > > > > > > > > include
> > >> > > > > > > > > > > > > > > > > > > > > > > > > user
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> Y
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> related
> > >> > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > > authorization),
> > >> > > > > > > > > etc..
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > One
> > >> > > > possible
> > >> > > > > > way
> > >> > > > > > > > to
> > >> > > > > > > > > > > > achieve
> > >> > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > add a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > metadata
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> file
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > create-topic
> > >> > > > > > > > > request,
> > >> > > > > > > > > > > > whose
> > >> > > > > > > > > > > > > > > value
> > >> > > > > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > > > > also
> > >> > > > > > > > > > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> written ZK
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > as
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> create
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> topic;
> > >> > > > then
> > >> > > > > > > > > > > > describe-topics
> > >> > > > > > > > > > > > > > can
> > >> > > > > > > > > > > > > > > > > > choose to
> > >> > > > > > > > > > > > > > > > > > > > > > batch
> > >> > > > > > > > > > > > > > > > > > > > > > > > > topics
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > based
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > on
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 1)
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > name
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> regex,
> > >> > > 2)
> > >> > > > > > config
> > >> > > > > > > > K-V
> > >> > > > > > > > > > > > > matching,
> > >> > > > > > > > > > > > > > > 3)
> > >> > > > > > > > > > > > > > > > > > metadata
> > >> > > > > > > > > > > > > > > > > > > > > > > regex,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > etc.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > Thoughts?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > Guozhang
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > On
> > >> > Thu,
> > >> > > > Mar
> > >> > > > > 5,
> > >> > > > > > > > 2015
> > >> > > > > > > > > at
> > >> > > > > > > > > > > > 4:37
> > >> > > > > > > > > > > > > > PM,
> > >> > > > > > > > > > > > > > > > > > Guozhang
> > >> > > > > > > > > > > > > > > > > > > > > Wang
> > >> > > > > > > > > > > > > > > > > > > > > > <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > wangguoz@gmail.com>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > Thanks
> > >> > > > for
> > >> > > > > > the
> > >> > > > > > > > > > updated
> > >> > > > > > > > > > > > > wiki.
> > >> > > > > > > > > > > > > > A
> > >> > > > > > > > > > > > > > > > few
> > >> > > > > > > > > > > > > > > > > > > > comments
> > >> > > > > > > > > > > > > > > > > > > > > > > below:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > 1.
> > >> > > Error
> > >> > > > > > > > > description
> > >> > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > response: I
> > >> > > > > > > > > > > > > > > > > > think
> > >> > > > > > > > > > > > > > > > > > > > > if
> > >> > > > > > > > > > > > > > > > > > > > > > > some
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > errorCode
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> indicate
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > several
> > >> > > > > > > different
> > >> > > > > > > > > > error
> > >> > > > > > > > > > > > > cases
> > >> > > > > > > > > > > > > > > > then
> > >> > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > should
> > >> > > > > > > > > > > > > > > > > > > > > > > really
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> change
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> multiple
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > codes.
> > >> > > In
> > >> > > > > > > general
> > >> > > > > > > > > the
> > >> > > > > > > > > > > > > > errorCode
> > >> > > > > > > > > > > > > > > > > > itself
> > >> > > > > > > > > > > > > > > > > > > > > would
> > >> > > > > > > > > > > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> precise
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > sufficient
> > >> > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > describing
> > >> > > > > > the
> > >> > > > > > > > > server
> > >> > > > > > > > > > > > side
> > >> > > > > > > > > > > > > > > > errors.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > 2.
> > >> > > > Describe
> > >> > > > > > > topic
> > >> > > > > > > > > > > > request:
> > >> > > > > > > > > > > > > it
> > >> > > > > > > > > > > > > > > > would
> > >> > > > > > > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > > > > > > great
> > >> > > > > > > > > > > > > > > > > > > > > > > to go
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> beyond
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > just
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> batching
> > >> > on
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> topic
> > >> > > > name
> > >> > > > > > > regex
> > >> > > > > > > > > for
> > >> > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > request.
> > >> > > > > > > > > > > > > > > > > > For
> > >> > > > > > > > > > > > > > > > > > > > > > > example, a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> very
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > common
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > use
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> case
> > >> of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> the
> > >> > > topic
> > >> > > > > > > command
> > >> > > > > > > > > is
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > > > list
> > >> > > > > > > > > > > > > > > all
> > >> > > > > > > > > > > > > > > > > > topics
> > >> > > > > > > > > > > > > > > > > > > > > whose
> > >> > > > > > > > > > > > > > > > > > > > > > > > > config
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> A's
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > value
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > B.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> With
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> topic
> > >> > > > name
> > >> > > > > > > regex
> > >> > > > > > > > > then
> > >> > > > > > > > > > > we
> > >> > > > > > > > > > > > > have
> > >> > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > first
> > >> > > > > > > > > > > > > > > > > > > > > > retrieve
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> __all__
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topics's
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > description
> > >> > > > > > > info
> > >> > > > > > > > > and
> > >> > > > > > > > > > > then
> > >> > > > > > > > > > > > > > > filter
> > >> > > > > > > > > > > > > > > > at
> > >> > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > client
> > >> > > > > > > > > > > > > > > > > > > > > > > end,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> which
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> huge
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > burden
> > >> > > on
> > >> > > > > ZK.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > 3.
> > >> > > Config
> > >> > > > > > K-Vs
> > >> > > > > > > in
> > >> > > > > > > > > > > create
> > >> > > > > > > > > > > > > > topic:
> > >> > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > related to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > previous
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> point;
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> maybe
> > >> > > we
> > >> > > > > can
> > >> > > > > > > add
> > >> > > > > > > > > > > another
> > >> > > > > > > > > > > > > > > metadata
> > >> > > > > > > > > > > > > > > > > > K-V or
> > >> > > > > > > > > > > > > > > > > > > > > > just a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> metadata
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > string
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > along
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> side
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> with
> > >> > > > config
> > >> > > > > > K-V
> > >> > > > > > > > in
> > >> > > > > > > > > > > create
> > >> > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > like
> > >> > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > did
> > >> > > > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> offset
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > commit
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> request.
> > >> > > This
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> field
> > >> > > can
> > >> > > > > be
> > >> > > > > > > > quite
> > >> > > > > > > > > > > useful
> > >> > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > storing
> > >> > > > > > > > > > > > > > > > > > > > > > > information
> > >> > > > > > > > > > > > > > > > > > > > > > > > > like
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > "owner"
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> who
> > >> > > issue
> > >> > > > > the
> > >> > > > > > > > > create
> > >> > > > > > > > > > > > > command,
> > >> > > > > > > > > > > > > > > > etc,
> > >> > > > > > > > > > > > > > > > > > which
> > >> > > > > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > quite
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > important
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > > multi-tenant
> > >> > > > > > > > > setting.
> > >> > > > > > > > > > > > Then
> > >> > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > describe
> > >> > > > > > > > > > > > > > > > > > > > > > > topic
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> request
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > can
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > also
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > batch
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > on
> > >> > > regex
> > >> > > > of
> > >> > > > > > the
> > >> > > > > > > > > > > metadata
> > >> > > > > > > > > > > > > > field.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > 4.
> > >> > > Today
> > >> > > > > all
> > >> > > > > > > the
> > >> > > > > > > > > > admin
> > >> > > > > > > > > > > > > > > operations
> > >> > > > > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > async
> > >> > > > > > > > > > > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> sense
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> command
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> will
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > return
> > >> > > > once
> > >> > > > > > it
> > >> > > > > > > is
> > >> > > > > > > > > > > written
> > >> > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > ZK,
> > >> > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > why we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> need
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > extra
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > > verification
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> like
> > >> > > > > > > > > > > > > > > > > testUtil.waitForTopicCreated() /
> > >> > > > > > > > > > > > > > > > > > > > > verify
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> partition
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> reassignment
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > request,
> > >> > > > > etc.
> > >> > > > > > > > With
> > >> > > > > > > > > > > admin
> > >> > > > > > > > > > > > > > > requests
> > >> > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > could
> > >> > > > > > > > > > > > > > > > > > > > > > add
> > >> > > > > > > > > > > > > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> flag to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > enable
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > /
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> disable
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > synchronous
> > >> > > > > > > > > requests;
> > >> > > > > > > > > > > > when
> > >> > > > > > > > > > > > > it
> > >> > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > turned
> > >> > > > > > > > > > > > > > > > > > > > on,
> > >> > > > > > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> response
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > return
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> until
> > >> > > the
> > >> > > > > > > request
> > >> > > > > > > > > has
> > >> > > > > > > > > > > > been
> > >> > > > > > > > > > > > > > > > > > completed. And
> > >> > > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > async
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > requests
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> add
> > a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > "token"
> > >> > > > > field
> > >> > > > > > > in
> > >> > > > > > > > > the
> > >> > > > > > > > > > > > > > response,
> > >> > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > then
> > >> > > > > > > > > > > > > > > > > > > > > only
> > >> > > > > > > > > > > > > > > > > > > > > > > need a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > general
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > "admin
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > > verification
> > >> > > > > > > > > request"
> > >> > > > > > > > > > > > with
> > >> > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > given
> > >> > > > > > > > > > > > > > > > > > > > token
> > >> > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > check
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> if the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > async
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> request
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> has
> > >> > > been
> > >> > > > > > > > completed.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> 5. +1
> > >> > > for
> > >> > > > > > > > extending
> > >> > > > > > > > > > > > > Metadata
> > >> > > > > > > > > > > > > > > > > request
> > >> > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > include
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > controller /
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > coordinator
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > > information,
> > >> > > > > > > and
> > >> > > > > > > > > then
> > >> > > > > > > > > > > we
> > >> > > > > > > > > > > > > can
> > >> > > > > > > > > > > > > > > > remove
> > >> > > > > > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > > > ClusterMetadata
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > requests.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > Guozhang
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > On
> > >> > Tue,
> > >> > > > Mar
> > >> > > > > > 3,
> > >> > > > > > > > 2015
> > >> > > > > > > > > > at
> > >> > > > > > > > > > > > > 10:23
> > >> > > > > > > > > > > > > > > AM,
> > >> > > > > > > > > > > > > > > > > Joel
> > >> > > > > > > > > > > > > > > > > > > > > Koshy <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > jjkoshy.w@gmail.com
> > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > Thanks
> > >> > > > for
> > >> > > > > > > > sending
> > >> > > > > > > > > > > that
> > >> > > > > > > > > > > > > out
> > >> > > > > > > > > > > > > > > Joe
> > >> > > > > > > > > > > > > > > > -
> > >> > > > > > > > > > > > > > > > > I
> > >> > > > > > > > > > > > > > > > > > > > don't
> > >> > > > > > > > > > > > > > > > > > > > > > > think I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> will be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > able
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> make
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> it
> > >> > > > today,
> > >> > > > > so
> > >> > > > > > > if
> > >> > > > > > > > > > notes
> > >> > > > > > > > > > > > can
> > >> > > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > > sent
> > >> > > > > > > > > > > > > > > > > > out
> > >> > > > > > > > > > > > > > > > > > > > > > > afterward
> > >> > > > > > > > > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > would
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> great.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> On
> > >> > > Mon,
> > >> > > > > Mar
> > >> > > > > > > 02,
> > >> > > > > > > > > 2015
> > >> > > > > > > > > > > at
> > >> > > > > > > > > > > > > > > > 09:16:13AM
> > >> > > > > > > > > > > > > > > > > > > > -0800,
> > >> > > > > > > > > > > > > > > > > > > > > > Gwen
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> Shapira
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > > Thanks
> > >> > > > > for
> > >> > > > > > > > > sending
> > >> > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > out
> > >> > > > > > > > > > > > > > > > Joe.
> > >> > > > > > > > > > > > > > > > > > > > Looking
> > >> > > > > > > > > > > > > > > > > > > > > > > forward
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > chatting
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > with
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > > everyone
> > >> > > > > :)
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > On
> > >> > > > Mon,
> > >> > > > > > Mar
> > >> > > > > > > 2,
> > >> > > > > > > > > > 2015
> > >> > > > > > > > > > > at
> > >> > > > > > > > > > > > > > 6:46
> > >> > > > > > > > > > > > > > > > AM,
> > >> > > > > > > > > > > > > > > > > > Joe
> > >> > > > > > > > > > > > > > > > > > > > > Stein
> > >> > > > > > > > > > > > > > > > > > > > > > <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > joe.stein@stealth.ly>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > > > Hey, I
> > >> > > > > > > just
> > >> > > > > > > > > sent
> > >> > > > > > > > > > > > out a
> > >> > > > > > > > > > > > > > > > google
> > >> > > > > > > > > > > > > > > > > > > > hangout
> > >> > > > > > > > > > > > > > > > > > > > > > > invite
> > >> > > > > > > > > > > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> all
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > pmc,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > committers
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > > > > > everyone I
> > >> > > > > > > > > found
> > >> > > > > > > > > > > > > working
> > >> > > > > > > > > > > > > > > on
> > >> > > > > > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > > > KIP.
> > >> > > > > > > > > > > > > > > > > > > > If
> > >> > > > > > > > > > > > > > > > > > > > > I
> > >> > > > > > > > > > > > > > > > > > > > > > > missed
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> anyone
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > invite
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > please
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > > let
> > >> > > > me
> > >> > > > > > > know
> > >> > > > > > > > > and
> > >> > > > > > > > > > > can
> > >> > > > > > > > > > > > > > update
> > >> > > > > > > > > > > > > > > > it,
> > >> > > > > > > > > > > > > > > > > > np.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > We
> > >> > > > > > should
> > >> > > > > > > do
> > >> > > > > > > > > > this
> > >> > > > > > > > > > > > > every
> > >> > > > > > > > > > > > > > > > > Tuesday
> > >> > > > > > > > > > > > > > > > > > @
> > >> > > > > > > > > > > > > > > > > > > > 2pm
> > >> > > > > > > > > > > > > > > > > > > > > > > Eastern
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> Time.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Maybe
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> get
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > INFRA
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > > help
> > >> > > > > to
> > >> > > > > > > > make a
> > >> > > > > > > > > > > > google
> > >> > > > > > > > > > > > > > > > account
> > >> > > > > > > > > > > > > > > > > > so we
> > >> > > > > > > > > > > > > > > > > > > > > can
> > >> > > > > > > > > > > > > > > > > > > > > > > manage
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > better?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > To
> > >> > > > > > discuss
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > >
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > in
> > >> > > > > > > progress
> > >> > > > > > > > > and
> > >> > > > > > > > > > > > > related
> > >> > > > > > > > > > > > > > > JIRA
> > >> > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > are
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> interdependent
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > common
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > work.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > ~
> > >> > > > Joe
> > >> > > > > > > Stein
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > On
> > >> > > > > Tue,
> > >> > > > > > > Feb
> > >> > > > > > > > > 24,
> > >> > > > > > > > > > > 2015
> > >> > > > > > > > > > > > > at
> > >> > > > > > > > > > > > > > > 2:59
> > >> > > > > > > > > > > > > > > > > > PM, Jay
> > >> > > > > > > > > > > > > > > > > > > > > > > Kreps <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > >> > > > > jay.kreps@gmail.com
> > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > Let's
> > >> > > > > > > stay
> > >> > > > > > > > on
> > >> > > > > > > > > > > > Google
> > >> > > > > > > > > > > > > > > > hangouts
> > >> > > > > > > > > > > > > > > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > will
> > >> > > > > > > > > > > > > > > > > > > > > > > also
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> record
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > make
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > > sessions
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > > > available
> > >> > > > > > > > on
> > >> > > > > > > > > > > > youtube.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > -Jay
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > On
> > >> > > > > Tue,
> > >> > > > > > > Feb
> > >> > > > > > > > > 24,
> > >> > > > > > > > > > > > 2015
> > >> > > > > > > > > > > > > at
> > >> > > > > > > > > > > > > > > > 11:49
> > >> > > > > > > > > > > > > > > > > > AM,
> > >> > > > > > > > > > > > > > > > > > > > > Jeff
> > >> > > > > > > > > > > > > > > > > > > > > > > > > Holoman
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > > > > > > > > > jholoman@cloudera.com
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > Jay /
> > >> > > > > > > Joe
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > We're
> > >> > > > > > > > happy
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > > send
> > >> > > > > > > > > > > > > > > out a
> > >> > > > > > > > > > > > > > > > > > Webex
> > >> > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > this
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> purpose.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > We
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > record
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> the
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > > > sessions
> > >> > > > > > > > if
> > >> > > > > > > > > > > there
> > >> > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > interest and
> > >> > > > > > > > > > > > > > > > > > > > > > > publish
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> them
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > out.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > > Thanks
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > Jeff
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > On
> > >> > > > > > Tue,
> > >> > > > > > > > Feb
> > >> > > > > > > > > > 24,
> > >> > > > > > > > > > > > > 2015
> > >> > > > > > > > > > > > > > at
> > >> > > > > > > > > > > > > > > > > > 11:28 AM,
> > >> > > > > > > > > > > > > > > > > > > > > Jay
> > >> > > > > > > > > > > > > > > > > > > > > > > > > Kreps <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > > > > > > jay.kreps@gmail.com>
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > Let's
> > >> > > > > > > > try
> > >> > > > > > > > > > to
> > >> > > > > > > > > > > > get
> > >> > > > > > > > > > > > > > the
> > >> > > > > > > > > > > > > > > > > > technical
> > >> > > > > > > > > > > > > > > > > > > > > > > hang-ups
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> sorted
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > out,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> though.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > really
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > think
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > there
> > >> > > > > > > > is
> > >> > > > > > > > > > some
> > >> > > > > > > > > > > > > > benefit
> > >> > > > > > > > > > > > > > > > to
> > >> > > > > > > > > > > > > > > > > > live
> > >> > > > > > > > > > > > > > > > > > > > > > > discussion
> > >> > > > > > > > > > > > > > > > > > > > > > > > > vs
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > writing. I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > am
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > > hopeful
> > >> > > > > that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > if
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > we
> > >> > > > > > > post
> > >> > > > > > > > > > > > > > instructions
> > >> > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > give
> > >> > > > > > > > > > > > > > > > > > > > > > > ourselves a
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> few
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > attempts
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> can
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> get
> > >> > it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > > > working.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > > Tuesday
> > >> > > > > > > > > at
> > >> > > > > > > > > > > that
> > >> > > > > > > > > > > > > > time
> > >> > > > > > > > > > > > > > > > > would
> > >> > > > > > > > > > > > > > > > > > work
> > >> > > > > > > > > > > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> me...any
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> objections?
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > -Jay
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > On
> > >> > > > > > > Tue,
> > >> > > > > > > > > Feb
> > >> > > > > > > > > > > 24,
> > >> > > > > > > > > > > > > > 2015
> > >> > > > > > > > > > > > > > > at
> > >> > > > > > > > > > > > > > > > > > 8:18
> > >> > > > > > > > > > > > > > > > > > > > AM,
> > >> > > > > > > > > > > > > > > > > > > > > > Joe
> > >> > > > > > > > > > > > > > > > > > > > > > > > > Stein
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > > > > > > joe.stein@stealth.ly
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > > Weekly
> > >> > > > > > > > > > > would
> > >> > > > > > > > > > > > be
> > >> > > > > > > > > > > > > > > great
> > >> > > > > > > > > > > > > > > > > > maybe
> > >> > > > > > > > > > > > > > > > > > > > > like
> > >> > > > > > > > > > > > > > > > > > > > > > > every
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 1pm
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > ET
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > /
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> 10am
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> PT
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > ????
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > I
> > >> > > > > > > > don't
> > >> > > > > > > > > > > mind
> > >> > > > > > > > > > > > > > google
> > >> > > > > > > > > > > > > > > > > > hangout
> > >> > > > > > > > > > > > > > > > > > > > but
> > >> > > > > > > > > > > > > > > > > > > > > > > there
> > >> > > > > > > > > > > > > > > > > > > > > > > > > is
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > always
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > some
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > issue
> > >> or
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > > > whatever
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > so
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > we
> > >> > > > > > > > know
> > >> > > > > > > > > > the
> > >> > > > > > > > > > > > > > apache
> > >> > > > > > > > > > > > > > > > irc
> > >> > > > > > > > > > > > > > > > > > > > channel
> > >> > > > > > > > > > > > > > > > > > > > > > > works.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > We
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > start
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > there
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> see
> > >> > > how
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > goes?
> > >> > > > > > > > > We
> > >> > > > > > > > > > > can
> > >> > > > > > > > > > > > > pull
> > >> > > > > > > > > > > > > > > > > > transcripts
> > >> > > > > > > > > > > > > > > > > > > > > too
> > >> > > > > > > > > > > > > > > > > > > > > > > and
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > associate
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> tickets
> > >> > if
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> need
> > >> > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > makes
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > it
> > >> > > > > > > > > > helpful
> > >> > > > > > > > > > > > for
> > >> > > > > > > > > > > > > > > > things.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > ~
> > >> > > > > > > > > > Joestein
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > On
> > >> > > > > > > > Tue,
> > >> > > > > > > > > > Feb
> > >> > > > > > > > > > > > 24,
> > >> > > > > > > > > > > > > > > 2015
> > >> > > > > > > > > > > > > > > > at
> > >> > > > > > > > > > > > > > > > > > 11:10
> > >> > > > > > > > > > > > > > > > > > > > > AM,
> > >> > > > > > > > > > > > > > > > > > > > > > > Jay
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> Kreps <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > > > > > > > > jay.kreps@gmail.com
> > >> > > > > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > We'd
> > >> > > > > > > > > > > talked
> > >> > > > > > > > > > > > > > about
> > >> > > > > > > > > > > > > > > > > > doing a
> > >> > > > > > > > > > > > > > > > > > > > > > Google
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> Hangout to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > chat
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > about
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > this.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> What
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > about
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > > > > > generalizing
> > >> > > > > > > > > > > > > > > that a
> > >> > > > > > > > > > > > > > > > > > little
> > >> > > > > > > > > > > > > > > > > > > > > > > > > further...I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > actually
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > think
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > would
> > >> > > be
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > good
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > for
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > > > everyone
> > >> > > > > > > > > > > > > > > spending a
> > >> > > > > > > > > > > > > > > > > > > > > reasonable
> > >> > > > > > > > > > > > > > > > > > > > > > > chunk
> > >> > > > > > > > > > > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > their
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > week
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > Kafka
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > stuff
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > maybe
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > sync
> > >> > > > > > > > > up
> > >> > > > > > > > > > > > once
> > >> > > > > > > > > > > > > a
> > >> > > > > > > > > > > > > > > > week.
> > >> > > > > > > > > > > > > > > > > I
> > >> > > > > > > > > > > > > > > > > > > > think
> > >> > > > > > > > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > > > > > > could
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> use
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > time
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > talk
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > > through
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > > design
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > > stuff,
> > >> > > > > > > > > > > make
> > >> > > > > > > > > > > > > > sure
> > >> > > > > > > > > > > > > > > we
> > >> > > > > > > > > > > > > > > > > > are on
> > >> > > > > > > > > > > > > > > > > > > > > top
> > >> > > > > > > > > > > > > > > > > > > > > > of
> > >> > > > > > > > > > > > > > > > > > > > > > > > > code
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > reviews,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > talk
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> through
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> any
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > > tricky
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > > issues,
> > >> > > > > > > > > > > > etc.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > We
> > >> > > > > > > > > can
> > >> > > > > > > > > > > make
> > >> > > > > > > > > > > > > it
> > >> > > > > > > > > > > > > > > > > publicly
> > >> > > > > > > > > > > > > > > > > > > > > > > available so
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > any
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > one
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > follow
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > along
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > who
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > > likes.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > Any
> > >> > > > > > > > > > > > interest
> > >> > > > > > > > > > > > > in
> > >> > > > > > > > > > > > > > > > doing
> > >> > > > > > > > > > > > > > > > > > this?
> > >> > > > > > > > > > > > > > > > > > > > > If
> > >> > > > > > > > > > > > > > > > > > > > > > so
> > >> > > > > > > > > > > > > > > > > > > > > > > > > I'll
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> try
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > set
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > it
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > up
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > > starting
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > next
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > week.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > -Jay
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > On
> > >> > > > > > > > > Tue,
> > >> > > > > > > > > > > Feb
> > >> > > > > > > > > > > > > 24,
> > >> > > > > > > > > > > > > > > > 2015
> > >> > > > > > > > > > > > > > > > > at
> > >> > > > > > > > > > > > > > > > > > > > 3:57
> > >> > > > > > > > > > > > > > > > > > > > > > AM,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > Andrii
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > <
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > >> > > > > > > > > > > > > > > > > > > > > > > wrote:
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > >
> > >> > > > > > > > Hi
> > >> > > > > > > > > > all,
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > >
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > >
> > >> > > > > > > > > I've
> > >> > > > > > > > > > > > > updated
> > >> > > > > > > > > > > > > > > KIP
> > >> > > > > > > > > > > > > > > > > > page,
> > >> > > > > > > > > > > > > > > > > > > > > fixed
> > >> > > > > > > > > > > > > > > > > > > > > > /
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> aligned
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > document
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >> > structure.
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > Also I
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>>
> > >> > >>
> > >> > > > >
> > >> > > > > > added
> > >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > >>>> >
> > >> > >> >
> > >> > > > > >
> > >> > > > > > > >
> > >> > > > > > > > > some
> > >> > > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > ...
> > >> > > > > > > > > > > > > > >
> > >> > > > > > > > > > > > > > > [Message clipped]
> > >> > > > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > > > --
> > >> > > > > > > > > > > > > -- Guozhang
> > >> > > > > > > > > > > > >
> > >> > > > > > > > > > > >
> > >> > > > > > > > > > >
> > >> > > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > --
> > >> > > > > > > -- Guozhang
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >> >
> > >> >
> > >> > --
> > >> > Thanks,
> > >> > Neha
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Hi all,

Let me list remaining issues for tomorrow call.

210. ListTopicsRequest - do we need it?
I agreed with Jun, it looks reasonable people would want
to have lightweight request to get all topics. Also this may
be related with the next item.

201. We need to include topic-level configuration somewhere -
DescribeTopicRequest vs TopicMetadataRequest_V1.
During the last call, I think, we agreed to go with TMR_V1.
But now, when I have looked a bit at this approach I see some difficulties.
Currently the TopicMetadataResponse is essentially
org.apache.kafka.common.Cluster.
We use Cluster instance to hold TopicMetadata in network client.
Now, if we evolve TMR to version V1 to include topic-level configs
it will break Cluster model (once we migrate clients). Also taking into
account MetadataRequest is used in NetworkClient
(= Producer & Consumer) are we okay with receiving some amount
of irrelevant data (topic-level configs) which has nothing to do
with cluster metadata? Can this affect clients performance?
I think this should be discussed separately again.

Also, I believe we are now more or less close to agree on Admin RQ/RP
schema. Can we also talk about AdminClient API which is a part of
public interfaces for these changes? I updated KIP to reflect current
Wire Protocol schema, but I think we need to look carefully at it,
especially
the way we will handle async semantics of the Admin requests.

Thanks,
Andrii Biletskyi



On Fri, Apr 3, 2015 at 8:14 PM, Jun Rao <ju...@confluent.io> wrote:

> There is one more item that we need to discuss.
>
> 210. Do we need a separate ListTopic request that just returns a list of
> all topics. Currently, we piggyback this functionality in TMR. If the topic
> list is empty in TMR, we return all topics and their metadata. The tradeoff
> is the following. If we piggyback on TMR, there is one fewer request that
> we have to deal with. On the other hand, it does mean that the response
> size can be large (e.g., if you have 10K topics and a 1KB metadata per
> topic, the response can be 10MB). There are probably cases (e.g., in
> supporting wildcard topics in the new java consumer) that a client just
> needs to see the full topic list, but not the metadata for every topic.
>
> Thanks,
>
> Jun
>
> On Tue, Mar 24, 2015 at 2:01 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > 201. I think it's probably easier to consolidate on DTR. If we evolve
> TMR,
> > the producer will be using the latest version of TMR. However, the latest
> > version of TMR won't trigger auto topic creation. This means that we need
> > to change the producer to use CreateTopicRequest. If we consolidate on
> DTR.
> > We can fix the producer logic later.
> >
> > 205. Also, there was a remaining item on how to get the controller info.
> > We can either add the controller id to the response of DTR or create a
> new
> > cluster metadata request/response. The cluster meta response will then
> > include the detailed information about every broker and the controller. I
> > think the latter is probably better. For example, when we add the
> > multi-port support (for security), the cluster meta response can include
> > all ports for a broker. In DTR, we only need to return one broker port to
> > the client.
> >
> > Thanks,
> >
> > Jun
> >
> > On Tue, Mar 24, 2015 at 12:26 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> >> Hi all,
> >>
> >> A short summary of our discussion:
> >>
> >> 201. DescribeTopicRequest(DTR) vs TopicMetadataRequest(TMR)
> >>
> >> *Issue*: Topic description should include topic-level configuration,
> >> other information is available in TMR - leader, ISR, AR.
> >> So now there are two options - either add full-fledged new DTR
> >> that will include TMR + all needed information (which will be used then
> >> instead TMR) or extend TMR to include topic-level configuration.
> >>
> >> *Resolution*: it was decided to vote on it: either a) *extend TMR* or b)
> >> *add
> >> DTR*.
> >> I'd go with a) - we will have to roll out TMR_V1 but it's better than
> >> having
> >> "dead" requests if we add DTR and leave TMR unused in future versions
> >>
> >> 202. CreateTopicRequest sync/async semantics
> >>
> >> *Issue*: What should be a post-condition of Create/Alter/Delete
> >> requests - the command has been completed or command has been
> >> initiated?
> >> Even though it looks reasonable that users would typically expect to
> >> have topic created upon CreateTopicRequest the corner case is that
> >> topic can be deleted right after creation, so received
> CreateTopicResponse
> >> cannot be a 100% guarantee topic is created and available, even if
> >> CreateTopicRequest is synchronous. Also, making request blocking on
> >> server may be a hard thing to do right now (check Jun's email from Mar
> >> 21).
> >>
> >> *Resolution*: for now it makes sense to start with async semantics for
> >> Create/Alter/Delete.
> >>
> >> 203. VerifyReassignPartition vs leverage DescribeTopicRequest
> >>
> >> *Issue*: DescribeTopicRequest (or TopicMetadaRequest) should be enough,
> >> since controller doesn't remove partitions from
> admin/reassign_partitions
> >> until
> >> the assignment actually matches target assignment so partition
> >> reassignment
> >> cannot fail for specific partition in the current implementation.
> >> Everything else
> >> ("Completed" and "In Progress" state) can be checked with assigned
> >> partitions
> >> field from DescribeTopicRequest.
> >>
> >> *Resolution*: remove VerifyReassignPartitionsRequest
> >>
> >> 204. Also it was discussed that after we finalize question 201 and port
> >> java protocol definition (KAFKA-1927) I will split patch to separate
> >> pieces,
> >> first will cover Wire Protocol changes, since we have a common vision on
> >> it.
> >>
> >> Please let me know your thoughts on q. 201!
> >>
> >> Thanks,
> >> Andrii Biletskyi
> >>
> >>
> >>
> >>
> >> On Tue, Mar 24, 2015 at 7:57 PM, Neha Narkhede <ne...@confluent.io>
> wrote:
> >>
> >> > I'd like to resurface the discussion of sync vs async topic creation.
> >> > Possibly, we can discuss that in today's KIP meeting. I am worried
> about
> >> > pushing the burden of handling async topic creation on the clients.
> >> This is
> >> > one of the most frequently asked questions on the mailing list and I
> >> hope
> >> > we find a way to address it as we separate the metadata request and
> >> topic
> >> > creation. Since topic creation doesn't take that long, there should
> be a
> >> > really easy way for clients to express their preference for a sync
> topic
> >> > creation request where the receipt of a response suggests completion
> of
> >> > topic creation.
> >> >
> >> > The behavior that I'm expecting is that if I'm a client and I choose
> to
> >> > create a topic synchronously, after I receive a response, my first
> send
> >> > request for that topic should *not* fail with some error.
> >> >
> >> > On Tue, Mar 24, 2015 at 8:02 AM, Jun Rao <ju...@confluent.io> wrote:
> >> >
> >> > > Andrii,
> >> > >
> >> > >
> >> > > 111.4 Yes, we can discuss this in the meeting today.
> >> > >
> >> > > 112. The question is when will a PartitionReassignment fail.
> >> Currently,
> >> > it
> >> > > can fail if the input is incorrect (e.g, invalid partition/replica).
> >> > > However, this can be detected when the ReassignPartitionRequest is
> >> > issued.
> >> > > Once the process of partition reassignment is started, it will not
> >> fail.
> >> > > The controller just waits until the process completes. We can
> improve
> >> the
> >> > > status reporting of partition reassignment. However, I am not sure
> if
> >> > that
> >> > > needs to be tied to this KIP.
> >> > >
> >> > > 115.1 Yes, that makes sense. I misunderstood it. I thought
> >> > > PreferredReplicaLeaderElectionInProgress and
> >> ReassignPartitionsInProgress
> >> > > are used in the status check.
> >> > >
> >> > > Thanks,
> >> > >
> >> > > Jun
> >> > >
> >> > >
> >> > > On Tue, Mar 24, 2015 at 3:46 AM, Andrii Biletskyi <
> >> > > andrii.biletskyi@stealth.ly> wrote:
> >> > >
> >> > > > Jun,
> >> > > >
> >> > > > Thanks for such an accurate review!
> >> > > >
> >> > > > Most of your remarks I didn't fix in the previous change because
> >> > > > I thought we'd finalize them today, during the call. Anyway, I've
> >> > > > fixed them now.
> >> > > > My comments to ensure I covered everything:
> >> > > >
> >> > > > 110. Fixed. Removed global error code.
> >> > > >
> >> > > > 111.1-3. Removed redundant TopicName, Partitions and
> >> ReplicationFactor
> >> > > > 111.4 Yes, now we are in the situation when TopicMetadataResponse
> >> > > > contains everything we need except topic level configs. I'm
> >> > > > okay with both solutions, probably makes sense to discuss and pick
> >> > > > the preferable one.
> >> > > >
> >> > > > 112. Need to think about it a bit more. Currently we may
> distinguish
> >> > > > 3 states of specific partition being reassigned:
> >> > > > a) Completed - absent in /reassign_partitions zk path and
> >> > DescribeTopic/
> >> > > > TopicMetadata shows expected assignment
> >> > > > 2) In progress - present in /reassign_partitions
> >> > > > 3) Failed - absent in /reassign_partitions but
> >> > > DescribeTopic/TopicMetadata
> >> > > > replicas field doesn't correspond expected
> >> > > > Now, if use only DescribeTopic to check reassignment status - how
> >> can
> >> > we
> >> > > > distinguish states "In Progress" and "Failed"?
> >> > > > Also, there are some concerns that current reassignment status
> >> provided
> >> > > > by ReaasignPartitionsCommand (and the same was intended in KIP-4)
> is
> >> > > > very spurious, non-informative. Probably it makes sense to add
> >> > additional
> >> > > > data there (thus separate request may be useful), I'm considering
> >> > > different
> >> > > > options now - btw, comments here are highly appreciated!
> >> > > >
> >> > > > 113-114. Okay, made it compliant with topic commands. About "The
> >> error
> >> > > code
> >> > > > will
> >> > > > then be per topic" - just a note: currently
> >> (ReassignPartitionsCommand)
> >> > > > validation
> >> > > > is done for the whole reassignment string, so in case of errors
> for
> >> > > > specific
> >> > > > partitions the reassignment is NOT started at all. Both options
> look
> >> > fine
> >> > > > for me,
> >> > > > it's just the logic will be slightly changed.
> >> > > >
> >> > > > 115.1 Removed NotControllerReceivedAdminCommand. But why should we
> >> > > > remove PreferredReplicaLeaderElectionInProgress and
> >> > > > ReassignPartitionsInProgress?
> >> > > > On Reassign/Preferred..Request we create admin zk path, we can of
> >> > course
> >> > > > update the path, but wouldn't it be safer if we simply refuse to
> >> start
> >> > > new
> >> > > > reassignment in case one is in progress?
> >> > > >
> >> > > > 115.2 Yes, paragraph Protocol Errors covers only errors that are
> >> > proposed
> >> > > > to be added. I didn't want to list all errors that are to be added
> >> > since
> >> > > > anyway
> >> > > > the final version will be clear after the development is
> completed.
> >> > > >
> >> > > > 100. Fixed, normalized json string in Create/Alter.
> >> > > >
> >> > > > Thanks,
> >> > > > Andrii Biletskyi
> >> > > >
> >> > > > On Tue, Mar 24, 2015 at 12:58 AM, Jun Rao <ju...@confluent.io>
> wrote:
> >> > > >
> >> > > > > Andrii,
> >> > > > >
> >> > > > > I looked at the latest wiki for this KIP. I have a few more
> >> comments.
> >> > > > >
> >> > > > > 110. In CreateTopicResponse, AlterTopicResponse,
> >> DeleteTopicResponse
> >> > > and
> >> > > > > DescribeTopicResponse, we probably don't need the global error
> >> code.
> >> > > Our
> >> > > > > current convention is to just use the same global error code in
> >> each
> >> > > > topic.
> >> > > > > This simplifies the error checking on the client side.
> >> > > > >
> >> > > > > 111. DescribeTopicResponse:
> >> > > > > 111.1 Our protocol definition doesn't support optional fields.
> >> So, we
> >> > > > can't
> >> > > > > make Leader an optional field.
> >> > > > > 111.2 Do we really need the field Partitions and
> >> ReplicationFactor in
> >> > > > > TopicConfigDetails?
> >> > > > > The former is basically the size of the TopicPartitionDetails
> >> array
> >> > and
> >> > > > the
> >> > > > > latter can be found from the size of the Replica array.
> >> > > > > 111.3 Do we need TopicName in TopicDescription since it's
> already
> >> in
> >> > > > > DescribeTopicResponse?
> >> > > > > 111.4 Finally, DescribeTopicResponse is very similar to
> >> > > > > TopicMetadataResponse. So, it probably makes sense just to keep
> >> one
> >> > of
> >> > > > them
> >> > > > > in the future. Should we just use DescribeTopicRequest/Response
> to
> >> > > > replace
> >> > > > > TopicMetadataRequest/Response in the producer/consumer client in
> >> the
> >> > > > future
> >> > > > > (since DescribeTopicRequest doesn't trigger auto topic
> creation)?
> >> If
> >> > > so,
> >> > > > we
> >> > > > > will need to add the broker list in DescribeTopicResponse.
> >> > > > >
> >> > > > > 112. Thinking about this a bit more, I don't think we need a
> >> separate
> >> > > > > VerifyReassignPartitionRequest/Response.
> >> > > > > We can just use DescribeTopic to get the assigned replicas and
> >> check
> >> > if
> >> > > > > they match the target replica assignment in the client. The
> >> > controller
> >> > > > > propagates the metadata change after the reassignment completes
> >> for
> >> > > each
> >> > > > > partition.
> >> > > > >
> >> > > > > 113. ReassignPartitionRequest: For consistency, we probably want
> >> to
> >> > > nest
> >> > > > > the partition data under topic. So instead of
> >> > > > >   ReassignPartitionRequest => [Topic PartitionId [ReplicaId]]
> >> > > > > we can have sth like
> >> > > > >   ReassignPartitionRequest => [Topic [PartitionId [ReplicaId]]]
> >> > > > > The error code will then be per topic.
> >> > > > >
> >> > > > > 114. PreferredReplicaLeaderElectionRequest: Same as the above.
> >> > Instead
> >> > > of
> >> > > > >   PreferredReplicaLeaderElectionRequest => [Topic PartitionId]
> >> > > > > we can have
> >> > > > >   PreferredReplicaLeaderElectionRequest => [Topic [PartitionId]]
> >> > > > > Again, the error code will be per topic.
> >> > > > >
> >> > > > > 115. ErrorCode:
> >> > > > > 115.1 NotControllerReceivedAdminRequest,
> >> > > > > PreferredReplicaLeaderElectionInProgress
> >> > > > > and ReassignPartitionsInProgress are likely not needed any more.
> >> > > > > 115.2 We probably need to add UnknownTopicOrPartitionCode since
> >> it's
> >> > a
> >> > > > > valid error code for some of the requests (e.g. describe, alter,
> >> etc)
> >> > > > when
> >> > > > > a non-existing topic is specified.
> >> > > > >
> >> > > > >
> >> > > > > Previous comments.
> >> > > > > 100. ReplicaAssignment in CreateTopicRequest and
> AlterTopicRequest
> >> > are
> >> > > > > still a json string. We need to flatten them.
> >> > > > >
> >> > > > >
> >> > > > > Thanks,
> >> > > > >
> >> > > > > Jun
> >> > > > >
> >> > > > >
> >> > > > > On Sat, Mar 21, 2015 at 2:03 AM, Andrii Biletskyi <
> >> > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > >
> >> > > > > > Guozhang,
> >> > > > > >
> >> > > > > > I'm not sure I understand how can we use those tools in CLI.
> >> > > > > > First of all, those are *Test*Utils, e.g.
> >> > > waitUntilMetadataIsPropagated
> >> > > > > > (which might be very useful for us) requires
> >> > servers:Seq[KafkaServer]
> >> > > > > > as an argument:
> >> > > > > >
> >> > > > > > def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer],
> >> topic:
> >> > > > > String,
> >> > > > > > partition: Int, timeout: Long = 5000L): Int
> >> > > > > >
> >> > > > > > how can we have access it at runtime in one of the brokers?
> >> > > > > >
> >> > > > > > Secondly, how can user directly call these tools if zookeeper
> >> might
> >> > > be
> >> > > > > > not accessible (which is used in the tools, right?) at all -
> >> e.g.
> >> > > > behind
> >> > > > > > the VPC
> >> > > > > > in AWS. Furthermore, I think with KIP-4 we are trying to
> create
> >> an
> >> > > > > > abstraction and
> >> > > > > > a single point of zookeeper interactions, to eliminate direct
> >> calls
> >> > > to
> >> > > > ZK
> >> > > > > > either to get
> >> > > > > > some cluster information or to change something.
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > > Andrii Biletskyi
> >> > > > > >
> >> > > > > > On Sat, Mar 21, 2015 at 3:37 AM, Guozhang Wang <
> >> wangguoz@gmail.com
> >> > >
> >> > > > > wrote:
> >> > > > > >
> >> > > > > > > Andrii,
> >> > > > > > >
> >> > > > > > > Actually the checking logic Jun mentioned is already
> >> implemented
> >> > as
> >> > > > > > > TestUtils.waitUntilXXX (LeaderIsElected,
> MetadataIsPropagated,
> >> > > > etc...)
> >> > > > > I
> >> > > > > > > think we can extend these functions as CLI tools like
> >> > TopicCommand
> >> > > so
> >> > > > > > that
> >> > > > > > > users re-implementing such endpoint can directly call
> >> something
> >> > > like
> >> > > > > > > java.tools.WaitUntilXXX (of course this requires them to
> have
> >> > javac
> >> > > > > > > installed, which should be a reasonable requirement?)
> >> > > > > > >
> >> > > > > > > Guozhang
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
> >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > >
> >> > > > > > > > Jun,
> >> > > > > > > >
> >> > > > > > > > Not that I was saying we need to make requests blocking on
> >> > > server,
> >> > > > > > > > it was just to emphasize that with async requests a client
> >> > > > > > > implementations
> >> > > > > > > > may be a little bit more than just issue request - get the
> >> > > > response.
> >> > > > > > > > Thanks for the explanation, I understand now that we can
> go
> >> > with
> >> > > > > agreed
> >> > > > > > > > solution though it may not be perfect.
> >> > > > > > > > I believe this was one of the last controversial questions
> >> from
> >> > > the
> >> > > > > > list.
> >> > > > > > > >
> >> > > > > > > > Thanks,
> >> > > > > > > > Andrii Biletskyi
> >> > > > > > > >
> >> > > > > > > > On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <
> jun@confluent.io
> >> >
> >> > > > wrote:
> >> > > > > > > >
> >> > > > > > > > > Andrii,
> >> > > > > > > > >
> >> > > > > > > > > A few points.
> >> > > > > > > > >
> >> > > > > > > > > 1. Create/Alter can typically complete quickly. So, it's
> >> > > possible
> >> > > > > to
> >> > > > > > > make
> >> > > > > > > > > the request block until it's completed. However,
> >> currently,
> >> > > doing
> >> > > > > > this
> >> > > > > > > at
> >> > > > > > > > > the broker is a bit involved. To make Create block, we
> >> will
> >> > > need
> >> > > > to
> >> > > > > > add
> >> > > > > > > > > some callbacks in KafkaController. This is possible.
> >> However,
> >> > > the
> >> > > > > > > > > controller logic currently is pretty completed. It would
> >> > > probably
> >> > > > > be
> >> > > > > > > > better
> >> > > > > > > > > if we clean it up first before adding more complexity to
> >> it.
> >> > > > Alter
> >> > > > > is
> >> > > > > > > > even
> >> > > > > > > > > trickier. Adding partition is currently handled through
> >> > > > > > > KafkaController.
> >> > > > > > > > So
> >> > > > > > > > > it can be dealt with in a similar way. However, Alter
> >> config
> >> > is
> >> > > > > done
> >> > > > > > > > > completely differently. It doesn't go through the
> >> controller.
> >> > > > > > Instead,
> >> > > > > > > > each
> >> > > > > > > > > broker listens to ZooKeeper directly. So, it's not clear
> >> if
> >> > > there
> >> > > > > is
> >> > > > > > an
> >> > > > > > > > > easy way on the broker to figure out whether a config is
> >> > > applied
> >> > > > on
> >> > > > > > > every
> >> > > > > > > > > broker.
> >> > > > > > > > >
> >> > > > > > > > > 2. Delete can potentially take long if a replica to be
> >> > deleted
> >> > > is
> >> > > > > > > > offline.
> >> > > > > > > > > PreferredLeader/PartitionReassign can also take long.
> So,
> >> we
> >> > > > can't
> >> > > > > > > really
> >> > > > > > > > > make those requests block on the broker.
> >> > > > > > > > >
> >> > > > > > > > > As you can see, at this moment it's not easy to make all
> >> > admin
> >> > > > > > requests
> >> > > > > > > > > block on the broker. So, if we want the blocking feature
> >> in
> >> > the
> >> > > > > admin
> >> > > > > > > > > utility in the short term, doing the completion check at
> >> the
> >> > > > admin
> >> > > > > > > client
> >> > > > > > > > > is probably an easier route, even though it may not be
> >> ideal.
> >> > > > > > > > >
> >> > > > > > > > > Thanks,
> >> > > > > > > > >
> >> > > > > > > > > Jun
> >> > > > > > > > >
> >> > > > > > > > > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> >> > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > > > >
> >> > > > > > > > > > Jun,
> >> > > > > > > > > >
> >> > > > > > > > > > I see your point. But wouldn't that lead to a "fat"
> >> client
> >> > > > > > > > > implementations?
> >> > > > > > > > > > Suppose someone would like to implement client for
> Admin
> >> > Wire
> >> > > > > > > protocol.
> >> > > > > > > > > > Not only people will have to code quite complicated
> >> logic
> >> > > like
> >> > > > > > "send
> >> > > > > > > > > > describe
> >> > > > > > > > > > request to each broker" (again state machin?) but it
> >> will
> >> > > also
> >> > > > > mean
> >> > > > > > > > > people
> >> > > > > > > > > > must understand internal kafka logic related to topic
> >> > storage
> >> > > > and
> >> > > > > > how
> >> > > > > > > > > > information is propageted from the controller to
> >> brokers.
> >> > > > > > > > > > I see this like a dilemma between having a concise
> Wire
> >> > > > Protocol
> >> > > > > > and
> >> > > > > > > > > > self-sufficient API to make client implementations
> >> simple.
> >> > > > > > > > > > I don't have a win-win solution though.
> >> > > > > > > > > >
> >> > > > > > > > > > Thanks,
> >> > > > > > > > > > Andrii Biletskyi
> >> > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <
> >> > jun@confluent.io>
> >> > > > > > wrote:
> >> > > > > > > > > >
> >> > > > > > > > > > > For 1), 2) and 3), blocking would probably mean that
> >> the
> >> > > new
> >> > > > > > > metadata
> >> > > > > > > > > is
> >> > > > > > > > > > > propagated to every broker. To achieve that, the
> >> client
> >> > can
> >> > > > > keep
> >> > > > > > > > > issuing
> >> > > > > > > > > > > the describe topic request to every broker until it
> >> sees
> >> > > the
> >> > > > > new
> >> > > > > > > > > metadata
> >> > > > > > > > > > > in the response.
> >> > > > > > > > > > >
> >> > > > > > > > > > > Thanks,
> >> > > > > > > > > > >
> >> > > > > > > > > > > Jun
> >> > > > > > > > > > >
> >> > > > > > > > > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> >> > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > > > > > >
> >> > > > > > > > > > > > Hm, actually the ticket you linked, Guozhang,
> >> brings as
> >> > > > back
> >> > > > > > > > > > > > to the problem what should be considered a
> >> > post-condition
> >> > > > for
> >> > > > > > > > > > > > each of the admin commands.
> >> > > > > > > > > > > > In my understanding:
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > 1) CreateTopic - broker created
> >> /brokers/topics/<topic>
> >> > > > > > > > > > > > (Not the controller picked up changes from zk and
> >> > > > broadcasted
> >> > > > > > > > > > > > LeaderAndIsr and UpdateMetadata)
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > 2) AlterTopic - same as 1) - broker changed
> >> assignment
> >> > > data
> >> > > > > > > > > > > > in zookeeper or created admin path for topic
> config
> >> > > change
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > 3) DeleteTopic - admin path /admin/delete_topics
> is
> >> > > created
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > 4) ReassignPartitions and PreferredReplica -
> >> > > corresponding
> >> > > > > > admin
> >> > > > > > > > > > > > path is created
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Now what can be considered a completed operation
> >> from
> >> > the
> >> > > > > > > client's
> >> > > > > > > > > > > > perspective?
> >> > > > > > > > > > > > 1) Topic is created once corresponding data is in
> zk
> >> > > > > > > > > > > > (I remember there were some thoughts that it'd be
> >> good
> >> > to
> >> > > > > > > consider
> >> > > > > > > > > > > > topic created once all replicas receive
> information
> >> > about
> >> > > > it
> >> > > > > > and
> >> > > > > > > > thus
> >> > > > > > > > > > > > clients can produce/consume from it, but as was
> >> > discussed
> >> > > > > this
> >> > > > > > > > seems
> >> > > > > > > > > > > > to be a hard thing to do)
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > 2) Probably same as 1), so right after AlterTopic
> is
> >> > > issued
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > 3) The topic has been removed from /brokers/topics
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > 4) ReassignPartitions and PrefferedReplica were
> >> > discussed
> >> > > > > > > earlier -
> >> > > > > > > > > > > > in short the former is completed once partition
> >> state
> >> > > info
> >> > > > in
> >> > > > > > zk
> >> > > > > > > > > > matches
> >> > > > > > > > > > > > reassignment request and admin path is empty, the
> >> > latter
> >> > > -
> >> > > > > once
> >> > > > > > > > data
> >> > > > > > > > > > > > in zk shows that head of assignned replicas of the
> >> > > > partition
> >> > > > > > and
> >> > > > > > > > > leader
> >> > > > > > > > > > > > is the same replica
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Thoughts?
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > Andrii Biletskyi
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <
> >> > > > > > > wangguoz@gmail.com
> >> > > > > > > > >
> >> > > > > > > > > > > wrote:
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > > I think while loop is fine for supporting
> >> blocking,
> >> > > just
> >> > > > > that
> >> > > > > > > we
> >> > > > > > > > > need
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > add back off to avoid bombarding brokers with
> >> > > > DescribeTopic
> >> > > > > > > > > requests.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Also I have linked KAFKA-1125
> >> > > > > > > > > > > > > <
> https://issues.apache.org/jira/browse/KAFKA-1125
> >> >
> >> > to
> >> > > > your
> >> > > > > > > > > proposal,
> >> > > > > > > > > > > and
> >> > > > > > > > > > > > > when KAFKA-1694 is done this ticket can also be
> >> > closed.
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > Guozhang
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii
> Biletskyi
> >> <
> >> > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Great.
> >> > > > > > > > > > > > > > I want to elaborate this a bit more, to see we
> >> are
> >> > on
> >> > > > the
> >> > > > > > > same
> >> > > > > > > > > page
> >> > > > > > > > > > > > > > concerning the client code.
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > So with all topic commands being async a
> client
> >> > > > > > (AdminClient
> >> > > > > > > in
> >> > > > > > > > > our
> >> > > > > > > > > > > > > > case or any other other client people would
> >> like to
> >> > > > > > > implement)
> >> > > > > > > > to
> >> > > > > > > > > > > > support
> >> > > > > > > > > > > > > > a blocking operation (which seems to be a
> >> natural
> >> > > > > use-case
> >> > > > > > > e.g.
> >> > > > > > > > > for
> >> > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > creation): would have to do:
> >> > > > > > > > > > > > > > 1. issue CreateTopicRequest
> >> > > > > > > > > > > > > > 2. if successful, in a "while" loop send
> >> > > > > > DescribeTopicRequest
> >> > > > > > > > and
> >> > > > > > > > > > > > > > break the loop once all topics are returned in
> >> > > response
> >> > > > > (or
> >> > > > > > > > upon
> >> > > > > > > > > > > > > timeout).
> >> > > > > > > > > > > > > > 3. if unsuccessful throw exception
> >> > > > > > > > > > > > > > Would it be okay?
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > Andrii Biletskyi
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <
> >> > > > > jun@confluent.io
> >> > > > > > >
> >> > > > > > > > > wrote:
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Andrii,
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > I think you are right. It seems that only
> >> > > > > > > ReassignPartitions
> >> > > > > > > > > > needs
> >> > > > > > > > > > > a
> >> > > > > > > > > > > > > > > separate verification request.
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > Jun
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii
> >> > Biletskyi <
> >> > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > Guys,
> >> > > > > > > > > > > > > > > > I like this idea too. Let's stick with
> that.
> >> > I'll
> >> > > > > > update
> >> > > > > > > > KIP
> >> > > > > > > > > > > > > > accordingly.
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > I was also thinking we can avoid adding
> >> > dedicated
> >> > > > > > status
> >> > > > > > > > > check
> >> > > > > > > > > > > > > > > > requests for topic commands. - We have
> >> > everything
> >> > > > in
> >> > > > > > > > > > > DescribeTopic
> >> > > > > > > > > > > > > > > > for that! E.g.:
> >> > > > > > > > > > > > > > > > User issued CreateTopic - to check the
> >> status
> >> > > > client
> >> > > > > > > sends
> >> > > > > > > > > > > > > > DescribeTopic
> >> > > > > > > > > > > > > > > > and checks whether is something returned
> for
> >> > that
> >> > > > > > topic.
> >> > > > > > > > The
> >> > > > > > > > > > same
> >> > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > alteration, deletion.
> >> > > > > > > > > > > > > > > > Btw, PreferredReplica status can be also
> >> > checked
> >> > > > with
> >> > > > > > > > > > > > > > > DescribeTopicRequest
> >> > > > > > > > > > > > > > > > (head of assigned replicas list ==
> leader).
> >> > > > > > > > > > > > > > > > For ReassignPartitions as discussed we'll
> >> need
> >> > to
> >> > > > > have
> >> > > > > > a
> >> > > > > > > > > > separate
> >> > > > > > > > > > > > > > > Verify...
> >> > > > > > > > > > > > > > > > request.
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > Andrii Biletskyi
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang
> >> Wang
> >> > <
> >> > > > > > > > > > > wangguoz@gmail.com
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > +1 on broker writing to ZK for async
> >> > handling.
> >> > > I
> >> > > > > was
> >> > > > > > > > > thinking
> >> > > > > > > > > > > > that
> >> > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > end state the admin requests would be
> >> > > eventually
> >> > > > > sent
> >> > > > > > > to
> >> > > > > > > > > > > > controller
> >> > > > > > > > > > > > > > > > either
> >> > > > > > > > > > > > > > > > > through re-routing or clients
> discovering
> >> > them,
> >> > > > > > instead
> >> > > > > > > > of
> >> > > > > > > > > > > > letting
> >> > > > > > > > > > > > > > > > > controller listen on ZK admin path. But
> >> > > thinking
> >> > > > > > about
> >> > > > > > > > it a
> >> > > > > > > > > > > > second
> >> > > > > > > > > > > > > > > time,
> >> > > > > > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > > > think it is actually simpler to let
> >> > controller
> >> > > > > manage
> >> > > > > > > > > > > > > > > > > incoming queued-up admin requests
> through
> >> ZK.
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > Guozhang
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel
> >> Koshy <
> >> > > > > > > > > > > jjkoshy.w@gmail.com
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > +1 as well. I think it helps to keep
> the
> >> > > > > rerouting
> >> > > > > > > > > approach
> >> > > > > > > > > > > > > > > orthogonal
> >> > > > > > > > > > > > > > > > > > to this KIP.
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM
> >> -0700,
> >> > Jay
> >> > > > > Kreps
> >> > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > I'm +1 on Jun's suggestion as long
> as
> >> it
> >> > > can
> >> > > > > work
> >> > > > > > > for
> >> > > > > > > > > all
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > requests.
> >> > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun
> >> Rao
> >> > <
> >> > > > > > > > > > jun@confluent.io
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > Andrii,
> >> > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > I think we agreed on the
> following.
> >> > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > (a) Admin requests can be sent to
> >> and
> >> > > > handled
> >> > > > > > by
> >> > > > > > > > any
> >> > > > > > > > > > > > broker.
> >> > > > > > > > > > > > > > > > > > > > (b) Admin requests are processed
> >> > > > > > asynchronously,
> >> > > > > > > at
> >> > > > > > > > > > least
> >> > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > now.
> >> > > > > > > > > > > > > > > > > > That is,
> >> > > > > > > > > > > > > > > > > > > > when the client gets a response,
> it
> >> > just
> >> > > > > means
> >> > > > > > > that
> >> > > > > > > > > the
> >> > > > > > > > > > > > > request
> >> > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > initiated, but not necessarily
> >> > completed.
> >> > > > > Then,
> >> > > > > > > > it's
> >> > > > > > > > > up
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > client
> >> > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > issue another request to check the
> >> > status
> >> > > > for
> >> > > > > > > > > > completion.
> >> > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > To support (a), we were thinking
> of
> >> > doing
> >> > > > > > request
> >> > > > > > > > > > > > forwarding
> >> > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > controller (utilizing KAFKA-1912).
> >> I am
> >> > > > > making
> >> > > > > > an
> >> > > > > > > > > > > > alternative
> >> > > > > > > > > > > > > > > > > proposal.
> >> > > > > > > > > > > > > > > > > > > > Basically, the broker can just
> >> write to
> >> > > > > > ZooKeeper
> >> > > > > > > > to
> >> > > > > > > > > > > inform
> >> > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > controller
> >> > > > > > > > > > > > > > > > > > > > about the request. For example, to
> >> > handle
> >> > > > > > > > > > > > > > partitionReassignment,
> >> > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > broker
> >> > > > > > > > > > > > > > > > > > > > will just write the requested
> >> > partitions
> >> > > to
> >> > > > > > > > > > > > > > > > > /admin/reassign_partitions
> >> > > > > > > > > > > > > > > > > > > > (like what AdminUtils currently
> >> does)
> >> > and
> >> > > > > then
> >> > > > > > > > send a
> >> > > > > > > > > > > > > response
> >> > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > client. This shouldn't take long
> and
> >> > the
> >> > > > > > > > > implementation
> >> > > > > > > > > > > > will
> >> > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > simpler
> >> > > > > > > > > > > > > > > > > > > > than forwarding the requests to
> the
> >> > > > > controller
> >> > > > > > > > > through
> >> > > > > > > > > > > RPC.
> >> > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > Jun
> >> > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM,
> >> Andrii
> >> > > > > > > Biletskyi <
> >> > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> wrote:
> >> > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > Jun,
> >> > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > I might be wrong but didn't we
> >> agree
> >> > we
> >> > > > > will
> >> > > > > > > let
> >> > > > > > > > > any
> >> > > > > > > > > > > > broker
> >> > > > > > > > > > > > > > > from
> >> > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > cluster handle *long-running*
> >> admin
> >> > > > > requests
> >> > > > > > > (at
> >> > > > > > > > > this
> >> > > > > > > > > > > > time
> >> > > > > > > > > > > > > > > > > > > > preferredReplica
> >> > > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > reassignPartitions), via zk
> admin
> >> > path.
> >> > > > > Thus
> >> > > > > > > > > > > CreateTopics
> >> > > > > > > > > > > > > etc
> >> > > > > > > > > > > > > > > > > should
> >> > > > > > > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > > > > > sent
> >> > > > > > > > > > > > > > > > > > > > > only to the controller.
> >> > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> >> > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55
> PM,
> >> Jun
> >> > > > Rao <
> >> > > > > > > > > > > > > jun@confluent.io>
> >> > > > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > Joel, Andril,
> >> > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > I think we agreed that those
> >> admin
> >> > > > > requests
> >> > > > > > > can
> >> > > > > > > > > be
> >> > > > > > > > > > > > issued
> >> > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > any
> >> > > > > > > > > > > > > > > > > > > > broker.
> >> > > > > > > > > > > > > > > > > > > > > > Because of that, there doesn't
> >> seem
> >> > > to
> >> > > > > be a
> >> > > > > > > > > strong
> >> > > > > > > > > > > need
> >> > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > know
> >> > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > controller. So, perhaps we can
> >> > > proceed
> >> > > > by
> >> > > > > > not
> >> > > > > > > > > > making
> >> > > > > > > > > > > > any
> >> > > > > > > > > > > > > > > change
> >> > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > format of TMR right now. When
> we
> >> > > start
> >> > > > > > using
> >> > > > > > > > > create
> >> > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > request
> >> > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > producer, we will need a new
> >> > version
> >> > > of
> >> > > > > TMR
> >> > > > > > > > that
> >> > > > > > > > > > > > doesn't
> >> > > > > > > > > > > > > > > > trigger
> >> > > > > > > > > > > > > > > > > > auto
> >> > > > > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > creation. But that can be done
> >> > later.
> >> > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > As a first cut
> implementation, I
> >> > > think
> >> > > > > the
> >> > > > > > > > broker
> >> > > > > > > > > > can
> >> > > > > > > > > > > > > just
> >> > > > > > > > > > > > > > > > write
> >> > > > > > > > > > > > > > > > > > to ZK
> >> > > > > > > > > > > > > > > > > > > > > > directly for
> >> > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > >
> >> > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> >> > > > > > > > > > > > > > > > > > > > > > requests, instead of
> forwarding
> >> > them
> >> > > to
> >> > > > > the
> >> > > > > > > > > > > controller.
> >> > > > > > > > > > > > > > This
> >> > > > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > > > > > simplify
> >> > > > > > > > > > > > > > > > > > > > > > the implementation on the
> broker
> >> > > side.
> >> > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > Jun
> >> > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58
> >> AM,
> >> > > Joel
> >> > > > > > Koshy
> >> > > > > > > <
> >> > > > > > > > > > > > > > > > > jjkoshy.w@gmail.com>
> >> > > > > > > > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > (Thanks Andrii for the
> >> summary)
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > For (1) yes we will circle
> >> back
> >> > on
> >> > > > that
> >> > > > > > > > shortly
> >> > > > > > > > > > > after
> >> > > > > > > > > > > > > > > syncing
> >> > > > > > > > > > > > > > > > > up
> >> > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > person. I think it is close
> to
> >> > > > getting
> >> > > > > > > > > committed
> >> > > > > > > > > > > > > although
> >> > > > > > > > > > > > > > > > > > development
> >> > > > > > > > > > > > > > > > > > > > > > > for KAFKA-1927 can probably
> >> begin
> >> > > > > without
> >> > > > > > > it.
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > There is one more item we
> >> covered
> >> > > at
> >> > > > > the
> >> > > > > > > > > hangout.
> >> > > > > > > > > > > > i.e.,
> >> > > > > > > > > > > > > > > > whether
> >> > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > want to add the coordinator
> to
> >> > the
> >> > > > > topic
> >> > > > > > > > > metadata
> >> > > > > > > > > > > > > > response
> >> > > > > > > > > > > > > > > or
> >> > > > > > > > > > > > > > > > > > provide
> >> > > > > > > > > > > > > > > > > > > > > > > a clearer
> >> ClusterMetadataRequest.
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > There are two reasons I
> think
> >> we
> >> > > > should
> >> > > > > > try
> >> > > > > > > > and
> >> > > > > > > > > > > avoid
> >> > > > > > > > > > > > > > > adding
> >> > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > field:
> >> > > > > > > > > > > > > > > > > > > > > > > - It is irrelevant to topic
> >> > > metadata
> >> > > > > > > > > > > > > > > > > > > > > > > - If we finally do request
> >> > > rerouting
> >> > > > in
> >> > > > > > > Kafka
> >> > > > > > > > > > then
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > field
> >> > > > > > > > > > > > > > > > > > would
> >> > > > > > > > > > > > > > > > > > > > add
> >> > > > > > > > > > > > > > > > > > > > > > >   little to no value. (It
> >> still
> >> > > helps
> >> > > > > to
> >> > > > > > > > have a
> >> > > > > > > > > > > > > separate
> >> > > > > > > > > > > > > > > > > > > > > > >   ClusterMetadataRequest to
> >> query
> >> > > for
> >> > > > > > > > > > cluster-wide
> >> > > > > > > > > > > > > > > > information
> >> > > > > > > > > > > > > > > > > > such
> >> > > > > > > > > > > > > > > > > > > > as
> >> > > > > > > > > > > > > > > > > > > > > > >   'which broker is the
> >> > controller?'
> >> > > > as
> >> > > > > > Joe
> >> > > > > > > > > > > > mentioned.)
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > I think it would be cleaner
> to
> >> > have
> >> > > > an
> >> > > > > > > > explicit
> >> > > > > > > > > > > > > > > > > > > > ClusterMetadataRequest
> >> > > > > > > > > > > > > > > > > > > > > > > that you can send to any
> >> broker
> >> > in
> >> > > > > order
> >> > > > > > to
> >> > > > > > > > > > obtain
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > controller
> >> > > > > > > > > > > > > > > > > > > > (and
> >> > > > > > > > > > > > > > > > > > > > > > > in the future possibly other
> >> > > > > cluster-wide
> >> > > > > > > > > > > > > information). I
> >> > > > > > > > > > > > > > > > think
> >> > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > main argument against doing
> >> this
> >> > > and
> >> > > > > > > instead
> >> > > > > > > > > > adding
> >> > > > > > > > > > > > it
> >> > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > metadata response was
> >> > convenience -
> >> > > > > i.e.,
> >> > > > > > > you
> >> > > > > > > > > > don't
> >> > > > > > > > > > > > > have
> >> > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > discover
> >> > > > > > > > > > > > > > > > > > > > > > > the controller in advance.
> >> > > However, I
> >> > > > > > don't
> >> > > > > > > > see
> >> > > > > > > > > > > much
> >> > > > > > > > > > > > > > actual
> >> > > > > > > > > > > > > > > > > > > > > > > benefit/convenience in this
> >> and
> >> > in
> >> > > > fact
> >> > > > > > > think
> >> > > > > > > > > it
> >> > > > > > > > > > > is a
> >> > > > > > > > > > > > > > > > > non-issue.
> >> > > > > > > > > > > > > > > > > > Let
> >> > > > > > > > > > > > > > > > > > > > > > > me know if I'm overlooking
> >> > > something
> >> > > > > > here.
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > As an example, say we need
> to
> >> > > > initiate
> >> > > > > > > > > partition
> >> > > > > > > > > > > > > > > reassignment
> >> > > > > > > > > > > > > > > > > by
> >> > > > > > > > > > > > > > > > > > > > > > > issuing the new
> >> > > > > ReassignPartitionsRequest
> >> > > > > > > to
> >> > > > > > > > > the
> >> > > > > > > > > > > > > > controller
> >> > > > > > > > > > > > > > > > > > (assume
> >> > > > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > already have the desired
> >> manual
> >> > > > > partition
> >> > > > > > > > > > > > assignment).
> >> > > > > > > > > > > > > > If
> >> > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > are to
> >> > > > > > > > > > > > > > > > > > > > > > > augment topic metadata
> >> response
> >> > > then
> >> > > > > the
> >> > > > > > > flow
> >> > > > > > > > > be
> >> > > > > > > > > > > > > > something
> >> > > > > > > > > > > > > > > > like
> >> > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > :
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > - Issue topic metadata
> >> request to
> >> > > any
> >> > > > > > > broker
> >> > > > > > > > > (and
> >> > > > > > > > > > > > > > discover
> >> > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > >   controller
> >> > > > > > > > > > > > > > > > > > > > > > > - Connect to controller if
> >> > required
> >> > > > > > (i.e.,
> >> > > > > > > if
> >> > > > > > > > > the
> >> > > > > > > > > > > > > broker
> >> > > > > > > > > > > > > > > > above
> >> > > > > > > > > > > > > > > > > !=
> >> > > > > > > > > > > > > > > > > > > > > > >   controller)
> >> > > > > > > > > > > > > > > > > > > > > > > - Issue the partition
> >> > reassignment
> >> > > > > > request
> >> > > > > > > to
> >> > > > > > > > > the
> >> > > > > > > > > > > > > > > controller.
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > With an explicit cluster
> >> metadata
> >> > > > > request
> >> > > > > > > it
> >> > > > > > > > > > would
> >> > > > > > > > > > > > be:
> >> > > > > > > > > > > > > > > > > > > > > > > - Issue cluster metadata
> >> request
> >> > to
> >> > > > any
> >> > > > > > > > broker
> >> > > > > > > > > > > > > > > > > > > > > > > - Connect to controller if
> >> > required
> >> > > > > > (i.e.,
> >> > > > > > > if
> >> > > > > > > > > the
> >> > > > > > > > > > > > > broker
> >> > > > > > > > > > > > > > > > above
> >> > > > > > > > > > > > > > > > > !=
> >> > > > > > > > > > > > > > > > > > > > > > >   controller)
> >> > > > > > > > > > > > > > > > > > > > > > > - Issue the partition
> >> > reassignment
> >> > > > > > request
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > So it seems to add little
> >> > practical
> >> > > > > value
> >> > > > > > > and
> >> > > > > > > > > > > bloats
> >> > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > metadata
> >> > > > > > > > > > > > > > > > > > > > > > > response with an irrelevant
> >> > detail.
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > The other angle to this is
> the
> >> > > > > following
> >> > > > > > -
> >> > > > > > > is
> >> > > > > > > > > it
> >> > > > > > > > > > a
> >> > > > > > > > > > > > > matter
> >> > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > naming?
> >> > > > > > > > > > > > > > > > > > > > > > > Should we just rename topic
> >> > > metadata
> >> > > > > > > > > > > request/response
> >> > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > just
> >> > > > > > > > > > > > > > > > > > > > > > > MetadataRequest/Response and
> >> add
> >> > > > > cluster
> >> > > > > > > > > metadata
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > it?
> >> > > > > > > > > > > > > > By
> >> > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > same
> >> > > > > > > > > > > > > > > > > > > > > > > token should we also allow
> >> > querying
> >> > > > for
> >> > > > > > the
> >> > > > > > > > > > > consumer
> >> > > > > > > > > > > > > > > > > coordinator
> >> > > > > > > > > > > > > > > > > > (and
> >> > > > > > > > > > > > > > > > > > > > > > > in future transaction
> >> > coordinator)
> >> > > as
> >> > > > > > well?
> >> > > > > > > > > This
> >> > > > > > > > > > > > leads
> >> > > > > > > > > > > > > > to a
> >> > > > > > > > > > > > > > > > > > bloated
> >> > > > > > > > > > > > > > > > > > > > > > > request which isn't very
> >> > appealing
> >> > > > and
> >> > > > > > > > > altogether
> >> > > > > > > > > > > > > > > confusing.
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > Joel
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at
> >> > 09:34:12AM
> >> > > > > -0700,
> >> > > > > > > Jun
> >> > > > > > > > > Rao
> >> > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > Andri,
> >> > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > Thanks for the summary.
> >> > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > 1. I just realized that in
> >> > order
> >> > > to
> >> > > > > > start
> >> > > > > > > > > > working
> >> > > > > > > > > > > > on
> >> > > > > > > > > > > > > > > > > > KAFKA-1927, we
> >> > > > > > > > > > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > > > > > > > > need to merge the changes
> to
> >> > > > > > > > > > OffsetCommitRequest
> >> > > > > > > > > > > > > (from
> >> > > > > > > > > > > > > > > > 0.8.2)
> >> > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > trunk.
> >> > > > > > > > > > > > > > > > > > > > > > > > This is planned to be done
> >> as
> >> > > part
> >> > > > of
> >> > > > > > > > > > KAFKA-1634.
> >> > > > > > > > > > > > So,
> >> > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > > need
> >> > > > > > > > > > > > > > > > > > > > > > > Guozhang
> >> > > > > > > > > > > > > > > > > > > > > > > > and Joel's help to wrap
> this
> >> > up.
> >> > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > 2. Thinking about this a
> bit
> >> > > more,
> >> > > > if
> >> > > > > > the
> >> > > > > > > > > > > semantic
> >> > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > those
> >> > > > > > > > > > > > > > > > > > "write"
> >> > > > > > > > > > > > > > > > > > > > > > > > requests is async (i.e.,
> >> after
> >> > > the
> >> > > > > > client
> >> > > > > > > > > gets
> >> > > > > > > > > > a
> >> > > > > > > > > > > > > > > response,
> >> > > > > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > > > > just
> >> > > > > > > > > > > > > > > > > > > > > > means
> >> > > > > > > > > > > > > > > > > > > > > > > > that the operation is
> >> > initiated,
> >> > > > but
> >> > > > > > not
> >> > > > > > > > > > > > necessarily
> >> > > > > > > > > > > > > > > > > > completed), we
> >> > > > > > > > > > > > > > > > > > > > > > don't
> >> > > > > > > > > > > > > > > > > > > > > > > > really need to forward the
> >> > > requests
> >> > > > > to
> >> > > > > > > the
> >> > > > > > > > > > > > > controller.
> >> > > > > > > > > > > > > > > > > > Instead, the
> >> > > > > > > > > > > > > > > > > > > > > > > > receiving broker can just
> >> write
> >> > > the
> >> > > > > > > > operation
> >> > > > > > > > > > to
> >> > > > > > > > > > > ZK
> >> > > > > > > > > > > > > as
> >> > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > admin
> >> > > > > > > > > > > > > > > > > > > > > > command
> >> > > > > > > > > > > > > > > > > > > > > > > > line tool previously does.
> >> This
> >> > > > will
> >> > > > > > > > simplify
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > implementation.
> >> > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > 8. There is another
> >> > > implementation
> >> > > > > > detail
> >> > > > > > > > for
> >> > > > > > > > > > > > > describe
> >> > > > > > > > > > > > > > > > topic.
> >> > > > > > > > > > > > > > > > > > > > > Ideally,
> >> > > > > > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > want to read the topic
> >> config
> >> > > from
> >> > > > > the
> >> > > > > > > > broker
> >> > > > > > > > > > > > cache,
> >> > > > > > > > > > > > > > > > instead
> >> > > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > > > > > > ZooKeeper.
> >> > > > > > > > > > > > > > > > > > > > > > > > Currently, every broker
> >> reads
> >> > the
> >> > > > > > > > topic-level
> >> > > > > > > > > > > > config
> >> > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > all
> >> > > > > > > > > > > > > > > > > > > > topics.
> >> > > > > > > > > > > > > > > > > > > > > > > > However, it ignores those
> >> for
> >> > > > topics
> >> > > > > > not
> >> > > > > > > > > hosted
> >> > > > > > > > > > > on
> >> > > > > > > > > > > > > > > itself.
> >> > > > > > > > > > > > > > > > > So,
> >> > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > may
> >> > > > > > > > > > > > > > > > > > > > > > > need
> >> > > > > > > > > > > > > > > > > > > > > > > > to change
> >> TopicConfigManager a
> >> > > bit
> >> > > > so
> >> > > > > > > that
> >> > > > > > > > it
> >> > > > > > > > > > > > caches
> >> > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > configs
> >> > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > all
> >> > > > > > > > > > > > > > > > > > > > > > > > topics.
> >> > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > Jun
> >> > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at
> 1:13
> >> > PM,
> >> > > > > Andrii
> >> > > > > > > > > > > Biletskyi <
> >> > > > > > > > > > > > > > > > > > > > > > > >
> andrii.biletskyi@stealth.ly
> >> >
> >> > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > Guys,
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > Thanks for a great
> >> > discussion!
> >> > > > > > > > > > > > > > > > > > > > > > > > > Here are the actions
> >> points:
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > 1. Q: Get rid of all
> scala
> >> > > > requests
> >> > > > > > > > > objects,
> >> > > > > > > > > > > use
> >> > > > > > > > > > > > > java
> >> > > > > > > > > > > > > > > > > > protocol
> >> > > > > > > > > > > > > > > > > > > > > > > definitions.
> >> > > > > > > > > > > > > > > > > > > > > > > > >     A: Gwen kindly took
> >> that
> >> > > > > > > > (KAFKA-1927).
> >> > > > > > > > > > It's
> >> > > > > > > > > > > > > > > important
> >> > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > speed
> >> > > > > > > > > > > > > > > > > > > > > up
> >> > > > > > > > > > > > > > > > > > > > > > > > > review procedure
> >> > > > > > > > > > > > > > > > > > > > > > > > >          there since
> this
> >> > > ticket
> >> > > > > > blocks
> >> > > > > > > > > other
> >> > > > > > > > > > > > > > important
> >> > > > > > > > > > > > > > > > > > changes.
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute
> >> > > facility
> >> > > > > vs
> >> > > > > > > > client
> >> > > > > > > > > > > > > > maintaining
> >> > > > > > > > > > > > > > > > > > cluster
> >> > > > > > > > > > > > > > > > > > > > > > state.
> >> > > > > > > > > > > > > > > > > > > > > > > > >     A: Jay has added
> >> pseudo
> >> > > code
> >> > > > to
> >> > > > > > > > > > KAFKA-1912
> >> > > > > > > > > > > -
> >> > > > > > > > > > > > > need
> >> > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > consider
> >> > > > > > > > > > > > > > > > > > > > > > > whether
> >> > > > > > > > > > > > > > > > > > > > > > > > > this will be
> >> > > > > > > > > > > > > > > > > > > > > > > > >         easy to
> implement
> >> as
> >> > a
> >> > > > > > > > server-side
> >> > > > > > > > > > > > feature
> >> > > > > > > > > > > > > > > > > (comments
> >> > > > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > > > > > > > > welcomed!).
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > 3. Q: Controller field
> in
> >> > wire
> >> > > > > > > protocol.
> >> > > > > > > > > > > > > > > > > > > > > > > > >     A: This might be
> >> useful
> >> > for
> >> > > > > > > clients,
> >> > > > > > > > > add
> >> > > > > > > > > > > this
> >> > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > TopicMetadataResponse
> >> > > > > > > > > > > > > > > > > > > > > > > > > (already in KIP).
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic
> >> > creation
> >> > > > > from
> >> > > > > > > TMR.
> >> > > > > > > > > > > > > > > > > > > > > > > > >     A: I will add
> >> proposed by
> >> > > Jun
> >> > > > > > > > solution
> >> > > > > > > > > > > (using
> >> > > > > > > > > > > > > > > > clientId
> >> > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > that)
> >> > > > > > > > > > > > > > > > > > > > > > > to the
> >> > > > > > > > > > > > > > > > > > > > > > > > > KIP.
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > 5. Q: Bumping new
> >> versions of
> >> > > TMR
> >> > > > > vs
> >> > > > > > > > > grabbing
> >> > > > > > > > > > > all
> >> > > > > > > > > > > > > > > > protocol
> >> > > > > > > > > > > > > > > > > > > > changes
> >> > > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > one
> >> > > > > > > > > > > > > > > > > > > > > > > > > version.
> >> > > > > > > > > > > > > > > > > > > > > > > > >     A: It was decided to
> >> try
> >> > to
> >> > > > > > gather
> >> > > > > > > > all
> >> > > > > > > > > > > > changes
> >> > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > protocol
> >> > > > > > > > > > > > > > > > > > > > > > (before
> >> > > > > > > > > > > > > > > > > > > > > > > > > release).
> >> > > > > > > > > > > > > > > > > > > > > > > > >         In case of TMR
> it
> >> > worth
> >> > > > > > > checking:
> >> > > > > > > > > > > > > KAFKA-2020
> >> > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > KIP-13
> >> > > > > > > > > > > > > > > > > > > > > > > (quotas)
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed
> >> to
> >> > > > > > deserialize
> >> > > > > > > > > > user's
> >> > > > > > > > > > > > > input
> >> > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > CLI
> >> > > > > > > > > > > > > > > > > > tool.
> >> > > > > > > > > > > > > > > > > > > > > > > > >     A: Use jackson for
> >> that,
> >> > > > /tools
> >> > > > > > > > project
> >> > > > > > > > > > is
> >> > > > > > > > > > > a
> >> > > > > > > > > > > > > > > separate
> >> > > > > > > > > > > > > > > > > > jar so
> >> > > > > > > > > > > > > > > > > > > > > > > shouldn't
> >> > > > > > > > > > > > > > > > > > > > > > > > > be a big deal.
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > 7.  Q:
> >> > VerifyReassingPartitions
> >> > > > vs
> >> > > > > > > > generic
> >> > > > > > > > > > > status
> >> > > > > > > > > > > > > > check
> >> > > > > > > > > > > > > > > > > > command.
> >> > > > > > > > > > > > > > > > > > > > > > > > >      A: For long-running
> >> > > requests
> >> > > > > > like
> >> > > > > > > > > > reassign
> >> > > > > > > > > > > > > > > > partitions
> >> > > > > > > > > > > > > > > > > > > > > *progress*
> >> > > > > > > > > > > > > > > > > > > > > > > check
> >> > > > > > > > > > > > > > > > > > > > > > > > > request is useful,
> >> > > > > > > > > > > > > > > > > > > > > > > > >          it makes sense
> to
> >> > > > > introduce
> >> > > > > > > it.
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > >  Please add, correct me
> >> if I
> >> > > > missed
> >> > > > > > > > > > something.
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at
> >> 6:20
> >> > > PM,
> >> > > > > > Andrii
> >> > > > > > > > > > > > Biletskyi <
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> andrii.biletskyi@stealth.ly>
> >> > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > > Joel,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > > You are right, I
> removed
> >> > > > > > > > ClusterMetadata
> >> > > > > > > > > > > > because
> >> > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > have
> >> > > > > > > > > > > > > > > > > > > > > partially
> >> > > > > > > > > > > > > > > > > > > > > > > > > > what we need in
> >> > > TopicMetadata.
> >> > > > > > Also,
> >> > > > > > > as
> >> > > > > > > > > Jay
> >> > > > > > > > > > > > > pointed
> >> > > > > > > > > > > > > > > out
> >> > > > > > > > > > > > > > > > > > > > earlier,
> >> > > > > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > > would like to have
> >> > > "orthogonal"
> >> > > > > > API,
> >> > > > > > > > but
> >> > > > > > > > > at
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > > same
> >> > > > > > > > > > > > > > > > time
> >> > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > need
> >> > > > > > > > > > > > > > > > > > > > > > > > > > to be backward
> >> compatible.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > > But I like your idea
> and
> >> > even
> >> > > > > have
> >> > > > > > > some
> >> > > > > > > > > > other
> >> > > > > > > > > > > > > > > arguments
> >> > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > option:
> >> > > > > > > > > > > > > > > > > > > > > > > > > > There is also
> >> > > > > DescribeTopicRequest
> >> > > > > > > > which
> >> > > > > > > > > > was
> >> > > > > > > > > > > > > > proposed
> >> > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > KIP,
> >> > > > > > > > > > > > > > > > > > > > > > > > > > it returns topic
> >> configs,
> >> > > > > > partitions,
> >> > > > > > > > > > > > replication
> >> > > > > > > > > > > > > > > > factor
> >> > > > > > > > > > > > > > > > > > plus
> >> > > > > > > > > > > > > > > > > > > > > > > partition
> >> > > > > > > > > > > > > > > > > > > > > > > > > > ISR, ASR,
> >> > > > > > > > > > > > > > > > > > > > > > > > > > leader replica. The
> >> later
> >> > > part
> >> > > > is
> >> > > > > > > > really
> >> > > > > > > > > > > > already
> >> > > > > > > > > > > > > > > there
> >> > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> >> > > > > > > > > > > > > > > > > > > > > > > > > > So again we'll have to
> >> add
> >> > > > stuff
> >> > > > > to
> >> > > > > > > > TMR,
> >> > > > > > > > > > not
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > duplicate
> >> > > > > > > > > > > > > > > > > > some
> >> > > > > > > > > > > > > > > > > > > > > info
> >> > > > > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > > > > newly added requests.
> >> > > However,
> >> > > > > this
> >> > > > > > > way
> >> > > > > > > > > > we'll
> >> > > > > > > > > > > > end
> >> > > > > > > > > > > > > > up
> >> > > > > > > > > > > > > > > > with
> >> > > > > > > > > > > > > > > > > > > > > "monster"
> >> > > > > > > > > > > > > > > > > > > > > > > > > > request which returns
> >> > cluster
> >> > > > > > > metadata,
> >> > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > replication
> >> > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > config
> >> > > > > > > > > > > > > > > > > > > > > > > info
> >> > > > > > > > > > > > > > > > > > > > > > > > > > plus partition
> >> replication
> >> > > > data.
> >> > > > > > > Seems
> >> > > > > > > > > > > logical
> >> > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > split
> >> > > > > > > > > > > > > > > > > > TMR to
> >> > > > > > > > > > > > > > > > > > > > > > > > > > - ClusterMetadata
> >> (brokers
> >> > +
> >> > > > > > > > controller,
> >> > > > > > > > > > > maybe
> >> > > > > > > > > > > > > smth
> >> > > > > > > > > > > > > > > > else)
> >> > > > > > > > > > > > > > > > > > > > > > > > > > - TopicMetadata (topic
> >> > info +
> >> > > > > > > partition
> >> > > > > > > > > > > > details)
> >> > > > > > > > > > > > > > > > > > > > > > > > > > But since current TMR
> is
> >> > > > involved
> >> > > > > > in
> >> > > > > > > > lots
> >> > > > > > > > > > of
> >> > > > > > > > > > > > > places
> >> > > > > > > > > > > > > > > > > > (including
> >> > > > > > > > > > > > > > > > > > > > > > > network
> >> > > > > > > > > > > > > > > > > > > > > > > > > > client,
> >> > > > > > > > > > > > > > > > > > > > > > > > > > as I understand) this
> >> might
> >> > > be
> >> > > > > very
> >> > > > > > > > > serious
> >> > > > > > > > > > > > > change
> >> > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > > > > > > > probably
> >> > > > > > > > > > > > > > > > > > > > > > > makes
> >> > > > > > > > > > > > > > > > > > > > > > > > > > sense to stick with
> >> current
> >> > > > > > approach.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> >> > > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015
> at
> >> > 5:29
> >> > > > PM,
> >> > > > > > Joel
> >> > > > > > > > > > Koshy <
> >> > > > > > > > > > > > > > > > > > > > jjkoshy.w@gmail.com
> >> > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> I may be missing some
> >> > > context
> >> > > > > but
> >> > > > > > > > > > hopefully
> >> > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > > also be
> >> > > > > > > > > > > > > > > > > > > > > > > covered
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> today: I thought the
> >> > earlier
> >> > > > > > > proposal
> >> > > > > > > > > > where
> >> > > > > > > > > > > > > there
> >> > > > > > > > > > > > > > > was
> >> > > > > > > > > > > > > > > > an
> >> > > > > > > > > > > > > > > > > > > > > explicit
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> ClusterMetadata
> request
> >> > was
> >> > > > > > clearer
> >> > > > > > > > and
> >> > > > > > > > > > > > > explicit.
> >> > > > > > > > > > > > > > > > During
> >> > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > course
> >> > > > > > > > > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> this thread I think
> the
> >> > > > > conclusion
> >> > > > > > > was
> >> > > > > > > > > > that
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > main
> >> > > > > > > > > > > > > > > > > need
> >> > > > > > > > > > > > > > > > > > was
> >> > > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> controller
> information
> >> and
> >> > > > that
> >> > > > > > can
> >> > > > > > > be
> >> > > > > > > > > > > rolled
> >> > > > > > > > > > > > > into
> >> > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > metadata
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> response but that
> >> seems a
> >> > > bit
> >> > > > > > > > irrelevant
> >> > > > > > > > > > to
> >> > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > metadata.
> >> > > > > > > > > > > > > > > > > > > > > FWIW I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> think the full
> >> broker-list
> >> > > is
> >> > > > > also
> >> > > > > > > > > > > irrelevant
> >> > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > metadata,
> >> > > > > > > > > > > > > > > > > > > > > > but
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> it is already there
> >> and in
> >> > > > use.
> >> > > > > I
> >> > > > > > > > think
> >> > > > > > > > > > > there
> >> > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > still
> >> > > > > > > > > > > > > > > > > > room
> >> > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > an
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> explicit
> >> ClusterMetadata
> >> > > > request
> >> > > > > > > since
> >> > > > > > > > > > there
> >> > > > > > > > > > > > may
> >> > > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > other
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> cluster-level
> >> information
> >> > > that
> >> > > > > we
> >> > > > > > > may
> >> > > > > > > > > want
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > add
> >> > > > > > > > > > > > > > > over
> >> > > > > > > > > > > > > > > > > > time
> >> > > > > > > > > > > > > > > > > > > > (and
> >> > > > > > > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> have nothing to do
> with
> >> > > topic
> >> > > > > > > > metadata).
> >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015
> at
> >> > > > > 02:45:30PM
> >> > > > > > > > > +0200,
> >> > > > > > > > > > > > Andrii
> >> > > > > > > > > > > > > > > > > Biletskyi
> >> > > > > > > > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > Jun,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > 101. Okay, if you
> say
> >> > that
> >> > > > > such
> >> > > > > > > use
> >> > > > > > > > > case
> >> > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > important. I
> >> > > > > > > > > > > > > > > > > > > > also
> >> > > > > > > > > > > > > > > > > > > > > > > think
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > using clientId for
> >> these
> >> > > > > > purposes
> >> > > > > > > is
> >> > > > > > > > > > fine
> >> > > > > > > > > > > -
> >> > > > > > > > > > > > if
> >> > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > already
> >> > > > > > > > > > > > > > > > > > > > have
> >> > > > > > > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> field
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > as part of all Wire
> >> > > protocol
> >> > > > > > > > messages,
> >> > > > > > > > > > why
> >> > > > > > > > > > > > not
> >> > > > > > > > > > > > > > use
> >> > > > > > > > > > > > > > > > > that.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > I will update KIP-4
> >> page
> >> > > if
> >> > > > > > nobody
> >> > > > > > > > has
> >> > > > > > > > > > > other
> >> > > > > > > > > > > > > > ideas
> >> > > > > > > > > > > > > > > > > > (which
> >> > > > > > > > > > > > > > > > > > > > may
> >> > > > > > > > > > > > > > > > > > > > > > > come up
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > during the call
> >> today).
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll
> >> update
> >> > > the
> >> > > > > KIP
> >> > > > > > > > > > > > accordingly.
> >> > > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > > think
> >> > > > > > > > > > > > > > > > > > we can
> >> > > > > > > > > > > > > > > > > > > > > add
> >> > > > > > > > > > > > > > > > > > > > > > > new,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > fine-grained error
> >> codes
> >> > > if
> >> > > > > some
> >> > > > > > > > error
> >> > > > > > > > > > > code
> >> > > > > > > > > > > > > > > received
> >> > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > specific
> >> > > > > > > > > > > > > > > > > > > > > > > case
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > won't give enough
> >> > context
> >> > > to
> >> > > > > > > return
> >> > > > > > > > a
> >> > > > > > > > > > > > > > descriptive
> >> > > > > > > > > > > > > > > > > error
> >> > > > > > > > > > > > > > > > > > > > > message
> >> > > > > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> user.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > Look forward to
> >> > discussing
> >> > > > all
> >> > > > > > > > > > outstanding
> >> > > > > > > > > > > > > > issues
> >> > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > detail
> >> > > > > > > > > > > > > > > > > > > > > > today
> >> > > > > > > > > > > > > > > > > > > > > > > > > during
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > the call.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16,
> 2015
> >> at
> >> > > > 10:59
> >> > > > > > PM,
> >> > > > > > > > Jun
> >> > > > > > > > > > Rao
> >> > > > > > > > > > > <
> >> > > > > > > > > > > > > > > > > > jun@confluent.io
> >> > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 101. There may
> be a
> >> > use
> >> > > > case
> >> > > > > > > where
> >> > > > > > > > > you
> >> > > > > > > > > > > > only
> >> > > > > > > > > > > > > > want
> >> > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > topics
> >> > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> created
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > manually by
> admins.
> >> > > > > Currently,
> >> > > > > > > you
> >> > > > > > > > > can
> >> > > > > > > > > > > do
> >> > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > by
> >> > > > > > > > > > > > > > > > > > > > disabling
> >> > > > > > > > > > > > > > > > > > > > > > auto
> >> > > > > > > > > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > creation and
> issue
> >> > topic
> >> > > > > > > creation
> >> > > > > > > > > from
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > TopicCommand.
> >> > > > > > > > > > > > > > > > > > > > If
> >> > > > > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> disable auto
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > topic creation
> >> > > completely
> >> > > > on
> >> > > > > > the
> >> > > > > > > > > > broker
> >> > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > don't
> >> > > > > > > > > > > > > > > > > > have a
> >> > > > > > > > > > > > > > > > > > > > way
> >> > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> distinguish
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > between topic
> >> creation
> >> > > > > > requests
> >> > > > > > > > from
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > > regular
> >> > > > > > > > > > > > > > > > > > clients
> >> > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> admin, we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > can't support
> >> manual
> >> > > topic
> >> > > > > > > > creation
> >> > > > > > > > > > any
> >> > > > > > > > > > > > > more.
> >> > > > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > > was
> >> > > > > > > > > > > > > > > > > > > > thinking
> >> > > > > > > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> another
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > way of
> >> distinguishing
> >> > > the
> >> > > > > > > clients
> >> > > > > > > > > > making
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > creation
> >> > > > > > > > > > > > > > > > > > > > > > > requests
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > using clientId.
> For
> >> > > > example,
> >> > > > > > the
> >> > > > > > > > > admin
> >> > > > > > > > > > > > tool
> >> > > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > set
> >> > > > > > > > > > > > > > > > > > it to
> >> > > > > > > > > > > > > > > > > > > > > > > something
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> like
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin and the
> >> broker
> >> > can
> >> > > > > treat
> >> > > > > > > > that
> >> > > > > > > > > > > > clientId
> >> > > > > > > > > > > > > > > > > > specially.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Also, there is a
> >> > related
> >> > > > > > > > discussion
> >> > > > > > > > > in
> >> > > > > > > > > > > > > > > KAFKA-2020.
> >> > > > > > > > > > > > > > > > > > > > > Currently,
> >> > > > > > > > > > > > > > > > > > > > > > > we do
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > following in
> >> > > > > > > > TopicMetadataResponse:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 1. If leader is
> not
> >> > > > > available,
> >> > > > > > > we
> >> > > > > > > > > set
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > partition
> >> > > > > > > > > > > > > > > > > > level
> >> > > > > > > > > > > > > > > > > > > > > > error
> >> > > > > > > > > > > > > > > > > > > > > > > code
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> LeaderNotAvailable.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 2. If a
> non-leader
> >> > > replica
> >> > > > > is
> >> > > > > > > not
> >> > > > > > > > > > > > available,
> >> > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > take
> >> > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > replica
> >> > > > > > > > > > > > > > > > > > > > > > > > > out
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > the assigned
> >> replica
> >> > > list
> >> > > > > and
> >> > > > > > > isr
> >> > > > > > > > in
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > > > response.
> >> > > > > > > > > > > > > > > > > As
> >> > > > > > > > > > > > > > > > > > an
> >> > > > > > > > > > > > > > > > > > > > > > > indication
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > doing that, we
> set
> >> the
> >> > > > > > partition
> >> > > > > > > > > level
> >> > > > > > > > > > > > error
> >> > > > > > > > > > > > > > > code
> >> > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > This has a few
> >> > problems.
> >> > > > > > First,
> >> > > > > > > > > > > > > > > > ReplicaNotAvailable
> >> > > > > > > > > > > > > > > > > > > > probably
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> shouldn't be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > an error, at
> least
> >> for
> >> > > the
> >> > > > > > > normal
> >> > > > > > > > > > > > > > > > producer/consumer
> >> > > > > > > > > > > > > > > > > > > > clients
> >> > > > > > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > just
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> want
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to find out the
> >> > leader.
> >> > > > > > Second,
> >> > > > > > > it
> >> > > > > > > > > can
> >> > > > > > > > > > > > > happen
> >> > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > both
> >> > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > leader
> >> > > > > > > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > another replica
> are
> >> > not
> >> > > > > > > available
> >> > > > > > > > at
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > same
> >> > > > > > > > > > > > > > > > time.
> >> > > > > > > > > > > > > > > > > > There
> >> > > > > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > no
> >> > > > > > > > > > > > > > > > > > > > > > > > > error
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> code
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to indicate both.
> >> > Third,
> >> > > > > even
> >> > > > > > > if a
> >> > > > > > > > > > > replica
> >> > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > not
> >> > > > > > > > > > > > > > > > > > > > available,
> >> > > > > > > > > > > > > > > > > > > > > > > it's
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> still
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > useful to return
> >> its
> >> > > > replica
> >> > > > > > id
> >> > > > > > > > > since
> >> > > > > > > > > > > some
> >> > > > > > > > > > > > > > > clients
> >> > > > > > > > > > > > > > > > > > (e.g.
> >> > > > > > > > > > > > > > > > > > > > > admin
> >> > > > > > > > > > > > > > > > > > > > > > > tool)
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> may
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > still make use of
> >> it.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > One way to
> address
> >> > this
> >> > > > > issue
> >> > > > > > is
> >> > > > > > > > to
> >> > > > > > > > > > > always
> >> > > > > > > > > > > > > > > return
> >> > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > replica
> >> > > > > > > > > > > > > > > > > > > > > > > id for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > leader, assigned
> >> > > replicas,
> >> > > > > and
> >> > > > > > > isr
> >> > > > > > > > > > > > > regardless
> >> > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > whether
> >> > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> corresponding
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > broker is live or
> >> not.
> >> > > > Since
> >> > > > > > we
> >> > > > > > > > also
> >> > > > > > > > > > > > return
> >> > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > list
> >> > > > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > > > live
> >> > > > > > > > > > > > > > > > > > > > > > > > > brokers,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > client can figure
> >> out
> >> > > > > whether
> >> > > > > > a
> >> > > > > > > > > leader
> >> > > > > > > > > > > or
> >> > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > replica
> >> > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > live
> >> > > > > > > > > > > > > > > > > > > > > or
> >> > > > > > > > > > > > > > > > > > > > > > > not
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> and act
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > accordingly. This
> >> way,
> >> > > we
> >> > > > > > don't
> >> > > > > > > > need
> >> > > > > > > > > > to
> >> > > > > > > > > > > > set
> >> > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > partition
> >> > > > > > > > > > > > > > > > > > > > > > level
> >> > > > > > > > > > > > > > > > > > > > > > > > > error
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> code
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > when the leader
> or
> >> a
> >> > > > replica
> >> > > > > > is
> >> > > > > > > > not
> >> > > > > > > > > > > > > available.
> >> > > > > > > > > > > > > > > > This
> >> > > > > > > > > > > > > > > > > > > > doesn't
> >> > > > > > > > > > > > > > > > > > > > > > > change
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> the wire
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > protocol, but
> does
> >> > > change
> >> > > > > the
> >> > > > > > > > > > semantics.
> >> > > > > > > > > > > > > Since
> >> > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > > > > evolving
> >> > > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> protocol
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > of
> >> > TopicMetadataRequest
> >> > > > > here,
> >> > > > > > we
> >> > > > > > > > can
> >> > > > > > > > > > > > > > potentially
> >> > > > > > > > > > > > > > > > > > piggyback
> >> > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > change.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 102.1 For those
> >> types
> >> > of
> >> > > > > > errors
> >> > > > > > > > due
> >> > > > > > > > > to
> >> > > > > > > > > > > > > invalid
> >> > > > > > > > > > > > > > > > > input,
> >> > > > > > > > > > > > > > > > > > > > > > shouldn't
> >> > > > > > > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > guard it at
> >> parameter
> >> > > > > > validation
> >> > > > > > > > > time
> >> > > > > > > > > > > and
> >> > > > > > > > > > > > > > throw
> >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> >> InvalidArgumentException
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > without even
> >> sending
> >> > the
> >> > > > > > request
> >> > > > > > > > to
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > > broker?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Jun
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16,
> >> 2015
> >> > at
> >> > > > > 10:37
> >> > > > > > > AM,
> >> > > > > > > > > > Andrii
> >> > > > > > > > > > > > > > > > Biletskyi <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > andrii.biletskyi@stealth.ly
> >> > > > > >
> >> > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Jun,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Answering your
> >> > > > questions:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 101. If I
> >> understand
> >> > > you
> >> > > > > > > > > correctly,
> >> > > > > > > > > > > you
> >> > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > saying
> >> > > > > > > > > > > > > > > > > > > > future
> >> > > > > > > > > > > > > > > > > > > > > > > producer
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > versions
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > (which
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will be ported
> to
> >> > > > TMR_V1)
> >> > > > > > > won't
> >> > > > > > > > be
> >> > > > > > > > > > > able
> >> > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > automatically
> >> > > > > > > > > > > > > > > > > > > > > > > create
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> topic (if
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > unconditionally
> >> > remove
> >> > > > > topic
> >> > > > > > > > > > creation
> >> > > > > > > > > > > > from
> >> > > > > > > > > > > > > > > > there).
> >> > > > > > > > > > > > > > > > > > But
> >> > > > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > need to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > preserve logic.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Ok, about your
> >> > > proposal:
> >> > > > > I'm
> >> > > > > > > > not a
> >> > > > > > > > > > big
> >> > > > > > > > > > > > fan
> >> > > > > > > > > > > > > > > too,
> >> > > > > > > > > > > > > > > > > > when it
> >> > > > > > > > > > > > > > > > > > > > > > comes
> >> > > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > differentiating
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > clients
> directly
> >> in
> >> > > > > protocol
> >> > > > > > > > > schema.
> >> > > > > > > > > > > And
> >> > > > > > > > > > > > > > also
> >> > > > > > > > > > > > > > > > I'm
> >> > > > > > > > > > > > > > > > > > not
> >> > > > > > > > > > > > > > > > > > > > > sure I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> understand
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > at
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > all why
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> >> > > > auto.create.topics.enable
> >> > > > > > is a
> >> > > > > > > > > > server
> >> > > > > > > > > > > > side
> >> > > > > > > > > > > > > > > > > > > > configuration.
> >> > > > > > > > > > > > > > > > > > > > > > Can
> >> > > > > > > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > deprecate
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > this setting
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in future
> >> versions,
> >> > > add
> >> > > > > this
> >> > > > > > > > > setting
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > > > producer
> >> > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > based
> >> > > > > > > > > > > > > > > > > > > > > > on
> >> > > > > > > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> upon
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > receiving
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > UnknownTopic
> >> create
> >> > > > topic
> >> > > > > > > > > explicitly
> >> > > > > > > > > > > by
> >> > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > separate
> >> > > > > > > > > > > > > > > > > > > > > producer
> >> > > > > > > > > > > > > > > > > > > > > > > call
> >> > > > > > > > > > > > > > > > > > > > > > > > > via
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > adminClient?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes.
> >> It's
> >> > > > > because
> >> > > > > > > we
> >> > > > > > > > > want
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > > > support
> >> > > > > > > > > > > > > > > > > > batching
> >> > > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > at
> >> > > > > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> same
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > time we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > want to give
> >> > > descriptive
> >> > > > > > error
> >> > > > > > > > > > > messages
> >> > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > clients.
> >> > > > > > > > > > > > > > > > > > > > Since
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> AdminClient
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > holds the
> context
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to construct
> such
> >> > > > messages
> >> > > > > > > (e.g.
> >> > > > > > > > > > > > > AdminClient
> >> > > > > > > > > > > > > > > > layer
> >> > > > > > > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > > > > > > know
> >> > > > > > > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> >> InvalidArgumentsCode
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > means two
> cases:
> >> > > either
> >> > > > > > > invalid
> >> > > > > > > > > > > number -
> >> > > > > > > > > > > > > > e.g.
> >> > > > > > > > > > > > > > > > -1;
> >> > > > > > > > > > > > > > > > > or
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> replication-factor
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > was
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > provided while
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > partitions
> >> argument
> >> > > > > wasn't)
> >> > > > > > -
> >> > > > > > > I
> >> > > > > > > > > > > wrapped
> >> > > > > > > > > > > > > > > > responses
> >> > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > Exceptions.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> But I'm
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > open to any
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > other ideas,
> this
> >> > was
> >> > > > just
> >> > > > > > > > initial
> >> > > > > > > > > > > > > version.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I
> >> agree.
> >> > > > I'll
> >> > > > > > > change
> >> > > > > > > > > > that
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > probably
> >> > > > > > > > > > > > > > > > > > some
> >> > > > > > > > > > > > > > > > > > > > > other
> >> > > > > > > > > > > > > > > > > > > > > > > dto.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Andrii
> Biletskyi
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13,
> >> 2015
> >> > > at
> >> > > > > 7:16
> >> > > > > > > PM,
> >> > > > > > > > > Jun
> >> > > > > > > > > > > > Rao <
> >> > > > > > > > > > > > > > > > > > > > > jun@confluent.io>
> >> > > > > > > > > > > > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 101. That's
> >> what I
> >> > > was
> >> > > > > > > > thinking
> >> > > > > > > > > > too,
> >> > > > > > > > > > > > but
> >> > > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > > may
> >> > > > > > > > > > > > > > > > > > not be
> >> > > > > > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> simple. In
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > TopicMetadataRequest_V1,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we can let it
> >> not
> >> > > > > trigger
> >> > > > > > > auto
> >> > > > > > > > > > topic
> >> > > > > > > > > > > > > > > creation.
> >> > > > > > > > > > > > > > > > > > Then,
> >> > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> producer
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > side,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > if it gets an
> >> > > > > > > > > > UnknownTopicException,
> >> > > > > > > > > > > > it
> >> > > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > > > explicitly
> >> > > > > > > > > > > > > > > > > > > > > > > issue a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> createTopicRequest
> >> > > for
> >> > > > > > auto
> >> > > > > > > > > topic
> >> > > > > > > > > > > > > > creation.
> >> > > > > > > > > > > > > > > On
> >> > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > consumer
> >> > > > > > > > > > > > > > > > > > > > > > > > > side,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > will
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > never issue
> >> > > > > > > > createTopicRequest.
> >> > > > > > > > > > This
> >> > > > > > > > > > > > > works
> >> > > > > > > > > > > > > > > > when
> >> > > > > > > > > > > > > > > > > > auto
> >> > > > > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> creation is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > enabled on
> the
> >> > > broker
> >> > > > > > side.
> >> > > > > > > > > > > However, I
> >> > > > > > > > > > > > > am
> >> > > > > > > > > > > > > > > not
> >> > > > > > > > > > > > > > > > > > sure how
> >> > > > > > > > > > > > > > > > > > > > > > > things
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> will work
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > when auto
> topic
> >> > > > creation
> >> > > > > > is
> >> > > > > > > > > > disabled
> >> > > > > > > > > > > > on
> >> > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > broker
> >> > > > > > > > > > > > > > > > > > > > side.
> >> > > > > > > > > > > > > > > > > > > > > > In
> >> > > > > > > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> case,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > want to have
> a
> >> way
> >> > > to
> >> > > > > > > manually
> >> > > > > > > > > > > create
> >> > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > topic,
> >> > > > > > > > > > > > > > > > > > > > > potentially
> >> > > > > > > > > > > > > > > > > > > > > > > > > through
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > commands.
> >> However,
> >> > > > then
> >> > > > > we
> >> > > > > > > > need
> >> > > > > > > > > a
> >> > > > > > > > > > > way
> >> > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > distinguish
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> createTopicRequest
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > issued from
> the
> >> > > > producer
> >> > > > > > > > clients
> >> > > > > > > > > > and
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > admin
> >> > > > > > > > > > > > > > > > > > tools.
> >> > > > > > > > > > > > > > > > > > > > > May
> >> > > > > > > > > > > > > > > > > > > > > > > be we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > new field in
> >> > > > > > > > createTopicRequest
> >> > > > > > > > > > and
> >> > > > > > > > > > > > set
> >> > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > > > > differently
> >> > > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> producer
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > client and
> the
> >> > admin
> >> > > > > > client.
> >> > > > > > > > > > > However,
> >> > > > > > > > > > > > I
> >> > > > > > > > > > > > > am
> >> > > > > > > > > > > > > > > not
> >> > > > > > > > > > > > > > > > > > sure if
> >> > > > > > > > > > > > > > > > > > > > > > > that's
> >> > > > > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> best
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > approach.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 2. Yes,
> >> > refactoring
> >> > > > > > existing
> >> > > > > > > > > > > requests
> >> > > > > > > > > > > > > is a
> >> > > > > > > > > > > > > > > > > > non-trivial
> >> > > > > > > > > > > > > > > > > > > > > > > amount of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> work.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > posted some
> >> > comments
> >> > > > in
> >> > > > > > > > > > KAFKA-1927.
> >> > > > > > > > > > > We
> >> > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > > probably
> >> > > > > > > > > > > > > > > > > > > > > have
> >> > > > > > > > > > > > > > > > > > > > > > > to fix
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > first, before
> >> > adding
> >> > > > the
> >> > > > > > new
> >> > > > > > > > > logic
> >> > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > KAFKA-1694.
> >> > > > > > > > > > > > > > > > > > > > > > > Otherwise, the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > changes
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > will be too
> >> big.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102. About
> the
> >> > > > > > AdminClient:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a
> >> bit
> >> > > > weird
> >> > > > > > that
> >> > > > > > > > we
> >> > > > > > > > > > > return
> >> > > > > > > > > > > > > > > > exception
> >> > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > api. It
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> seems
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we should
> >> either
> >> > > > return
> >> > > > > > > error
> >> > > > > > > > > code
> >> > > > > > > > > > > or
> >> > > > > > > > > > > > > > throw
> >> > > > > > > > > > > > > > > an
> >> > > > > > > > > > > > > > > > > > > > exception
> >> > > > > > > > > > > > > > > > > > > > > > > when
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> getting
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > response
> state.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.2. We
> >> probably
> >> > > > > > shouldn't
> >> > > > > > > > > > > > explicitly
> >> > > > > > > > > > > > > > use
> >> > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > request
> >> > > > > > > > > > > > > > > > > > > > > > > object
> >> > > > > > > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > api.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Not every
> >> request
> >> > > > > > evolution
> >> > > > > > > > > > requires
> >> > > > > > > > > > > > an
> >> > > > > > > > > > > > > > api
> >> > > > > > > > > > > > > > > > > > change.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Jun
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar
> 13,
> >> > 2015
> >> > > > at
> >> > > > > > 4:08
> >> > > > > > > > AM,
> >> > > > > > > > > > > > Andrii
> >> > > > > > > > > > > > > > > > > Biletskyi
> >> > > > > > > > > > > > > > > > > > <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > > > andrii.biletskyi@stealth.ly
> >> > > > > > > >
> >> > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks for
> >> you
> >> > > > > comments.
> >> > > > > > > > > Answers
> >> > > > > > > > > > > > > inline:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 100. There
> >> are a
> >> > > few
> >> > > > > > > fields
> >> > > > > > > > > such
> >> > > > > > > > > > > as
> >> > > > > > > > > > > > > > > > > > > > ReplicaAssignment,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > ReassignPartitionRequest,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> >> > > > > > PartitionsSerialized
> >> > > > > > > > > that
> >> > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > represented
> >> > > > > > > > > > > > > > > > > > as a
> >> > > > > > > > > > > > > > > > > > > > > > > string,
> >> > > > > > > > > > > > > > > > > > > > > > > > > but
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
> >> > > > structures
> >> > > > > > in
> >> > > > > > > > > json.
> >> > > > > > > > > > > > Could
> >> > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > flatten
> >> > > > > > > > > > > > > > > > > > > > them
> >> > > > > > > > > > > > > > > > > > > > > > out
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> directly in
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
> >> > > > definition
> >> > > > > as
> >> > > > > > > > > > > > > arrays/records?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, now
> with
> >> > > Admin
> >> > > > > > Client
> >> > > > > > > > > this
> >> > > > > > > > > > > > looks
> >> > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > bit
> >> > > > > > > > > > > > > > > > > > weird.
> >> > > > > > > > > > > > > > > > > > > > My
> >> > > > > > > > > > > > > > > > > > > > > > > initial
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > motivation
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > was:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > ReassignPartitionCommand
> >> > > > > > > > > accepts
> >> > > > > > > > > > > > input
> >> > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > json,
> >> > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > want
> >> > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> remain
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > tools'
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > interfaces
> >> > > > unchanged,
> >> > > > > > > where
> >> > > > > > > > > > > > possible.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > If we port
> >> it to
> >> > > > > > > > deserialized
> >> > > > > > > > > > > > format,
> >> > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > CLI
> >> > > > > > > > > > > > > > > > > > (/tools
> >> > > > > > > > > > > > > > > > > > > > > > > project)
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> we will
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > have
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to add some
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json
> library
> >> > since
> >> > > > > > /tools
> >> > > > > > > is
> >> > > > > > > > > > > written
> >> > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > java
> >> > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > we'll
> >> > > > > > > > > > > > > > > > > > > > > > > need to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > deserialize
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json file
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > provided
> by a
> >> > > user.
> >> > > > > Can
> >> > > > > > we
> >> > > > > > > > > > quickly
> >> > > > > > > > > > > > > agree
> >> > > > > > > > > > > > > > > on
> >> > > > > > > > > > > > > > > > > what
> >> > > > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > library
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> should
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > (Jackson,
> >> GSON,
> >> > > > > > whatever)?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> >> > > > > > > > TopicMetadataRequest
> >> > > > > > > > > > v1
> >> > > > > > > > > > > > > still
> >> > > > > > > > > > > > > > > > > trigger
> >> > > > > > > > > > > > > > > > > > auto
> >> > > > > > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> creation?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > This
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a
> >> bit
> >> > > > weird
> >> > > > > > now
> >> > > > > > > > that
> >> > > > > > > > > > we
> >> > > > > > > > > > > > > have a
> >> > > > > > > > > > > > > > > > > > separate
> >> > > > > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> creation api.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you
> thought
> >> > > about
> >> > > > > how
> >> > > > > > > the
> >> > > > > > > > > new
> >> > > > > > > > > > > > > > > > > > createTopicRequest
> >> > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > TopicMetadataRequest
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will
> be
> >> > used
> >> > > in
> >> > > > > the
> >> > > > > > > > > > > > > > producer/consumer
> >> > > > > > > > > > > > > > > > > > client,
> >> > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > addition
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools?
> For
> >> > > > example,
> >> > > > > > > > ideally,
> >> > > > > > > > > > we
> >> > > > > > > > > > > > > don't
> >> > > > > > > > > > > > > > > want
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > from
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer
> to
> >> > > > trigger
> >> > > > > > auto
> >> > > > > > > > > topic
> >> > > > > > > > > > > > > > creation.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > I agree,
> this
> >> > > > strange
> >> > > > > > > logic
> >> > > > > > > > > > should
> >> > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > fixed.
> >> > > > > > > > > > > > > > > > > > I'm not
> >> > > > > > > > > > > > > > > > > > > > > > > confident
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> in
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka part
> so
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > correct me
> if
> >> > I'm
> >> > > > > wrong,
> >> > > > > > > but
> >> > > > > > > > > it
> >> > > > > > > > > > > > > doesn't
> >> > > > > > > > > > > > > > > look
> >> > > > > > > > > > > > > > > > > > like a
> >> > > > > > > > > > > > > > > > > > > > > hard
> >> > > > > > > > > > > > > > > > > > > > > > > thing
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > do, I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > think we
> can
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > leverage
> >> > > AdminClient
> >> > > > > for
> >> > > > > > > > that
> >> > > > > > > > > in
> >> > > > > > > > > > > > > > Producer
> >> > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > unconditionally
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> remove
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > creation
> from
> >> > the
> >> > > > > > > > > > > > > > TopicMetadataRequest_V1.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 2. I think
> >> Jay
> >> > > meant
> >> > > > > > > getting
> >> > > > > > > > > rid
> >> > > > > > > > > > > of
> >> > > > > > > > > > > > > > scala
> >> > > > > > > > > > > > > > > > > > classes
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> >> > > > > > > > > HeartbeatRequestAndHeader
> >> > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> >> > HeartbeatResponseAndHeader.
> >> > > We
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > did
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a
> >> stop-gap
> >> > > > thing
> >> > > > > > when
> >> > > > > > > > > > adding
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > new
> >> > > > > > > > > > > > > > > > > > requests
> >> > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> consumers.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However,
> >> the
> >> > > long
> >> > > > > term
> >> > > > > > > > plan
> >> > > > > > > > > is
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > get
> >> > > > > > > > > > > > > > > rid
> >> > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > all
> >> > > > > > > > > > > > > > > > > > > > > those
> >> > > > > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> >> > > > > request/response
> >> > > > > > in
> >> > > > > > > > the
> >> > > > > > > > > > > > client.
> >> > > > > > > > > > > > > > > Since
> >> > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > KIP
> >> > > > > > > > > > > > > > > > > > > > > > > proposes
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> significant
> >> > > number
> >> > > > > of
> >> > > > > > > new
> >> > > > > > > > > > > > requests,
> >> > > > > > > > > > > > > > > > perhaps
> >> > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > should
> >> > > > > > > > > > > > > > > > > > > > > > > bite
> >> > > > > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up
> >> the
> >> > > > > existing
> >> > > > > > > > scala
> >> > > > > > > > > > > > requests
> >> > > > > > > > > > > > > > > first
> >> > > > > > > > > > > > > > > > > > before
> >> > > > > > > > > > > > > > > > > > > > > > > adding new
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> ones?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks
> >> like
> >> > I
> >> > > > > > > > > misunderstood
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > > point
> >> > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Okay, I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > will
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > rework
> that.
> >> The
> >> > > > only
> >> > > > > > > thing
> >> > > > > > > > is
> >> > > > > > > > > > > that
> >> > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > don't
> >> > > > > > > > > > > > > > > > > see
> >> > > > > > > > > > > > > > > > > > any
> >> > > > > > > > > > > > > > > > > > > > > > > example
> >> > > > > > > > > > > > > > > > > > > > > > > > > how
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > was
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > done
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > for at
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > least one
> >> > existing
> >> > > > > > > protocol
> >> > > > > > > > > > > message.
> >> > > > > > > > > > > > > > Thus,
> >> > > > > > > > > > > > > > > > as
> >> > > > > > > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > > > > > > > > > understand, I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> have to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > think
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > how we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are going
> to
> >> do
> >> > > it.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Re porting
> >> all
> >> > > > > existing
> >> > > > > > > > RQ/RP
> >> > > > > > > > > in
> >> > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > patch.
> >> > > > > > > > > > > > > > > > > > Sounds
> >> > > > > > > > > > > > > > > > > > > > > > > > > reasonable,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> but
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > if
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > it's
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > an
> >> *obligatory*
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > requirement
> >> to
> >> > > have
> >> > > > > > Admin
> >> > > > > > > > KIP
> >> > > > > > > > > > > done,
> >> > > > > > > > > > > > > I'm
> >> > > > > > > > > > > > > > > > afraid
> >> > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > > > > > > > > be a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> serious
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > blocker for
> >> us.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > There are
> 13
> >> > > > protocol
> >> > > > > > > > messages
> >> > > > > > > > > > and
> >> > > > > > > > > > > > all
> >> > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > would
> >> > > > > > > > > > > > > > > > > > > > > > require
> >> > > > > > > > > > > > > > > > > > > > > > > not
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> only
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > unit
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > tests but
> >> quite
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > intensive
> >> manual
> >> > > > > > testing,
> >> > > > > > > > no?
> >> > > > > > > > > > I'm
> >> > > > > > > > > > > > > afraid
> >> > > > > > > > > > > > > > > I'm
> >> > > > > > > > > > > > > > > > > > not the
> >> > > > > > > > > > > > > > > > > > > > > > > right guy
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > cover
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > pretty much
> >> all
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka core
> >> > > internals
> >> > > > > :).
> >> > > > > > > Let
> >> > > > > > > > > me
> >> > > > > > > > > > > know
> >> > > > > > > > > > > > > > your
> >> > > > > > > > > > > > > > > > > > thoughts
> >> > > > > > > > > > > > > > > > > > > > on
> >> > > > > > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> item. Btw
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > there
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is a ticket
> >> to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > follow-up
> >> this
> >> > > > issue (
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > >
> >> https://issues.apache.org/jira/browse/KAFKA-2006
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > ).
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Andrii
> >> Biletskyi
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar
> >> 13,
> >> > > 2015
> >> > > > > at
> >> > > > > > > 6:40
> >> > > > > > > > > AM,
> >> > > > > > > > > > > Jun
> >> > > > > > > > > > > > > > Rao <
> >> > > > > > > > > > > > > > > > > > > > > > > jun@confluent.io>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > A few
> more
> >> > > > comments.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 100.
> There
> >> > are a
> >> > > > few
> >> > > > > > > > fields
> >> > > > > > > > > > such
> >> > > > > > > > > > > > as
> >> > > > > > > > > > > > > > > > > > > > > ReplicaAssignment,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > ReassignPartitionRequest,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> >> > > > > > PartitionsSerialized
> >> > > > > > > > > that
> >> > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > represented
> >> > > > > > > > > > > > > > > > > > as a
> >> > > > > > > > > > > > > > > > > > > > > > > string,
> >> > > > > > > > > > > > > > > > > > > > > > > > > but
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
> >> > > > structures
> >> > > > > > in
> >> > > > > > > > > json.
> >> > > > > > > > > > > > Could
> >> > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > flatten
> >> > > > > > > > > > > > > > > > > > > > them
> >> > > > > > > > > > > > > > > > > > > > > > out
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> directly in
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
> >> > > > definition
> >> > > > > as
> >> > > > > > > > > > > > > arrays/records?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> >> > > > > > > > > TopicMetadataRequest
> >> > > > > > > > > > > v1
> >> > > > > > > > > > > > > > still
> >> > > > > > > > > > > > > > > > > > trigger
> >> > > > > > > > > > > > > > > > > > > > auto
> >> > > > > > > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > creation?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > This
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a
> >> bit
> >> > > > weird
> >> > > > > > now
> >> > > > > > > > that
> >> > > > > > > > > > we
> >> > > > > > > > > > > > > have a
> >> > > > > > > > > > > > > > > > > > separate
> >> > > > > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> creation api.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you
> thought
> >> > > about
> >> > > > > how
> >> > > > > > > the
> >> > > > > > > > > new
> >> > > > > > > > > > > > > > > > > > createTopicRequest
> >> > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > TopicMetadataRequest
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will
> be
> >> > used
> >> > > in
> >> > > > > the
> >> > > > > > > > > > > > > > producer/consumer
> >> > > > > > > > > > > > > > > > > > client,
> >> > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > addition
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools?
> For
> >> > > > example,
> >> > > > > > > > ideally,
> >> > > > > > > > > > we
> >> > > > > > > > > > > > > don't
> >> > > > > > > > > > > > > > > want
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > from
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer
> to
> >> > > > trigger
> >> > > > > > auto
> >> > > > > > > > > topic
> >> > > > > > > > > > > > > > creation.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 2. I
> think
> >> Jay
> >> > > > meant
> >> > > > > > > > getting
> >> > > > > > > > > > rid
> >> > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > scala
> >> > > > > > > > > > > > > > > > > > classes
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> >> > > > > > > > > HeartbeatRequestAndHeader
> >> > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> >> > HeartbeatResponseAndHeader.
> >> > > We
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > did
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a
> >> stop-gap
> >> > > > thing
> >> > > > > > when
> >> > > > > > > > > > adding
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > new
> >> > > > > > > > > > > > > > > > > > requests
> >> > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> consumers.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However,
> >> the
> >> > > long
> >> > > > > term
> >> > > > > > > > plan
> >> > > > > > > > > is
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > get
> >> > > > > > > > > > > > > > > rid
> >> > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > all
> >> > > > > > > > > > > > > > > > > > > > > those
> >> > > > > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> >> > > > > request/response
> >> > > > > > in
> >> > > > > > > > the
> >> > > > > > > > > > > > client.
> >> > > > > > > > > > > > > > > Since
> >> > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > KIP
> >> > > > > > > > > > > > > > > > > > > > > > > proposes
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> significant
> >> > > number
> >> > > > > of
> >> > > > > > > new
> >> > > > > > > > > > > > requests,
> >> > > > > > > > > > > > > > > > perhaps
> >> > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > should
> >> > > > > > > > > > > > > > > > > > > > > > > bite
> >> > > > > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up
> >> the
> >> > > > > existing
> >> > > > > > > > scala
> >> > > > > > > > > > > > requests
> >> > > > > > > > > > > > > > > first
> >> > > > > > > > > > > > > > > > > > before
> >> > > > > > > > > > > > > > > > > > > > > > > adding new
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> ones?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > On Thu,
> Mar
> >> > 12,
> >> > > > 2015
> >> > > > > > at
> >> > > > > > > > 3:37
> >> > > > > > > > > > PM,
> >> > > > > > > > > > > > > > Andrii
> >> > > > > > > > > > > > > > > > > > Biletskyi
> >> > > > > > > > > > > > > > > > > > > > <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > andrii.biletskyi@stealth.ly
> >> > > > > > > > > >
> >> > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > As said
> >> > above
> >> > > -
> >> > > > I
> >> > > > > > list
> >> > > > > > > > > again
> >> > > > > > > > > > > all
> >> > > > > > > > > > > > > > > > comments
> >> > > > > > > > > > > > > > > > > > from
> >> > > > > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > thread
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> so we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can see
> >> > what's
> >> > > > > left
> >> > > > > > > and
> >> > > > > > > > > > > finalize
> >> > > > > > > > > > > > > all
> >> > > > > > > > > > > > > > > > > pending
> >> > > > > > > > > > > > > > > > > > > > > issues.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> Comments
> >> > from
> >> > > > Jay:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This
> >> is
> >> > > much
> >> > > > > > needed
> >> > > > > > > > > > > > > > functionality,
> >> > > > > > > > > > > > > > > > but
> >> > > > > > > > > > > > > > > > > > there
> >> > > > > > > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > > > > > > > > lot
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> of the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > so
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > let's
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > really
> >> think
> >> > > > these
> >> > > > > > > > > protocols
> >> > > > > > > > > > > > > > through.
> >> > > > > > > > > > > > > > > We
> >> > > > > > > > > > > > > > > > > > really
> >> > > > > > > > > > > > > > > > > > > > > want
> >> > > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > end
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> up
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > with a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > set
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > of well
> >> > > > > thought-out,
> >> > > > > > > > > > > orthoganol
> >> > > > > > > > > > > > > > apis.
> >> > > > > > > > > > > > > > > > For
> >> > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > reason
> >> > > > > > > > > > > > > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> think it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > really
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> important to
> >> > > > think
> >> > > > > > > > through
> >> > > > > > > > > > the
> >> > > > > > > > > > > > end
> >> > > > > > > > > > > > > > > state
> >> > > > > > > > > > > > > > > > > > even if
> >> > > > > > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> includes
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > APIs
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > won't
> >> > > implement
> >> > > > in
> >> > > > > > the
> >> > > > > > > > > first
> >> > > > > > > > > > > > > phase.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> >> > Definitely
> >> > > > > behind
> >> > > > > > > > this.
> >> > > > > > > > > > > Would
> >> > > > > > > > > > > > > > > > > appreciate
> >> > > > > > > > > > > > > > > > > > if
> >> > > > > > > > > > > > > > > > > > > > > there
> >> > > > > > > > > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> concrete
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > comments
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > how
> this
> >> can
> >> > > be
> >> > > > > > > > improved.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 2.
> Let's
> >> > > please
> >> > > > > > please
> >> > > > > > > > > > please
> >> > > > > > > > > > > > wait
> >> > > > > > > > > > > > > > > until
> >> > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > have
> >> > > > > > > > > > > > > > > > > > > > > > > switched
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > server
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > over
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to the
> >> new
> >> > > java
> >> > > > > > > protocol
> >> > > > > > > > > > > > > > definitions.
> >> > > > > > > > > > > > > > > If
> >> > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > add
> >> > > > > > > > > > > > > > > > > > > > > > upteen
> >> > > > > > > > > > > > > > > > > > > > > > > > > more
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> ad
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > hoc
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > scala
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > objects
> >> that
> >> > > is
> >> > > > > just
> >> > > > > > > > > > > generating
> >> > > > > > > > > > > > > more
> >> > > > > > > > > > > > > > > > work
> >> > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> conversion we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > know
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > have to
> >> do.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> Fixed
> >> in
> >> > > the
> >> > > > > > latest
> >> > > > > > > > > > patch -
> >> > > > > > > > > > > > > > removed
> >> > > > > > > > > > > > > > > > > scala
> >> > > > > > > > > > > > > > > > > > > > > > protocol
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> classes.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This
> >> > > proposal
> >> > > > > > > > > introduces
> >> > > > > > > > > > a
> >> > > > > > > > > > > > new
> >> > > > > > > > > > > > > > type
> >> > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > > > optional
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> parameter.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > This
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> inconsistent
> >> > > > with
> >> > > > > > > > > everything
> >> > > > > > > > > > > > else
> >> > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > protocol
> >> > > > > > > > > > > > > > > > > > > > > > > where we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> use -1
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > or
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > some
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > other
> >> marker
> >> > > > > value.
> >> > > > > > > You
> >> > > > > > > > > > could
> >> > > > > > > > > > > > > argue
> >> > > > > > > > > > > > > > > > either
> >> > > > > > > > > > > > > > > > > > way
> >> > > > > > > > > > > > > > > > > > > > but
> >> > > > > > > > > > > > > > > > > > > > > > > let's
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> stick
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > with
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > for
> >> > > consistency.
> >> > > > > For
> >> > > > > > > > > clients
> >> > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > implemented
> >> > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > protocol
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> in a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > better
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > way
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
> our
> >> > scala
> >> > > > > code
> >> > > > > > > > these
> >> > > > > > > > > > > basic
> >> > > > > > > > > > > > > > > > primitives
> >> > > > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > > > > hard
> >> > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> change.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> Fixed
> >> in
> >> > > the
> >> > > > > > latest
> >> > > > > > > > > > patch -
> >> > > > > > > > > > > > > > removed
> >> > > > > > > > > > > > > > > > > > MaybeOf
> >> > > > > > > > > > > > > > > > > > > > > type
> >> > > > > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> changed
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > protocol
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> accordingly.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 4.
> >> > > > > ClusterMetadata:
> >> > > > > > > This
> >> > > > > > > > > > seems
> >> > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > duplicate
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > which
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> brokers,
> >> > > topics,
> >> > > > > and
> >> > > > > > > > > > > > partitions. I
> >> > > > > > > > > > > > > > > think
> >> > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > should
> >> > > > > > > > > > > > > > > > > > > > > > > rename
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > request
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > ClusterMetadataRequest
> >> > > > > > > > (or
> >> > > > > > > > > > > just
> >> > > > > > > > > > > > > > > > > > MetadataRequest)
> >> > > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> include the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > id
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> controller.
> >> > Or
> >> > > > are
> >> > > > > > > there
> >> > > > > > > > > > other
> >> > > > > > > > > > > > > > things
> >> > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > could
> >> > > > > > > > > > > > > > > > > > > > add
> >> > > > > > > > > > > > > > > > > > > > > > > here?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I
> >> agree.
> >> > > > > Updated
> >> > > > > > > the
> >> > > > > > > > > KIP.
> >> > > > > > > > > > > > Let's
> >> > > > > > > > > > > > > > > > extends
> >> > > > > > > > > > > > > > > > > > > > > > > TopicMetadata
> >> > > > > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > version 2
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > include
> >> > > > > controller.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We
> >> have a
> >> > > > > > tendency
> >> > > > > > > to
> >> > > > > > > > > try
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > > make a
> >> > > > > > > > > > > > > > > > lot
> >> > > > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > > > > > requests
> >> > > > > > > > > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > only
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > go
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> particular
> >> > > > nodes.
> >> > > > > > This
> >> > > > > > > > > adds
> >> > > > > > > > > > a
> >> > > > > > > > > > > > lot
> >> > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > burden
> >> > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > client
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> implementations
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > (it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > sounds
> >> easy
> >> > > but
> >> > > > > each
> >> > > > > > > > > > discovery
> >> > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > fail
> >> > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > many
> >> > > > > > > > > > > > > > > > > > > > > > parts
> >> > > > > > > > > > > > > > > > > > > > > > > so
> >> > > > > > > > > > > > > > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> ends
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > up
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > being a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > full
> >> state
> >> > > > machine
> >> > > > > > to
> >> > > > > > > do
> >> > > > > > > > > > > > right). I
> >> > > > > > > > > > > > > > > think
> >> > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > should
> >> > > > > > > > > > > > > > > > > > > > > > > > > consider
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > making
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > admin
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> commands
> >> and
> >> > > > > ideally
> >> > > > > > > as
> >> > > > > > > > > many
> >> > > > > > > > > > > of
> >> > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > other
> >> > > > > > > > > > > > > > > > > > apis
> >> > > > > > > > > > > > > > > > > > > > as
> >> > > > > > > > > > > > > > > > > > > > > > > possible
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > available
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > on
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > all
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers
> >> and
> >> > > just
> >> > > > > > > > redirect
> >> > > > > > > > > to
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > controller
> >> > > > > > > > > > > > > > > > > > on
> >> > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > broker
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> side.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > there
> >> would
> >> > > be a
> >> > > > > > > general
> >> > > > > > > > > way
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > encapsulate
> >> > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > re-routing
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > behavior.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> It's a
> >> > very
> >> > > > > > > > interesting
> >> > > > > > > > > > > idea,
> >> > > > > > > > > > > > > but
> >> > > > > > > > > > > > > > > > seems
> >> > > > > > > > > > > > > > > > > > there
> >> > > > > > > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > > > > > > some
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > concerns
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > about
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > feature
> >> > (like
> >> > > > > > > > performance
> >> > > > > > > > > > > > > > > > considerations,
> >> > > > > > > > > > > > > > > > > > how
> >> > > > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > complicate
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > server
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I
> believe
> >> > this
> >> > > > > > > shouldn't
> >> > > > > > > > > be
> >> > > > > > > > > > a
> >> > > > > > > > > > > > > > blocker.
> >> > > > > > > > > > > > > > > > If
> >> > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > feature is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > implemented
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > at
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > some
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point
> it
> >> > won't
> >> > > > > > affect
> >> > > > > > > > > Admin
> >> > > > > > > > > > > > > changes
> >> > > > > > > > > > > > > > -
> >> > > > > > > > > > > > > > > at
> >> > > > > > > > > > > > > > > > > > least
> >> > > > > > > > > > > > > > > > > > > > no
> >> > > > > > > > > > > > > > > > > > > > > > > changes
> >> > > > > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > public
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > API
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
> >> > > > required.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We
> >> should
> >> > > > > > probably
> >> > > > > > > > > > > normalize
> >> > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > key
> >> > > > > > > > > > > > > > > > > > value
> >> > > > > > > > > > > > > > > > > > > > > pairs
> >> > > > > > > > > > > > > > > > > > > > > > > used
> >> > > > > > > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > configs
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
> >> > > embedding a
> >> > > > > new
> >> > > > > > > > > > > formatting.
> >> > > > > > > > > > > > > So
> >> > > > > > > > > > > > > > > two
> >> > > > > > > > > > > > > > > > > > strings
> >> > > > > > > > > > > > > > > > > > > > > > rather
> >> > > > > > > > > > > > > > > > > > > > > > > > > than
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> one
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > with
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > an
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> internal
> >> > > equals
> >> > > > > > sign.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> Fixed
> >> in
> >> > > the
> >> > > > > > latest
> >> > > > > > > > > > patch -
> >> > > > > > > > > > > > > > > > normalized
> >> > > > > > > > > > > > > > > > > > > > configs
> >> > > > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> changed
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > protocol
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> accordingly.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is
> the
> >> > > > > > > postcondition
> >> > > > > > > > of
> >> > > > > > > > > > > these
> >> > > > > > > > > > > > > > APIs
> >> > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > command has
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> begun
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > or
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> >> command
> >> > > has
> >> > > > > been
> >> > > > > > > > > > > completed?
> >> > > > > > > > > > > > It
> >> > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > > lot
> >> > > > > > > > > > > > > > > > > > more
> >> > > > > > > > > > > > > > > > > > > > > > > usable if
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > been
> >> > completed
> >> > > > so
> >> > > > > > you
> >> > > > > > > > know
> >> > > > > > > > > > > that
> >> > > > > > > > > > > > if
> >> > > > > > > > > > > > > > you
> >> > > > > > > > > > > > > > > > > > create a
> >> > > > > > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> then
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > publish
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it you
> >> won't
> >> > > get
> >> > > > > an
> >> > > > > > > > > > exception
> >> > > > > > > > > > > > > about
> >> > > > > > > > > > > > > > > > there
> >> > > > > > > > > > > > > > > > > > being
> >> > > > > > > > > > > > > > > > > > > > no
> >> > > > > > > > > > > > > > > > > > > > > > > such
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> topic.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: For
> >> long
> >> > > > > running
> >> > > > > > > > > requests
> >> > > > > > > > > > > > (like
> >> > > > > > > > > > > > > > > > > reassign
> >> > > > > > > > > > > > > > > > > > > > > > > partitions) -
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > post
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> condition is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
> >> has
> >> > > > begun
> >> > > > > -
> >> > > > > > so
> >> > > > > > > > we
> >> > > > > > > > > > > don't
> >> > > > > > > > > > > > > > block
> >> > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > client.
> >> > > > > > > > > > > > > > > > > > > > > In
> >> > > > > > > > > > > > > > > > > > > > > > > case
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> of your
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > example -
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
> >> > > commands,
> >> > > > > this
> >> > > > > > > > will
> >> > > > > > > > > be
> >> > > > > > > > > > > > > > > refactored
> >> > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > commands
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> will
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > executed
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> immediately,
> >> > > > since
> >> > > > > > the
> >> > > > > > > > > > > > Controller
> >> > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > serve
> >> > > > > > > > > > > > > > > > > > > > Admin
> >> > > > > > > > > > > > > > > > > > > > > > > > > requests
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> (follow-up
> >> > > > ticket
> >> > > > > > > > > > KAFKA-1777).
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 8.
> >> Describe
> >> > > > topic
> >> > > > > > and
> >> > > > > > > > list
> >> > > > > > > > > > > > topics
> >> > > > > > > > > > > > > > > > > duplicate
> >> > > > > > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > > > > > lot
> >> > > > > > > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > > > > > > stuff
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> in the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > metadata
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> request.
> >> Is
> >> > > > there
> >> > > > > a
> >> > > > > > > > reason
> >> > > > > > > > > > to
> >> > > > > > > > > > > > give
> >> > > > > > > > > > > > > > > back
> >> > > > > > > > > > > > > > > > > > topics
> >> > > > > > > > > > > > > > > > > > > > > > marked
> >> > > > > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > deletion? I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > feel
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like if
> >> we
> >> > > just
> >> > > > > make
> >> > > > > > > the
> >> > > > > > > > > > > > > > > post-condition
> >> > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > delete
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> command be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
> is
> >> > > deleted
> >> > > > > > that
> >> > > > > > > > will
> >> > > > > > > > > > get
> >> > > > > > > > > > > > rid
> >> > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > need
> >> > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> right?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > And
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be much
> >> more
> >> > > > > > > intuitive.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> Fixed
> >> in
> >> > > the
> >> > > > > > latest
> >> > > > > > > > > > patch -
> >> > > > > > > > > > > > > > removed
> >> > > > > > > > > > > > > > > > > > topics
> >> > > > > > > > > > > > > > > > > > > > > marked
> >> > > > > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> deletion
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > ListTopicsRequest.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 9.
> >> Should we
> >> > > > > > consider
> >> > > > > > > > > > batching
> >> > > > > > > > > > > > > these
> >> > > > > > > > > > > > > > > > > > requests?
> >> > > > > > > > > > > > > > > > > > > > We
> >> > > > > > > > > > > > > > > > > > > > > > have
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> generally
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > tried
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > allow
> >> > multiple
> >> > > > > > > > operations
> >> > > > > > > > > to
> >> > > > > > > > > > > be
> >> > > > > > > > > > > > > > > batched.
> >> > > > > > > > > > > > > > > > > My
> >> > > > > > > > > > > > > > > > > > > > > > suspicion
> >> > > > > > > > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > without
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we will
> >> get
> >> > a
> >> > > > lot
> >> > > > > of
> >> > > > > > > > code
> >> > > > > > > > > > that
> >> > > > > > > > > > > > > does
> >> > > > > > > > > > > > > > > > > > something
> >> > > > > > > > > > > > > > > > > > > > like
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > for(topic:
> >> > > > > > > > > > > > > > > adminClient.listTopics())
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > >  adminClient.describeTopic(topic)
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> code
> >> > will
> >> > > > > work
> >> > > > > > > > great
> >> > > > > > > > > > when
> >> > > > > > > > > > > > you
> >> > > > > > > > > > > > > > > test
> >> > > > > > > > > > > > > > > > > on 5
> >> > > > > > > > > > > > > > > > > > > > > topics
> >> > > > > > > > > > > > > > > > > > > > > > > but
> >> > > > > > > > > > > > > > > > > > > > > > > > > not
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> do as
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > well
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > if
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > you
> have
> >> > 50k.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> >> Updated
> >> > the
> >> > > > > KIP -
> >> > > > > > > > > please
> >> > > > > > > > > > > > check
> >> > > > > > > > > > > > > > > "Topic
> >> > > > > > > > > > > > > > > > > > Admin
> >> > > > > > > > > > > > > > > > > > > > > > Schema"
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> section.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I
> >> think
> >> > we
> >> > > > > > should
> >> > > > > > > > also
> >> > > > > > > > > > > > discuss
> >> > > > > > > > > > > > > > how
> >> > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > want
> >> > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > expose a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > programmatic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > client
> >> api
> >> > for
> >> > > > > these
> >> > > > > > > > > > > operations.
> >> > > > > > > > > > > > > > > > Currently
> >> > > > > > > > > > > > > > > > > > > > people
> >> > > > > > > > > > > > > > > > > > > > > > > rely on
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > which
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > is
> >> totally
> >> > > > > sketchy.
> >> > > > > > I
> >> > > > > > > > > think
> >> > > > > > > > > > we
> >> > > > > > > > > > > > > > > probably
> >> > > > > > > > > > > > > > > > > need
> >> > > > > > > > > > > > > > > > > > > > > another
> >> > > > > > > > > > > > > > > > > > > > > > > > > client
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> under
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clients/
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > that
> >> exposes
> >> > > > > > > > > administrative
> >> > > > > > > > > > > > > > > > functionality.
> >> > > > > > > > > > > > > > > > > > We
> >> > > > > > > > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > > > > > > > need
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> this just
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> properly
> >> > test
> >> > > > the
> >> > > > > > new
> >> > > > > > > > > apis,
> >> > > > > > > > > > I
> >> > > > > > > > > > > > > > suspect.
> >> > > > > > > > > > > > > > > > We
> >> > > > > > > > > > > > > > > > > > should
> >> > > > > > > > > > > > > > > > > > > > > > > figure
> >> > > > > > > > > > > > > > > > > > > > > > > > > out
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > API.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> >> Updated
> >> > the
> >> > > > > KIP -
> >> > > > > > > > > please
> >> > > > > > > > > > > > check
> >> > > > > > > > > > > > > > > "Admin
> >> > > > > > > > > > > > > > > > > > Client"
> >> > > > > > > > > > > > > > > > > > > > > > > section
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> with an
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > initial
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > API
> >> > proposal.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The
> >> > other
> >> > > > > > > > information
> >> > > > > > > > > > that
> >> > > > > > > > > > > > > would
> >> > > > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > > really
> >> > > > > > > > > > > > > > > > > > > > > > useful
> >> > > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > get
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > would
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> information
> >> > > > about
> >> > > > > > > > > > > > partitions--how
> >> > > > > > > > > > > > > > much
> >> > > > > > > > > > > > > > > > > data
> >> > > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> partition,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > what
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> >> segment
> >> > > > > offsets,
> >> > > > > > > > what
> >> > > > > > > > > is
> >> > > > > > > > > > > the
> >> > > > > > > > > > > > > > > log-end
> >> > > > > > > > > > > > > > > > > > offset
> >> > > > > > > > > > > > > > > > > > > > > > (i.e.
> >> > > > > > > > > > > > > > > > > > > > > > > last
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > offset),
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > what
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> >> > compaction
> >> > > > > > point,
> >> > > > > > > > > etc. I
> >> > > > > > > > > > > > think
> >> > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > done
> >> > > > > > > > > > > > > > > > > > > > right
> >> > > > > > > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> would be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> successor to
> >> > > the
> >> > > > > > very
> >> > > > > > > > > > awkward
> >> > > > > > > > > > > > > > > > > OffsetRequest
> >> > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > have
> >> > > > > > > > > > > > > > > > > > > > > > > today.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I
> >> removed
> >> > > > > > > > > > > > > > > ConsumerGroupOffsetsRequest
> >> > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > latest
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> patch. I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > believe
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> >> should
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be
> >> resolved
> >> > > in a
> >> > > > > > > > separate
> >> > > > > > > > > > KIP
> >> > > > > > > > > > > /
> >> > > > > > > > > > > > > jira
> >> > > > > > > > > > > > > > > > > ticket.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 12.
> >> > Generally
> >> > > we
> >> > > > > can
> >> > > > > > > do
> >> > > > > > > > > good
> >> > > > > > > > > > > > error
> >> > > > > > > > > > > > > > > > > handling
> >> > > > > > > > > > > > > > > > > > > > > without
> >> > > > > > > > > > > > > > > > > > > > > > > > > needing
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > custom
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> server-side
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> messages.
> >> > I.e.
> >> > > > > > > generally
> >> > > > > > > > > the
> >> > > > > > > > > > > > > client
> >> > > > > > > > > > > > > > > has
> >> > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > context
> >> > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > know
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > if
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > got
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > an
> error
> >> > that
> >> > > > the
> >> > > > > > > topic
> >> > > > > > > > > > > doesn't
> >> > > > > > > > > > > > > > exist
> >> > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > say
> >> > > > > > > > > > > > > > > > > > > > > "Topic
> >> > > > > > > > > > > > > > > > > > > > > > X
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> doesn't
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > exist"
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
> >> "error
> >> > > code
> >> > > > > 14"
> >> > > > > > > (or
> >> > > > > > > > > > > > > whatever).
> >> > > > > > > > > > > > > > > > Maybe
> >> > > > > > > > > > > > > > > > > > there
> >> > > > > > > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > > > > > > > > specific
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > cases
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > where
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this is
> >> > hard?
> >> > > If
> >> > > > > we
> >> > > > > > > want
> >> > > > > > > > > to
> >> > > > > > > > > > > add
> >> > > > > > > > > > > > > > > > > server-side
> >> > > > > > > > > > > > > > > > > > > > error
> >> > > > > > > > > > > > > > > > > > > > > > > messages
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > really
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > do
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > need to
> >> do
> >> > > this
> >> > > > > in a
> >> > > > > > > > > > > consistent
> >> > > > > > > > > > > > > way
> >> > > > > > > > > > > > > > > > across
> >> > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > protocol.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> >> Updated
> >> > the
> >> > > > > KIP -
> >> > > > > > > > > please
> >> > > > > > > > > > > > check
> >> > > > > > > > > > > > > > > > > "Protocol
> >> > > > > > > > > > > > > > > > > > > > > Errors"
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> section. I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > added
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > comprehensive,
> >> > > > > > > > > fine-grained
> >> > > > > > > > > > > list
> >> > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > error
> >> > > > > > > > > > > > > > > > > > codes.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> Comments
> >> > from
> >> > > > > > > Guozhang:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 13.
> >> Describe
> >> > > > topic
> >> > > > > > > > > request:
> >> > > > > > > > > > it
> >> > > > > > > > > > > > > would
> >> > > > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > > great to
> >> > > > > > > > > > > > > > > > > > > > > go
> >> > > > > > > > > > > > > > > > > > > > > > > beyond
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > batching
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
> >> name
> >> > > regex
> >> > > > > for
> >> > > > > > > > this
> >> > > > > > > > > > > > request.
> >> > > > > > > > > > > > > > For
> >> > > > > > > > > > > > > > > > > > example,
> >> > > > > > > > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > > > > > > > very
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> common use
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > case
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> topic
> >> > > > command
> >> > > > > is
> >> > > > > > > to
> >> > > > > > > > > list
> >> > > > > > > > > > > all
> >> > > > > > > > > > > > > > > topics
> >> > > > > > > > > > > > > > > > > > whose
> >> > > > > > > > > > > > > > > > > > > > > config
> >> > > > > > > > > > > > > > > > > > > > > > > A's
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> value is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > B.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > With
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
> >> name
> >> > > regex
> >> > > > > > then
> >> > > > > > > we
> >> > > > > > > > > > have
> >> > > > > > > > > > > to
> >> > > > > > > > > > > > > > first
> >> > > > > > > > > > > > > > > > > > retrieve
> >> > > > > > > > > > > > > > > > > > > > > > > __all__
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> topics's
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> description
> >> > > info
> >> > > > > and
> >> > > > > > > > then
> >> > > > > > > > > > > filter
> >> > > > > > > > > > > > > at
> >> > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > client
> >> > > > > > > > > > > > > > > > > > > > > end,
> >> > > > > > > > > > > > > > > > > > > > > > > which
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> will
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > huge
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > burden
> on
> >> > ZK.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 14.
> >> Config
> >> > > K-Vs
> >> > > > in
> >> > > > > > > > create
> >> > > > > > > > > > > topic:
> >> > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > related
> >> > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> previous
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > point;
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > maybe
> we
> >> can
> >> > > add
> >> > > > > > > another
> >> > > > > > > > > > > > metadata
> >> > > > > > > > > > > > > > K-V
> >> > > > > > > > > > > > > > > or
> >> > > > > > > > > > > > > > > > > > just a
> >> > > > > > > > > > > > > > > > > > > > > > > metadata
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> string
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > along
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > side
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > with
> >> config
> >> > > K-V
> >> > > > in
> >> > > > > > > > create
> >> > > > > > > > > > > topic
> >> > > > > > > > > > > > > like
> >> > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > did
> >> > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > offset
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> commit
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > request.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > This
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > field
> >> can be
> >> > > > quite
> >> > > > > > > > useful
> >> > > > > > > > > in
> >> > > > > > > > > > > > > storing
> >> > > > > > > > > > > > > > > > > > information
> >> > > > > > > > > > > > > > > > > > > > > > like
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> "owner" of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > who
> issue
> >> > the
> >> > > > > create
> >> > > > > > > > > > command,
> >> > > > > > > > > > > > etc,
> >> > > > > > > > > > > > > > > which
> >> > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > quite
> >> > > > > > > > > > > > > > > > > > > > > > > > > important
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> for a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> multi-tenant
> >> > > > > > setting.
> >> > > > > > > > Then
> >> > > > > > > > > > in
> >> > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > describe
> >> > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > request
> >> > > > > > > > > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > also
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > batch
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > on
> regex
> >> of
> >> > > the
> >> > > > > > > metadata
> >> > > > > > > > > > > field.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: As
> >> > > discussed
> >> > > > it
> >> > > > > > is
> >> > > > > > > > very
> >> > > > > > > > > > > > > > interesting
> >> > > > > > > > > > > > > > > > but
> >> > > > > > > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> implemented
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > later
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > after
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we have
> >> some
> >> > > > basic
> >> > > > > > > > > > > functionality
> >> > > > > > > > > > > > > > > there.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 15.
> Today
> >> > all
> >> > > > the
> >> > > > > > > admin
> >> > > > > > > > > > > > operations
> >> > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > async in
> >> > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > sense
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > return
> >> once
> >> > it
> >> > > > is
> >> > > > > > > > written
> >> > > > > > > > > in
> >> > > > > > > > > > > ZK,
> >> > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > is why
> >> > > > > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > need
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> extra
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> verification
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> >> > > > > > > > > > > > > testUtil.waitForTopicCreated()
> >> > > > > > > > > > > > > > /
> >> > > > > > > > > > > > > > > > > verify
> >> > > > > > > > > > > > > > > > > > > > > > partition
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reassignment
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> request,
> >> > etc.
> >> > > > With
> >> > > > > > > admin
> >> > > > > > > > > > > > requests
> >> > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > could
> >> > > > > > > > > > > > > > > > > > add a
> >> > > > > > > > > > > > > > > > > > > > > > flag
> >> > > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> enable /
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > disable
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> synchronous
> >> > > > > > requests;
> >> > > > > > > > when
> >> > > > > > > > > > it
> >> > > > > > > > > > > is
> >> > > > > > > > > > > > > > > turned
> >> > > > > > > > > > > > > > > > > on,
> >> > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > response
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> will not
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > return
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > until
> the
> >> > > > request
> >> > > > > > has
> >> > > > > > > > been
> >> > > > > > > > > > > > > > completed.
> >> > > > > > > > > > > > > > > > And
> >> > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > async
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> requests we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > can
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > add a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > "token"
> >> > field
> >> > > in
> >> > > > > the
> >> > > > > > > > > > response,
> >> > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > then
> >> > > > > > > > > > > > > > > > > only
> >> > > > > > > > > > > > > > > > > > > > need
> >> > > > > > > > > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > > > > > > > > > > general
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > "admin
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> verification
> >> > > > > > request"
> >> > > > > > > > with
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > given
> >> > > > > > > > > > > > > > > > token
> >> > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > check
> >> > > > > > > > > > > > > > > > > > > > > > > if the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> async
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > request
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > has
> been
> >> > > > > completed.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I
> see
> >> > your
> >> > > > > point.
> >> > > > > > > My
> >> > > > > > > > > idea
> >> > > > > > > > > > > was
> >> > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > provide
> >> > > > > > > > > > > > > > > > > > > > > specific
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> Verify...Request
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > per
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > each
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long
> >> running
> >> > > > > > request,
> >> > > > > > > > > where
> >> > > > > > > > > > > > > needed.
> >> > > > > > > > > > > > > > We
> >> > > > > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > > > do it
> >> > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > way
> >> > > > > > > > > > > > > > > > > > > > > > > > > you
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > suggest.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > The
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > only
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > concern
> >> is
> >> > > that
> >> > > > > > > > > introducing
> >> > > > > > > > > > a
> >> > > > > > > > > > > > > token
> >> > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > again
> >> > > > > > > > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > > > > > > make
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> schema
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > We
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to do
> >> > similar
> >> > > > > thing
> >> > > > > > > > > > > introducing
> >> > > > > > > > > > > > > > single
> >> > > > > > > > > > > > > > > > > > > > > AdminRequest
> >> > > > > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > all
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > commands
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > but
> >> rejected
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> idea
> >> > > > because
> >> > > > > we
> >> > > > > > > > > wanted
> >> > > > > > > > > > to
> >> > > > > > > > > > > > > have
> >> > > > > > > > > > > > > > > > schema
> >> > > > > > > > > > > > > > > > > > > > > defined.
> >> > > > > > > > > > > > > > > > > > > > > > So
> >> > > > > > > > > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > more a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > choice
> >> > > between:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > a) have
> >> > fixed
> >> > > > > schema
> >> > > > > > > but
> >> > > > > > > > > > > > introduce
> >> > > > > > > > > > > > > > > each
> >> > > > > > > > > > > > > > > > > > time new
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> Verify...Request
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> long-running
> >> > > > > > requests
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > b) use
> >> one
> >> > > > request
> >> > > > > > for
> >> > > > > > > > > > > > > verification
> >> > > > > > > > > > > > > > > but
> >> > > > > > > > > > > > > > > > > > > > generalize
> >> > > > > > > > > > > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > > > > > > > > > with
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> token
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I'm
> fine
> >> > with
> >> > > > > > whatever
> >> > > > > > > > > > > decision
> >> > > > > > > > > > > > > > > > community
> >> > > > > > > > > > > > > > > > > > come
> >> > > > > > > > > > > > > > > > > > > > to.
> >> > > > > > > > > > > > > > > > > > > > > > > Just
> >> > > > > > > > > > > > > > > > > > > > > > > > > let
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> me
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > know
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > your
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> thoughts.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comment
> >> from
> >> > > > Gwen:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 16.
> >> > > Specifically
> >> > > > > for
> >> > > > > > > > > > > ownership,
> >> > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > think
> >> > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > plan
> >> > > > > > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > to add
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> ACL
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sounds
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> you
> >> are
> >> > > > > > > describing
> >> > > > > > > > > ACL)
> >> > > > > > > > > > > via
> >> > > > > > > > > > > > > an
> >> > > > > > > > > > > > > > > > > external
> >> > > > > > > > > > > > > > > > > > > > > system
> >> > > > > > > > > > > > > > > > > > > > > > > > > (Argus,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Sentry).
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I
> >> remember
> >> > > > KIP-11
> >> > > > > > > > > described
> >> > > > > > > > > > > > this,
> >> > > > > > > > > > > > > > but
> >> > > > > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > > > > can't
> >> > > > > > > > > > > > > > > > > > > > find
> >> > > > > > > > > > > > > > > > > > > > > > > the KIP
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> any
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > longer.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> Okay,
> >> no
> >> > > > > problem.
> >> > > > > > > Not
> >> > > > > > > > > > sure
> >> > > > > > > > > > > > > though
> >> > > > > > > > > > > > > > > how
> >> > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > > > > > going
> >> > > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> handle
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > it.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Wait
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > which
> KIP
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
> >> > > > committed
> >> > > > > > > first
> >> > > > > > > > > and
> >> > > > > > > > > > > > > include
> >> > > > > > > > > > > > > > > > > changes
> >> > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadata from
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > later
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> Anyway, I
> >> > > added
> >> > > > > this
> >> > > > > > > > note
> >> > > > > > > > > to
> >> > > > > > > > > > > > "Open
> >> > > > > > > > > > > > > > > > > > Questions"
> >> > > > > > > > > > > > > > > > > > > > > > section
> >> > > > > > > > > > > > > > > > > > > > > > > so
> >> > > > > > > > > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> don't
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > miss
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii
> >> > > Biletskyi
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri,
> >> Mar
> >> > > 13,
> >> > > > > 2015
> >> > > > > > > at
> >> > > > > > > > > > 12:34
> >> > > > > > > > > > > > AM,
> >> > > > > > > > > > > > > > > Andrii
> >> > > > > > > > > > > > > > > > > > > > > Biletskyi <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > andrii.biletskyi@stealth.ly
> >> > > > > > > > > > >
> >> > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi
> all,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> Today I
> >> > > > uploaded
> >> > > > > > the
> >> > > > > > > > > patch
> >> > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > covers
> >> > > > > > > > > > > > > > > > > > some of
> >> > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> discussed
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > agreed
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> items:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> >> removed
> >> > > > > MaybeOf
> >> > > > > > > > > optional
> >> > > > > > > > > > > > type
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> >> switched
> >> > > to
> >> > > > > java
> >> > > > > > > > > > protocol
> >> > > > > > > > > > > > > > > > definitions
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> >> > simplified
> >> > > > > > > messages
> >> > > > > > > > > > > > > (normalized
> >> > > > > > > > > > > > > > > > > configs,
> >> > > > > > > > > > > > > > > > > > > > > removed
> >> > > > > > > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> marked
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> >> deletion)
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I
> also
> >> > > updated
> >> > > > > the
> >> > > > > > > > KIP-4
> >> > > > > > > > > > > with
> >> > > > > > > > > > > > > > > > respective
> >> > > > > > > > > > > > > > > > > > > > changes
> >> > > > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> wrote down
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > my
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> >> proposal
> >> > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> pending
> >> > > items:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> Batch
> >> > > Admin
> >> > > > > > > > Operations
> >> > > > > > > > > > ->
> >> > > > > > > > > > > > > > updated
> >> > > > > > > > > > > > > > > > Wire
> >> > > > > > > > > > > > > > > > > > > > > Protocol
> >> > > > > > > > > > > > > > > > > > > > > > > schema
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > proposal
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> >> Remove
> >> > > > > > > > ClusterMetadata
> >> > > > > > > > > > ->
> >> > > > > > > > > > > > > > changed
> >> > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > extend
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> >> TopicMetadataRequest
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> Admin
> >> > > Client
> >> > > > > ->
> >> > > > > > > > > updated
> >> > > > > > > > > > my
> >> > > > > > > > > > > > > > initial
> >> > > > > > > > > > > > > > > > > > proposal
> >> > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > reflect
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > batching
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> Error
> >> > > codes
> >> > > > ->
> >> > > > > > > > > proposed
> >> > > > > > > > > > > > > > > fine-grained
> >> > > > > > > > > > > > > > > > > > error
> >> > > > > > > > > > > > > > > > > > > > > code
> >> > > > > > > > > > > > > > > > > > > > > > > > > instead
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> >> > > > > AdminRequestFailed
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I
> will
> >> > also
> >> > > > > send a
> >> > > > > > > > > > separate
> >> > > > > > > > > > > > > email
> >> > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > cover all
> >> > > > > > > > > > > > > > > > > > > > > > > comments
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> from
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> Thanks,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> Andrii
> >> > > > Biletskyi
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > On
> Thu,
> >> > Mar
> >> > > > 12,
> >> > > > > > 2015
> >> > > > > > > > at
> >> > > > > > > > > > 9:26
> >> > > > > > > > > > > > PM,
> >> > > > > > > > > > > > > > > Gwen
> >> > > > > > > > > > > > > > > > > > Shapira
> >> > > > > > > > > > > > > > > > > > > > <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > gshapira@cloudera.com
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> Found
> >> > > KIP-11
> >> > > > (
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> It
> >> > actually
> >> > > > > > > specifies
> >> > > > > > > > > > > changes
> >> > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > Metadata
> >> > > > > > > > > > > > > > > > > > > > > > > protocol,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> so
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > making
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sure
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> both
> >> KIPs
> >> > > are
> >> > > > > > > > > consistent
> >> > > > > > > > > > in
> >> > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > regard
> >> > > > > > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > > > > > > > good.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> On
> >> Thu,
> >> > Mar
> >> > > > 12,
> >> > > > > > > 2015
> >> > > > > > > > at
> >> > > > > > > > > > > 12:21
> >> > > > > > > > > > > > > PM,
> >> > > > > > > > > > > > > > > > Gwen
> >> > > > > > > > > > > > > > > > > > > > Shapira
> >> > > > > > > > > > > > > > > > > > > > > <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > gshapira@cloudera.com
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> >> > > > Specifically
> >> > > > > > for
> >> > > > > > > > > > > > ownership, I
> >> > > > > > > > > > > > > > > think
> >> > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > plan
> >> > > > > > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > add
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> ACL
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> like
> >> > you
> >> > > > are
> >> > > > > > > > > describing
> >> > > > > > > > > > > > ACL)
> >> > > > > > > > > > > > > > via
> >> > > > > > > > > > > > > > > an
> >> > > > > > > > > > > > > > > > > > > > external
> >> > > > > > > > > > > > > > > > > > > > > > > system
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> (Argus,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I
> >> > > remember
> >> > > > > > KIP-11
> >> > > > > > > > > > > described
> >> > > > > > > > > > > > > > this,
> >> > > > > > > > > > > > > > > > > but I
> >> > > > > > > > > > > > > > > > > > > > can't
> >> > > > > > > > > > > > > > > > > > > > > > > find
> >> > > > > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> KIP
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > any
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> >> > > > Regardless, I
> >> > > > > > > think
> >> > > > > > > > > > KIP-4
> >> > > > > > > > > > > > > > focuses
> >> > > > > > > > > > > > > > > > on
> >> > > > > > > > > > > > > > > > > > > > getting
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> information
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > already
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> >> exists
> >> > > from
> >> > > > > > Kafka
> >> > > > > > > > > > > brokers,
> >> > > > > > > > > > > > > not
> >> > > > > > > > > > > > > > on
> >> > > > > > > > > > > > > > > > > > adding
> >> > > > > > > > > > > > > > > > > > > > > > > information
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > perhaps
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> >> should
> >> > > > exist
> >> > > > > > but
> >> > > > > > > > > > doesn't
> >> > > > > > > > > > > > yet?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> Gwen
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On
> >> Thu,
> >> > > Mar
> >> > > > > 12,
> >> > > > > > > > 2015
> >> > > > > > > > > at
> >> > > > > > > > > > > > 6:37
> >> > > > > > > > > > > > > > AM,
> >> > > > > > > > > > > > > > > > > > Guozhang
> >> > > > > > > > > > > > > > > > > > > > > Wang
> >> > > > > > > > > > > > > > > > > > > > > > <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > wangguoz@gmail.com>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> Folks,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> Just
> >> > > want
> >> > > > to
> >> > > > > > > > > > elaborate a
> >> > > > > > > > > > > > bit
> >> > > > > > > > > > > > > > > more
> >> > > > > > > > > > > > > > > > on
> >> > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > create-topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > metadata
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> batching
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > describe-topic
> >> > > > > > > > based
> >> > > > > > > > > > on
> >> > > > > > > > > > > > > > config /
> >> > > > > > > > > > > > > > > > > > metadata
> >> > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > my
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> previous
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > email
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > as
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> on
> >> > > > > KAFKA-1694.
> >> > > > > > > The
> >> > > > > > > > > > main
> >> > > > > > > > > > > > > > > motivation
> >> > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > have
> >> > > > > > > > > > > > > > > > > > > > > > > some
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> sort of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > management
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > mechanisms,
> >> > > > > > > which
> >> > > > > > > > I
> >> > > > > > > > > > > think
> >> > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > quite
> >> > > > > > > > > > > > > > > > > > > > important
> >> > > > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > /
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > architecture:
> >> > > > > > > > today
> >> > > > > > > > > > > anyone
> >> > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > > create
> >> > > > > > > > > > > > > > > > > > > > topics
> >> > > > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> shared
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Kafka
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> cluster,
> >> > > but
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> there
> >> > is
> >> > > > no
> >> > > > > > > > concept
> >> > > > > > > > > or
> >> > > > > > > > > > > > > > > "ownership"
> >> > > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > > > topics
> >> > > > > > > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > created
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > by
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> different
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> users.
> >> > > For
> >> > > > > > > > example,
> >> > > > > > > > > at
> >> > > > > > > > > > > > > > LinkedIn
> >> > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > basically
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> distinguish
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > owners
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> some
> >> > > > casual
> >> > > > > > > topic
> >> > > > > > > > > name
> >> > > > > > > > > > > > > prefix,
> >> > > > > > > > > > > > > > > > which
> >> > > > > > > > > > > > > > > > > > is a
> >> > > > > > > > > > > > > > > > > > > > > bit
> >> > > > > > > > > > > > > > > > > > > > > > > > > awkward
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > does
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> we
> >> > scale
> >> > > > our
> >> > > > > > > > > > customers.
> >> > > > > > > > > > > It
> >> > > > > > > > > > > > > > would
> >> > > > > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > > great
> >> > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > use
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> describe-topics
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > such
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > Describe
> >> > > > all
> >> > > > > > > > topics
> >> > > > > > > > > > that
> >> > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > created
> >> > > > > > > > > > > > > > > > > > by me.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > Describe
> >> > > > all
> >> > > > > > > > topics
> >> > > > > > > > > > > whose
> >> > > > > > > > > > > > > > > > retention
> >> > > > > > > > > > > > > > > > > > time
> >> > > > > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > > > overriden
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> to X.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > Describe
> >> > > > all
> >> > > > > > > > topics
> >> > > > > > > > > > > whose
> >> > > > > > > > > > > > > > > writable
> >> > > > > > > > > > > > > > > > > > group
> >> > > > > > > > > > > > > > > > > > > > > > include
> >> > > > > > > > > > > > > > > > > > > > > > > > > user
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> Y
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> related
> >> > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > > authorization),
> >> > > > > > > > > etc..
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> One
> >> > > > possible
> >> > > > > > way
> >> > > > > > > > to
> >> > > > > > > > > > > > achieve
> >> > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > add a
> >> > > > > > > > > > > > > > > > > > > > > > > > > metadata
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> file
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > in
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > create-topic
> >> > > > > > > > > request,
> >> > > > > > > > > > > > whose
> >> > > > > > > > > > > > > > > value
> >> > > > > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > > > > also
> >> > > > > > > > > > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> written ZK
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > as
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > create
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> topic;
> >> > > > then
> >> > > > > > > > > > > > describe-topics
> >> > > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > > > choose to
> >> > > > > > > > > > > > > > > > > > > > > > batch
> >> > > > > > > > > > > > > > > > > > > > > > > > > topics
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > based
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > on
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 1)
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > name
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> regex,
> >> > > 2)
> >> > > > > > config
> >> > > > > > > > K-V
> >> > > > > > > > > > > > > matching,
> >> > > > > > > > > > > > > > > 3)
> >> > > > > > > > > > > > > > > > > > metadata
> >> > > > > > > > > > > > > > > > > > > > > > > regex,
> >> > > > > > > > > > > > > > > > > > > > > > > > > etc.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > Thoughts?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > Guozhang
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> On
> >> > Thu,
> >> > > > Mar
> >> > > > > 5,
> >> > > > > > > > 2015
> >> > > > > > > > > at
> >> > > > > > > > > > > > 4:37
> >> > > > > > > > > > > > > > PM,
> >> > > > > > > > > > > > > > > > > > Guozhang
> >> > > > > > > > > > > > > > > > > > > > > Wang
> >> > > > > > > > > > > > > > > > > > > > > > <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > wangguoz@gmail.com>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > Thanks
> >> > > > for
> >> > > > > > the
> >> > > > > > > > > > updated
> >> > > > > > > > > > > > > wiki.
> >> > > > > > > > > > > > > > A
> >> > > > > > > > > > > > > > > > few
> >> > > > > > > > > > > > > > > > > > > > comments
> >> > > > > > > > > > > > > > > > > > > > > > > below:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> 1.
> >> > > Error
> >> > > > > > > > > description
> >> > > > > > > > > > in
> >> > > > > > > > > > > > > > > > response: I
> >> > > > > > > > > > > > > > > > > > think
> >> > > > > > > > > > > > > > > > > > > > > if
> >> > > > > > > > > > > > > > > > > > > > > > > some
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > errorCode
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> indicate
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > several
> >> > > > > > > different
> >> > > > > > > > > > error
> >> > > > > > > > > > > > > cases
> >> > > > > > > > > > > > > > > > then
> >> > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > should
> >> > > > > > > > > > > > > > > > > > > > > > > really
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> change
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> multiple
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > codes.
> >> > > In
> >> > > > > > > general
> >> > > > > > > > > the
> >> > > > > > > > > > > > > > errorCode
> >> > > > > > > > > > > > > > > > > > itself
> >> > > > > > > > > > > > > > > > > > > > > would
> >> > > > > > > > > > > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> precise
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > sufficient
> >> > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > describing
> >> > > > > > the
> >> > > > > > > > > server
> >> > > > > > > > > > > > side
> >> > > > > > > > > > > > > > > > errors.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> 2.
> >> > > > Describe
> >> > > > > > > topic
> >> > > > > > > > > > > > request:
> >> > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > > would
> >> > > > > > > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > > > > > > great
> >> > > > > > > > > > > > > > > > > > > > > > > to go
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> beyond
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > just
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> batching
> >> > on
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> topic
> >> > > > name
> >> > > > > > > regex
> >> > > > > > > > > for
> >> > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > request.
> >> > > > > > > > > > > > > > > > > > For
> >> > > > > > > > > > > > > > > > > > > > > > > example, a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> very
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > common
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > use
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> case
> >> of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> the
> >> > > topic
> >> > > > > > > command
> >> > > > > > > > > is
> >> > > > > > > > > > to
> >> > > > > > > > > > > > > list
> >> > > > > > > > > > > > > > > all
> >> > > > > > > > > > > > > > > > > > topics
> >> > > > > > > > > > > > > > > > > > > > > whose
> >> > > > > > > > > > > > > > > > > > > > > > > > > config
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> A's
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > value
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > B.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> topic
> >> > > > name
> >> > > > > > > regex
> >> > > > > > > > > then
> >> > > > > > > > > > > we
> >> > > > > > > > > > > > > have
> >> > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > first
> >> > > > > > > > > > > > > > > > > > > > > > retrieve
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> __all__
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topics's
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > description
> >> > > > > > > info
> >> > > > > > > > > and
> >> > > > > > > > > > > then
> >> > > > > > > > > > > > > > > filter
> >> > > > > > > > > > > > > > > > at
> >> > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > client
> >> > > > > > > > > > > > > > > > > > > > > > > end,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> which
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > burden
> >> > > on
> >> > > > > ZK.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> 3.
> >> > > Config
> >> > > > > > K-Vs
> >> > > > > > > in
> >> > > > > > > > > > > create
> >> > > > > > > > > > > > > > topic:
> >> > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > related to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > previous
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point;
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> maybe
> >> > > we
> >> > > > > can
> >> > > > > > > add
> >> > > > > > > > > > > another
> >> > > > > > > > > > > > > > > metadata
> >> > > > > > > > > > > > > > > > > > K-V or
> >> > > > > > > > > > > > > > > > > > > > > > just a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> metadata
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > string
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > along
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> with
> >> > > > config
> >> > > > > > K-V
> >> > > > > > > > in
> >> > > > > > > > > > > create
> >> > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > like
> >> > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > did
> >> > > > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> offset
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > commit
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> request.
> >> > > This
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> field
> >> > > can
> >> > > > > be
> >> > > > > > > > quite
> >> > > > > > > > > > > useful
> >> > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > storing
> >> > > > > > > > > > > > > > > > > > > > > > > information
> >> > > > > > > > > > > > > > > > > > > > > > > > > like
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > "owner"
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> who
> >> > > issue
> >> > > > > the
> >> > > > > > > > > create
> >> > > > > > > > > > > > > command,
> >> > > > > > > > > > > > > > > > etc,
> >> > > > > > > > > > > > > > > > > > which
> >> > > > > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > quite
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > important
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > > multi-tenant
> >> > > > > > > > > setting.
> >> > > > > > > > > > > > Then
> >> > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > describe
> >> > > > > > > > > > > > > > > > > > > > > > > topic
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> request
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > can
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > also
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> batch
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> on
> >> > > regex
> >> > > > of
> >> > > > > > the
> >> > > > > > > > > > > metadata
> >> > > > > > > > > > > > > > field.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> 4.
> >> > > Today
> >> > > > > all
> >> > > > > > > the
> >> > > > > > > > > > admin
> >> > > > > > > > > > > > > > > operations
> >> > > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > > > async
> >> > > > > > > > > > > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> sense
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > return
> >> > > > once
> >> > > > > > it
> >> > > > > > > is
> >> > > > > > > > > > > written
> >> > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > ZK,
> >> > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > why we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> need
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > extra
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > > verification
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> like
> >> > > > > > > > > > > > > > > > > testUtil.waitForTopicCreated() /
> >> > > > > > > > > > > > > > > > > > > > > verify
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> partition
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> reassignment
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > request,
> >> > > > > etc.
> >> > > > > > > > With
> >> > > > > > > > > > > admin
> >> > > > > > > > > > > > > > > requests
> >> > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > could
> >> > > > > > > > > > > > > > > > > > > > > > add
> >> > > > > > > > > > > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> flag to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > enable
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > /
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> disable
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > synchronous
> >> > > > > > > > > requests;
> >> > > > > > > > > > > > when
> >> > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > turned
> >> > > > > > > > > > > > > > > > > > > > on,
> >> > > > > > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> response
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> return
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> until
> >> > > the
> >> > > > > > > request
> >> > > > > > > > > has
> >> > > > > > > > > > > > been
> >> > > > > > > > > > > > > > > > > > completed. And
> >> > > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > async
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > requests
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> add
> a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > "token"
> >> > > > > field
> >> > > > > > > in
> >> > > > > > > > > the
> >> > > > > > > > > > > > > > response,
> >> > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > then
> >> > > > > > > > > > > > > > > > > > > > > only
> >> > > > > > > > > > > > > > > > > > > > > > > need a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > general
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > "admin
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > > verification
> >> > > > > > > > > request"
> >> > > > > > > > > > > > with
> >> > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > given
> >> > > > > > > > > > > > > > > > > > > > token
> >> > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > check
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> if the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > async
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> request
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> has
> >> > > been
> >> > > > > > > > completed.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> 5. +1
> >> > > for
> >> > > > > > > > extending
> >> > > > > > > > > > > > > Metadata
> >> > > > > > > > > > > > > > > > > request
> >> > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > include
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > controller /
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > coordinator
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > > information,
> >> > > > > > > and
> >> > > > > > > > > then
> >> > > > > > > > > > > we
> >> > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > remove
> >> > > > > > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > > > ClusterMetadata
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > requests.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > Guozhang
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> On
> >> > Tue,
> >> > > > Mar
> >> > > > > > 3,
> >> > > > > > > > 2015
> >> > > > > > > > > > at
> >> > > > > > > > > > > > > 10:23
> >> > > > > > > > > > > > > > > AM,
> >> > > > > > > > > > > > > > > > > Joel
> >> > > > > > > > > > > > > > > > > > > > > Koshy <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > jjkoshy.w@gmail.com
> >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > Thanks
> >> > > > for
> >> > > > > > > > sending
> >> > > > > > > > > > > that
> >> > > > > > > > > > > > > out
> >> > > > > > > > > > > > > > > Joe
> >> > > > > > > > > > > > > > > > -
> >> > > > > > > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > > > > > > don't
> >> > > > > > > > > > > > > > > > > > > > > > > think I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> will be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > able
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> it
> >> > > > today,
> >> > > > > so
> >> > > > > > > if
> >> > > > > > > > > > notes
> >> > > > > > > > > > > > can
> >> > > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > > sent
> >> > > > > > > > > > > > > > > > > > out
> >> > > > > > > > > > > > > > > > > > > > > > > afterward
> >> > > > > > > > > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > would
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > great.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> On
> >> > > Mon,
> >> > > > > Mar
> >> > > > > > > 02,
> >> > > > > > > > > 2015
> >> > > > > > > > > > > at
> >> > > > > > > > > > > > > > > > 09:16:13AM
> >> > > > > > > > > > > > > > > > > > > > -0800,
> >> > > > > > > > > > > > > > > > > > > > > > Gwen
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> Shapira
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > > Thanks
> >> > > > > for
> >> > > > > > > > > sending
> >> > > > > > > > > > > > this
> >> > > > > > > > > > > > > > out
> >> > > > > > > > > > > > > > > > Joe.
> >> > > > > > > > > > > > > > > > > > > > Looking
> >> > > > > > > > > > > > > > > > > > > > > > > forward
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > chatting
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > with
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > > everyone
> >> > > > > :)
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > On
> >> > > > Mon,
> >> > > > > > Mar
> >> > > > > > > 2,
> >> > > > > > > > > > 2015
> >> > > > > > > > > > > at
> >> > > > > > > > > > > > > > 6:46
> >> > > > > > > > > > > > > > > > AM,
> >> > > > > > > > > > > > > > > > > > Joe
> >> > > > > > > > > > > > > > > > > > > > > Stein
> >> > > > > > > > > > > > > > > > > > > > > > <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > joe.stein@stealth.ly>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > > > Hey, I
> >> > > > > > > just
> >> > > > > > > > > sent
> >> > > > > > > > > > > > out a
> >> > > > > > > > > > > > > > > > google
> >> > > > > > > > > > > > > > > > > > > > hangout
> >> > > > > > > > > > > > > > > > > > > > > > > invite
> >> > > > > > > > > > > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> all
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > pmc,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > committers
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > > > > > everyone I
> >> > > > > > > > > found
> >> > > > > > > > > > > > > working
> >> > > > > > > > > > > > > > > on
> >> > > > > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > > > KIP.
> >> > > > > > > > > > > > > > > > > > > > If
> >> > > > > > > > > > > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > > > > > > > > > missed
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> anyone
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> invite
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > please
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > > let
> >> > > > me
> >> > > > > > > know
> >> > > > > > > > > and
> >> > > > > > > > > > > can
> >> > > > > > > > > > > > > > update
> >> > > > > > > > > > > > > > > > it,
> >> > > > > > > > > > > > > > > > > > np.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > We
> >> > > > > > should
> >> > > > > > > do
> >> > > > > > > > > > this
> >> > > > > > > > > > > > > every
> >> > > > > > > > > > > > > > > > > Tuesday
> >> > > > > > > > > > > > > > > > > > @
> >> > > > > > > > > > > > > > > > > > > > 2pm
> >> > > > > > > > > > > > > > > > > > > > > > > Eastern
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> Time.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Maybe
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > INFRA
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > > help
> >> > > > > to
> >> > > > > > > > make a
> >> > > > > > > > > > > > google
> >> > > > > > > > > > > > > > > > account
> >> > > > > > > > > > > > > > > > > > so we
> >> > > > > > > > > > > > > > > > > > > > > can
> >> > > > > > > > > > > > > > > > > > > > > > > manage
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > better?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > To
> >> > > > > > discuss
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >>
> >> > > > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > in
> >> > > > > > > progress
> >> > > > > > > > > and
> >> > > > > > > > > > > > > related
> >> > > > > > > > > > > > > > > JIRA
> >> > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > are
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> interdependent
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > common
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> work.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > ~
> >> > > > Joe
> >> > > > > > > Stein
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > On
> >> > > > > Tue,
> >> > > > > > > Feb
> >> > > > > > > > > 24,
> >> > > > > > > > > > > 2015
> >> > > > > > > > > > > > > at
> >> > > > > > > > > > > > > > > 2:59
> >> > > > > > > > > > > > > > > > > > PM, Jay
> >> > > > > > > > > > > > > > > > > > > > > > > Kreps <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> >> > > > > jay.kreps@gmail.com
> >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > Let's
> >> > > > > > > stay
> >> > > > > > > > on
> >> > > > > > > > > > > > Google
> >> > > > > > > > > > > > > > > > hangouts
> >> > > > > > > > > > > > > > > > > > that
> >> > > > > > > > > > > > > > > > > > > > > will
> >> > > > > > > > > > > > > > > > > > > > > > > also
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> record
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > make
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > > sessions
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > > > available
> >> > > > > > > > on
> >> > > > > > > > > > > > youtube.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > -Jay
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > On
> >> > > > > Tue,
> >> > > > > > > Feb
> >> > > > > > > > > 24,
> >> > > > > > > > > > > > 2015
> >> > > > > > > > > > > > > at
> >> > > > > > > > > > > > > > > > 11:49
> >> > > > > > > > > > > > > > > > > > AM,
> >> > > > > > > > > > > > > > > > > > > > > Jeff
> >> > > > > > > > > > > > > > > > > > > > > > > > > Holoman
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > > > > > > > > > jholoman@cloudera.com
> >> > > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > Jay /
> >> > > > > > > Joe
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > We're
> >> > > > > > > > happy
> >> > > > > > > > > > to
> >> > > > > > > > > > > > send
> >> > > > > > > > > > > > > > > out a
> >> > > > > > > > > > > > > > > > > > Webex
> >> > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > this
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> purpose.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > We
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> record
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> the
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > > > sessions
> >> > > > > > > > if
> >> > > > > > > > > > > there
> >> > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > interest and
> >> > > > > > > > > > > > > > > > > > > > > > > publish
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> them
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > out.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > > Thanks
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > Jeff
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > On
> >> > > > > > Tue,
> >> > > > > > > > Feb
> >> > > > > > > > > > 24,
> >> > > > > > > > > > > > > 2015
> >> > > > > > > > > > > > > > at
> >> > > > > > > > > > > > > > > > > > 11:28 AM,
> >> > > > > > > > > > > > > > > > > > > > > Jay
> >> > > > > > > > > > > > > > > > > > > > > > > > > Kreps <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > > > > > > jay.kreps@gmail.com>
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > Let's
> >> > > > > > > > try
> >> > > > > > > > > > to
> >> > > > > > > > > > > > get
> >> > > > > > > > > > > > > > the
> >> > > > > > > > > > > > > > > > > > technical
> >> > > > > > > > > > > > > > > > > > > > > > > hang-ups
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> sorted
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > out,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > though.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > really
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > think
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > there
> >> > > > > > > > is
> >> > > > > > > > > > some
> >> > > > > > > > > > > > > > benefit
> >> > > > > > > > > > > > > > > > to
> >> > > > > > > > > > > > > > > > > > live
> >> > > > > > > > > > > > > > > > > > > > > > > discussion
> >> > > > > > > > > > > > > > > > > > > > > > > > > vs
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > writing. I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > am
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > > hopeful
> >> > > > > that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > if
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > we
> >> > > > > > > post
> >> > > > > > > > > > > > > > instructions
> >> > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > give
> >> > > > > > > > > > > > > > > > > > > > > > > ourselves a
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> few
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > attempts
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> get
> >> > it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > > > working.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > > Tuesday
> >> > > > > > > > > at
> >> > > > > > > > > > > that
> >> > > > > > > > > > > > > > time
> >> > > > > > > > > > > > > > > > > would
> >> > > > > > > > > > > > > > > > > > work
> >> > > > > > > > > > > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> me...any
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > objections?
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > -Jay
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > On
> >> > > > > > > Tue,
> >> > > > > > > > > Feb
> >> > > > > > > > > > > 24,
> >> > > > > > > > > > > > > > 2015
> >> > > > > > > > > > > > > > > at
> >> > > > > > > > > > > > > > > > > > 8:18
> >> > > > > > > > > > > > > > > > > > > > AM,
> >> > > > > > > > > > > > > > > > > > > > > > Joe
> >> > > > > > > > > > > > > > > > > > > > > > > > > Stein
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > > > > > > joe.stein@stealth.ly
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > > Weekly
> >> > > > > > > > > > > would
> >> > > > > > > > > > > > be
> >> > > > > > > > > > > > > > > great
> >> > > > > > > > > > > > > > > > > > maybe
> >> > > > > > > > > > > > > > > > > > > > > like
> >> > > > > > > > > > > > > > > > > > > > > > > every
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 1pm
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > ET
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > /
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> PT
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > ????
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > I
> >> > > > > > > > don't
> >> > > > > > > > > > > mind
> >> > > > > > > > > > > > > > google
> >> > > > > > > > > > > > > > > > > > hangout
> >> > > > > > > > > > > > > > > > > > > > but
> >> > > > > > > > > > > > > > > > > > > > > > > there
> >> > > > > > > > > > > > > > > > > > > > > > > > > is
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > always
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > some
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> issue
> >> or
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > > > whatever
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > so
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > we
> >> > > > > > > > know
> >> > > > > > > > > > the
> >> > > > > > > > > > > > > > apache
> >> > > > > > > > > > > > > > > > irc
> >> > > > > > > > > > > > > > > > > > > > channel
> >> > > > > > > > > > > > > > > > > > > > > > > works.
> >> > > > > > > > > > > > > > > > > > > > > > > > > We
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > start
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > there
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> see
> >> > > how
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > goes?
> >> > > > > > > > > We
> >> > > > > > > > > > > can
> >> > > > > > > > > > > > > pull
> >> > > > > > > > > > > > > > > > > > transcripts
> >> > > > > > > > > > > > > > > > > > > > > too
> >> > > > > > > > > > > > > > > > > > > > > > > and
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > associate
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> tickets
> >> > if
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> need
> >> > > be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > makes
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > it
> >> > > > > > > > > > helpful
> >> > > > > > > > > > > > for
> >> > > > > > > > > > > > > > > > things.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > ~
> >> > > > > > > > > > Joestein
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > On
> >> > > > > > > > Tue,
> >> > > > > > > > > > Feb
> >> > > > > > > > > > > > 24,
> >> > > > > > > > > > > > > > > 2015
> >> > > > > > > > > > > > > > > > at
> >> > > > > > > > > > > > > > > > > > 11:10
> >> > > > > > > > > > > > > > > > > > > > > AM,
> >> > > > > > > > > > > > > > > > > > > > > > > Jay
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> Kreps <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > > > > > > > > jay.kreps@gmail.com
> >> > > > > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > We'd
> >> > > > > > > > > > > talked
> >> > > > > > > > > > > > > > about
> >> > > > > > > > > > > > > > > > > > doing a
> >> > > > > > > > > > > > > > > > > > > > > > Google
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> Hangout to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > chat
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > about
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> this.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> What
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > about
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > > > > > generalizing
> >> > > > > > > > > > > > > > > that a
> >> > > > > > > > > > > > > > > > > > little
> >> > > > > > > > > > > > > > > > > > > > > > > > > further...I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > actually
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > think
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > would
> >> > > be
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > good
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > for
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > > > everyone
> >> > > > > > > > > > > > > > > spending a
> >> > > > > > > > > > > > > > > > > > > > > reasonable
> >> > > > > > > > > > > > > > > > > > > > > > > chunk
> >> > > > > > > > > > > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > their
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > week
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> Kafka
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > stuff
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > maybe
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > sync
> >> > > > > > > > > up
> >> > > > > > > > > > > > once
> >> > > > > > > > > > > > > a
> >> > > > > > > > > > > > > > > > week.
> >> > > > > > > > > > > > > > > > > I
> >> > > > > > > > > > > > > > > > > > > > think
> >> > > > > > > > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > > > > > > could
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> use
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > time
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > talk
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > > through
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > > design
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > > stuff,
> >> > > > > > > > > > > make
> >> > > > > > > > > > > > > > sure
> >> > > > > > > > > > > > > > > we
> >> > > > > > > > > > > > > > > > > > are on
> >> > > > > > > > > > > > > > > > > > > > > top
> >> > > > > > > > > > > > > > > > > > > > > > of
> >> > > > > > > > > > > > > > > > > > > > > > > > > code
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > reviews,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > talk
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> through
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> any
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > > tricky
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > > issues,
> >> > > > > > > > > > > > etc.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > We
> >> > > > > > > > > can
> >> > > > > > > > > > > make
> >> > > > > > > > > > > > > it
> >> > > > > > > > > > > > > > > > > publicly
> >> > > > > > > > > > > > > > > > > > > > > > > available so
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > any
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > one
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> follow
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > along
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > who
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > > likes.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > Any
> >> > > > > > > > > > > > interest
> >> > > > > > > > > > > > > in
> >> > > > > > > > > > > > > > > > doing
> >> > > > > > > > > > > > > > > > > > this?
> >> > > > > > > > > > > > > > > > > > > > > If
> >> > > > > > > > > > > > > > > > > > > > > > so
> >> > > > > > > > > > > > > > > > > > > > > > > > > I'll
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> try
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > set
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > it
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > up
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > > starting
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > next
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > week.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > -Jay
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > On
> >> > > > > > > > > Tue,
> >> > > > > > > > > > > Feb
> >> > > > > > > > > > > > > 24,
> >> > > > > > > > > > > > > > > > 2015
> >> > > > > > > > > > > > > > > > > at
> >> > > > > > > > > > > > > > > > > > > > 3:57
> >> > > > > > > > > > > > > > > > > > > > > > AM,
> >> > > > > > > > > > > > > > > > > > > > > > > > > Andrii
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > <
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> >> > > > > > > > > > > > > > > > > > > > > > > wrote:
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > >
> >> > > > > > > > Hi
> >> > > > > > > > > > all,
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > >
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > >
> >> > > > > > > > > I've
> >> > > > > > > > > > > > > updated
> >> > > > > > > > > > > > > > > KIP
> >> > > > > > > > > > > > > > > > > > page,
> >> > > > > > > > > > > > > > > > > > > > > fixed
> >> > > > > > > > > > > > > > > > > > > > > > /
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> aligned
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > document
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >> > structure.
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > Also I
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> >> > >>
> >> > > > >
> >> > > > > > added
> >> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> >>>> >
> >> > >> >
> >> > > > > >
> >> > > > > > > >
> >> > > > > > > > > some
> >> > > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > ...
> >> > > > > > > > > > > > > > >
> >> > > > > > > > > > > > > > > [Message clipped]
> >> > > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > > > --
> >> > > > > > > > > > > > > -- Guozhang
> >> > > > > > > > > > > > >
> >> > > > > > > > > > > >
> >> > > > > > > > > > >
> >> > > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > --
> >> > > > > > > -- Guozhang
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >> >
> >> >
> >> > --
> >> > Thanks,
> >> > Neha
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
There is one more item that we need to discuss.

210. Do we need a separate ListTopic request that just returns a list of
all topics. Currently, we piggyback this functionality in TMR. If the topic
list is empty in TMR, we return all topics and their metadata. The tradeoff
is the following. If we piggyback on TMR, there is one fewer request that
we have to deal with. On the other hand, it does mean that the response
size can be large (e.g., if you have 10K topics and a 1KB metadata per
topic, the response can be 10MB). There are probably cases (e.g., in
supporting wildcard topics in the new java consumer) that a client just
needs to see the full topic list, but not the metadata for every topic.

Thanks,

Jun

On Tue, Mar 24, 2015 at 2:01 PM, Jun Rao <ju...@confluent.io> wrote:

> 201. I think it's probably easier to consolidate on DTR. If we evolve TMR,
> the producer will be using the latest version of TMR. However, the latest
> version of TMR won't trigger auto topic creation. This means that we need
> to change the producer to use CreateTopicRequest. If we consolidate on DTR.
> We can fix the producer logic later.
>
> 205. Also, there was a remaining item on how to get the controller info.
> We can either add the controller id to the response of DTR or create a new
> cluster metadata request/response. The cluster meta response will then
> include the detailed information about every broker and the controller. I
> think the latter is probably better. For example, when we add the
> multi-port support (for security), the cluster meta response can include
> all ports for a broker. In DTR, we only need to return one broker port to
> the client.
>
> Thanks,
>
> Jun
>
> On Tue, Mar 24, 2015 at 12:26 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
>> Hi all,
>>
>> A short summary of our discussion:
>>
>> 201. DescribeTopicRequest(DTR) vs TopicMetadataRequest(TMR)
>>
>> *Issue*: Topic description should include topic-level configuration,
>> other information is available in TMR - leader, ISR, AR.
>> So now there are two options - either add full-fledged new DTR
>> that will include TMR + all needed information (which will be used then
>> instead TMR) or extend TMR to include topic-level configuration.
>>
>> *Resolution*: it was decided to vote on it: either a) *extend TMR* or b)
>> *add
>> DTR*.
>> I'd go with a) - we will have to roll out TMR_V1 but it's better than
>> having
>> "dead" requests if we add DTR and leave TMR unused in future versions
>>
>> 202. CreateTopicRequest sync/async semantics
>>
>> *Issue*: What should be a post-condition of Create/Alter/Delete
>> requests - the command has been completed or command has been
>> initiated?
>> Even though it looks reasonable that users would typically expect to
>> have topic created upon CreateTopicRequest the corner case is that
>> topic can be deleted right after creation, so received CreateTopicResponse
>> cannot be a 100% guarantee topic is created and available, even if
>> CreateTopicRequest is synchronous. Also, making request blocking on
>> server may be a hard thing to do right now (check Jun's email from Mar
>> 21).
>>
>> *Resolution*: for now it makes sense to start with async semantics for
>> Create/Alter/Delete.
>>
>> 203. VerifyReassignPartition vs leverage DescribeTopicRequest
>>
>> *Issue*: DescribeTopicRequest (or TopicMetadaRequest) should be enough,
>> since controller doesn't remove partitions from admin/reassign_partitions
>> until
>> the assignment actually matches target assignment so partition
>> reassignment
>> cannot fail for specific partition in the current implementation.
>> Everything else
>> ("Completed" and "In Progress" state) can be checked with assigned
>> partitions
>> field from DescribeTopicRequest.
>>
>> *Resolution*: remove VerifyReassignPartitionsRequest
>>
>> 204. Also it was discussed that after we finalize question 201 and port
>> java protocol definition (KAFKA-1927) I will split patch to separate
>> pieces,
>> first will cover Wire Protocol changes, since we have a common vision on
>> it.
>>
>> Please let me know your thoughts on q. 201!
>>
>> Thanks,
>> Andrii Biletskyi
>>
>>
>>
>>
>> On Tue, Mar 24, 2015 at 7:57 PM, Neha Narkhede <ne...@confluent.io> wrote:
>>
>> > I'd like to resurface the discussion of sync vs async topic creation.
>> > Possibly, we can discuss that in today's KIP meeting. I am worried about
>> > pushing the burden of handling async topic creation on the clients.
>> This is
>> > one of the most frequently asked questions on the mailing list and I
>> hope
>> > we find a way to address it as we separate the metadata request and
>> topic
>> > creation. Since topic creation doesn't take that long, there should be a
>> > really easy way for clients to express their preference for a sync topic
>> > creation request where the receipt of a response suggests completion of
>> > topic creation.
>> >
>> > The behavior that I'm expecting is that if I'm a client and I choose to
>> > create a topic synchronously, after I receive a response, my first send
>> > request for that topic should *not* fail with some error.
>> >
>> > On Tue, Mar 24, 2015 at 8:02 AM, Jun Rao <ju...@confluent.io> wrote:
>> >
>> > > Andrii,
>> > >
>> > >
>> > > 111.4 Yes, we can discuss this in the meeting today.
>> > >
>> > > 112. The question is when will a PartitionReassignment fail.
>> Currently,
>> > it
>> > > can fail if the input is incorrect (e.g, invalid partition/replica).
>> > > However, this can be detected when the ReassignPartitionRequest is
>> > issued.
>> > > Once the process of partition reassignment is started, it will not
>> fail.
>> > > The controller just waits until the process completes. We can improve
>> the
>> > > status reporting of partition reassignment. However, I am not sure if
>> > that
>> > > needs to be tied to this KIP.
>> > >
>> > > 115.1 Yes, that makes sense. I misunderstood it. I thought
>> > > PreferredReplicaLeaderElectionInProgress and
>> ReassignPartitionsInProgress
>> > > are used in the status check.
>> > >
>> > > Thanks,
>> > >
>> > > Jun
>> > >
>> > >
>> > > On Tue, Mar 24, 2015 at 3:46 AM, Andrii Biletskyi <
>> > > andrii.biletskyi@stealth.ly> wrote:
>> > >
>> > > > Jun,
>> > > >
>> > > > Thanks for such an accurate review!
>> > > >
>> > > > Most of your remarks I didn't fix in the previous change because
>> > > > I thought we'd finalize them today, during the call. Anyway, I've
>> > > > fixed them now.
>> > > > My comments to ensure I covered everything:
>> > > >
>> > > > 110. Fixed. Removed global error code.
>> > > >
>> > > > 111.1-3. Removed redundant TopicName, Partitions and
>> ReplicationFactor
>> > > > 111.4 Yes, now we are in the situation when TopicMetadataResponse
>> > > > contains everything we need except topic level configs. I'm
>> > > > okay with both solutions, probably makes sense to discuss and pick
>> > > > the preferable one.
>> > > >
>> > > > 112. Need to think about it a bit more. Currently we may distinguish
>> > > > 3 states of specific partition being reassigned:
>> > > > a) Completed - absent in /reassign_partitions zk path and
>> > DescribeTopic/
>> > > > TopicMetadata shows expected assignment
>> > > > 2) In progress - present in /reassign_partitions
>> > > > 3) Failed - absent in /reassign_partitions but
>> > > DescribeTopic/TopicMetadata
>> > > > replicas field doesn't correspond expected
>> > > > Now, if use only DescribeTopic to check reassignment status - how
>> can
>> > we
>> > > > distinguish states "In Progress" and "Failed"?
>> > > > Also, there are some concerns that current reassignment status
>> provided
>> > > > by ReaasignPartitionsCommand (and the same was intended in KIP-4) is
>> > > > very spurious, non-informative. Probably it makes sense to add
>> > additional
>> > > > data there (thus separate request may be useful), I'm considering
>> > > different
>> > > > options now - btw, comments here are highly appreciated!
>> > > >
>> > > > 113-114. Okay, made it compliant with topic commands. About "The
>> error
>> > > code
>> > > > will
>> > > > then be per topic" - just a note: currently
>> (ReassignPartitionsCommand)
>> > > > validation
>> > > > is done for the whole reassignment string, so in case of errors for
>> > > > specific
>> > > > partitions the reassignment is NOT started at all. Both options look
>> > fine
>> > > > for me,
>> > > > it's just the logic will be slightly changed.
>> > > >
>> > > > 115.1 Removed NotControllerReceivedAdminCommand. But why should we
>> > > > remove PreferredReplicaLeaderElectionInProgress and
>> > > > ReassignPartitionsInProgress?
>> > > > On Reassign/Preferred..Request we create admin zk path, we can of
>> > course
>> > > > update the path, but wouldn't it be safer if we simply refuse to
>> start
>> > > new
>> > > > reassignment in case one is in progress?
>> > > >
>> > > > 115.2 Yes, paragraph Protocol Errors covers only errors that are
>> > proposed
>> > > > to be added. I didn't want to list all errors that are to be added
>> > since
>> > > > anyway
>> > > > the final version will be clear after the development is completed.
>> > > >
>> > > > 100. Fixed, normalized json string in Create/Alter.
>> > > >
>> > > > Thanks,
>> > > > Andrii Biletskyi
>> > > >
>> > > > On Tue, Mar 24, 2015 at 12:58 AM, Jun Rao <ju...@confluent.io> wrote:
>> > > >
>> > > > > Andrii,
>> > > > >
>> > > > > I looked at the latest wiki for this KIP. I have a few more
>> comments.
>> > > > >
>> > > > > 110. In CreateTopicResponse, AlterTopicResponse,
>> DeleteTopicResponse
>> > > and
>> > > > > DescribeTopicResponse, we probably don't need the global error
>> code.
>> > > Our
>> > > > > current convention is to just use the same global error code in
>> each
>> > > > topic.
>> > > > > This simplifies the error checking on the client side.
>> > > > >
>> > > > > 111. DescribeTopicResponse:
>> > > > > 111.1 Our protocol definition doesn't support optional fields.
>> So, we
>> > > > can't
>> > > > > make Leader an optional field.
>> > > > > 111.2 Do we really need the field Partitions and
>> ReplicationFactor in
>> > > > > TopicConfigDetails?
>> > > > > The former is basically the size of the TopicPartitionDetails
>> array
>> > and
>> > > > the
>> > > > > latter can be found from the size of the Replica array.
>> > > > > 111.3 Do we need TopicName in TopicDescription since it's already
>> in
>> > > > > DescribeTopicResponse?
>> > > > > 111.4 Finally, DescribeTopicResponse is very similar to
>> > > > > TopicMetadataResponse. So, it probably makes sense just to keep
>> one
>> > of
>> > > > them
>> > > > > in the future. Should we just use DescribeTopicRequest/Response to
>> > > > replace
>> > > > > TopicMetadataRequest/Response in the producer/consumer client in
>> the
>> > > > future
>> > > > > (since DescribeTopicRequest doesn't trigger auto topic creation)?
>> If
>> > > so,
>> > > > we
>> > > > > will need to add the broker list in DescribeTopicResponse.
>> > > > >
>> > > > > 112. Thinking about this a bit more, I don't think we need a
>> separate
>> > > > > VerifyReassignPartitionRequest/Response.
>> > > > > We can just use DescribeTopic to get the assigned replicas and
>> check
>> > if
>> > > > > they match the target replica assignment in the client. The
>> > controller
>> > > > > propagates the metadata change after the reassignment completes
>> for
>> > > each
>> > > > > partition.
>> > > > >
>> > > > > 113. ReassignPartitionRequest: For consistency, we probably want
>> to
>> > > nest
>> > > > > the partition data under topic. So instead of
>> > > > >   ReassignPartitionRequest => [Topic PartitionId [ReplicaId]]
>> > > > > we can have sth like
>> > > > >   ReassignPartitionRequest => [Topic [PartitionId [ReplicaId]]]
>> > > > > The error code will then be per topic.
>> > > > >
>> > > > > 114. PreferredReplicaLeaderElectionRequest: Same as the above.
>> > Instead
>> > > of
>> > > > >   PreferredReplicaLeaderElectionRequest => [Topic PartitionId]
>> > > > > we can have
>> > > > >   PreferredReplicaLeaderElectionRequest => [Topic [PartitionId]]
>> > > > > Again, the error code will be per topic.
>> > > > >
>> > > > > 115. ErrorCode:
>> > > > > 115.1 NotControllerReceivedAdminRequest,
>> > > > > PreferredReplicaLeaderElectionInProgress
>> > > > > and ReassignPartitionsInProgress are likely not needed any more.
>> > > > > 115.2 We probably need to add UnknownTopicOrPartitionCode since
>> it's
>> > a
>> > > > > valid error code for some of the requests (e.g. describe, alter,
>> etc)
>> > > > when
>> > > > > a non-existing topic is specified.
>> > > > >
>> > > > >
>> > > > > Previous comments.
>> > > > > 100. ReplicaAssignment in CreateTopicRequest and AlterTopicRequest
>> > are
>> > > > > still a json string. We need to flatten them.
>> > > > >
>> > > > >
>> > > > > Thanks,
>> > > > >
>> > > > > Jun
>> > > > >
>> > > > >
>> > > > > On Sat, Mar 21, 2015 at 2:03 AM, Andrii Biletskyi <
>> > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > >
>> > > > > > Guozhang,
>> > > > > >
>> > > > > > I'm not sure I understand how can we use those tools in CLI.
>> > > > > > First of all, those are *Test*Utils, e.g.
>> > > waitUntilMetadataIsPropagated
>> > > > > > (which might be very useful for us) requires
>> > servers:Seq[KafkaServer]
>> > > > > > as an argument:
>> > > > > >
>> > > > > > def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer],
>> topic:
>> > > > > String,
>> > > > > > partition: Int, timeout: Long = 5000L): Int
>> > > > > >
>> > > > > > how can we have access it at runtime in one of the brokers?
>> > > > > >
>> > > > > > Secondly, how can user directly call these tools if zookeeper
>> might
>> > > be
>> > > > > > not accessible (which is used in the tools, right?) at all -
>> e.g.
>> > > > behind
>> > > > > > the VPC
>> > > > > > in AWS. Furthermore, I think with KIP-4 we are trying to create
>> an
>> > > > > > abstraction and
>> > > > > > a single point of zookeeper interactions, to eliminate direct
>> calls
>> > > to
>> > > > ZK
>> > > > > > either to get
>> > > > > > some cluster information or to change something.
>> > > > > >
>> > > > > > Thanks,
>> > > > > > Andrii Biletskyi
>> > > > > >
>> > > > > > On Sat, Mar 21, 2015 at 3:37 AM, Guozhang Wang <
>> wangguoz@gmail.com
>> > >
>> > > > > wrote:
>> > > > > >
>> > > > > > > Andrii,
>> > > > > > >
>> > > > > > > Actually the checking logic Jun mentioned is already
>> implemented
>> > as
>> > > > > > > TestUtils.waitUntilXXX (LeaderIsElected, MetadataIsPropagated,
>> > > > etc...)
>> > > > > I
>> > > > > > > think we can extend these functions as CLI tools like
>> > TopicCommand
>> > > so
>> > > > > > that
>> > > > > > > users re-implementing such endpoint can directly call
>> something
>> > > like
>> > > > > > > java.tools.WaitUntilXXX (of course this requires them to have
>> > javac
>> > > > > > > installed, which should be a reasonable requirement?)
>> > > > > > >
>> > > > > > > Guozhang
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
>> > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > >
>> > > > > > > > Jun,
>> > > > > > > >
>> > > > > > > > Not that I was saying we need to make requests blocking on
>> > > server,
>> > > > > > > > it was just to emphasize that with async requests a client
>> > > > > > > implementations
>> > > > > > > > may be a little bit more than just issue request - get the
>> > > > response.
>> > > > > > > > Thanks for the explanation, I understand now that we can go
>> > with
>> > > > > agreed
>> > > > > > > > solution though it may not be perfect.
>> > > > > > > > I believe this was one of the last controversial questions
>> from
>> > > the
>> > > > > > list.
>> > > > > > > >
>> > > > > > > > Thanks,
>> > > > > > > > Andrii Biletskyi
>> > > > > > > >
>> > > > > > > > On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <jun@confluent.io
>> >
>> > > > wrote:
>> > > > > > > >
>> > > > > > > > > Andrii,
>> > > > > > > > >
>> > > > > > > > > A few points.
>> > > > > > > > >
>> > > > > > > > > 1. Create/Alter can typically complete quickly. So, it's
>> > > possible
>> > > > > to
>> > > > > > > make
>> > > > > > > > > the request block until it's completed. However,
>> currently,
>> > > doing
>> > > > > > this
>> > > > > > > at
>> > > > > > > > > the broker is a bit involved. To make Create block, we
>> will
>> > > need
>> > > > to
>> > > > > > add
>> > > > > > > > > some callbacks in KafkaController. This is possible.
>> However,
>> > > the
>> > > > > > > > > controller logic currently is pretty completed. It would
>> > > probably
>> > > > > be
>> > > > > > > > better
>> > > > > > > > > if we clean it up first before adding more complexity to
>> it.
>> > > > Alter
>> > > > > is
>> > > > > > > > even
>> > > > > > > > > trickier. Adding partition is currently handled through
>> > > > > > > KafkaController.
>> > > > > > > > So
>> > > > > > > > > it can be dealt with in a similar way. However, Alter
>> config
>> > is
>> > > > > done
>> > > > > > > > > completely differently. It doesn't go through the
>> controller.
>> > > > > > Instead,
>> > > > > > > > each
>> > > > > > > > > broker listens to ZooKeeper directly. So, it's not clear
>> if
>> > > there
>> > > > > is
>> > > > > > an
>> > > > > > > > > easy way on the broker to figure out whether a config is
>> > > applied
>> > > > on
>> > > > > > > every
>> > > > > > > > > broker.
>> > > > > > > > >
>> > > > > > > > > 2. Delete can potentially take long if a replica to be
>> > deleted
>> > > is
>> > > > > > > > offline.
>> > > > > > > > > PreferredLeader/PartitionReassign can also take long. So,
>> we
>> > > > can't
>> > > > > > > really
>> > > > > > > > > make those requests block on the broker.
>> > > > > > > > >
>> > > > > > > > > As you can see, at this moment it's not easy to make all
>> > admin
>> > > > > > requests
>> > > > > > > > > block on the broker. So, if we want the blocking feature
>> in
>> > the
>> > > > > admin
>> > > > > > > > > utility in the short term, doing the completion check at
>> the
>> > > > admin
>> > > > > > > client
>> > > > > > > > > is probably an easier route, even though it may not be
>> ideal.
>> > > > > > > > >
>> > > > > > > > > Thanks,
>> > > > > > > > >
>> > > > > > > > > Jun
>> > > > > > > > >
>> > > > > > > > > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
>> > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > > > >
>> > > > > > > > > > Jun,
>> > > > > > > > > >
>> > > > > > > > > > I see your point. But wouldn't that lead to a "fat"
>> client
>> > > > > > > > > implementations?
>> > > > > > > > > > Suppose someone would like to implement client for Admin
>> > Wire
>> > > > > > > protocol.
>> > > > > > > > > > Not only people will have to code quite complicated
>> logic
>> > > like
>> > > > > > "send
>> > > > > > > > > > describe
>> > > > > > > > > > request to each broker" (again state machin?) but it
>> will
>> > > also
>> > > > > mean
>> > > > > > > > > people
>> > > > > > > > > > must understand internal kafka logic related to topic
>> > storage
>> > > > and
>> > > > > > how
>> > > > > > > > > > information is propageted from the controller to
>> brokers.
>> > > > > > > > > > I see this like a dilemma between having a concise Wire
>> > > > Protocol
>> > > > > > and
>> > > > > > > > > > self-sufficient API to make client implementations
>> simple.
>> > > > > > > > > > I don't have a win-win solution though.
>> > > > > > > > > >
>> > > > > > > > > > Thanks,
>> > > > > > > > > > Andrii Biletskyi
>> > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <
>> > jun@confluent.io>
>> > > > > > wrote:
>> > > > > > > > > >
>> > > > > > > > > > > For 1), 2) and 3), blocking would probably mean that
>> the
>> > > new
>> > > > > > > metadata
>> > > > > > > > > is
>> > > > > > > > > > > propagated to every broker. To achieve that, the
>> client
>> > can
>> > > > > keep
>> > > > > > > > > issuing
>> > > > > > > > > > > the describe topic request to every broker until it
>> sees
>> > > the
>> > > > > new
>> > > > > > > > > metadata
>> > > > > > > > > > > in the response.
>> > > > > > > > > > >
>> > > > > > > > > > > Thanks,
>> > > > > > > > > > >
>> > > > > > > > > > > Jun
>> > > > > > > > > > >
>> > > > > > > > > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
>> > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > > > > > >
>> > > > > > > > > > > > Hm, actually the ticket you linked, Guozhang,
>> brings as
>> > > > back
>> > > > > > > > > > > > to the problem what should be considered a
>> > post-condition
>> > > > for
>> > > > > > > > > > > > each of the admin commands.
>> > > > > > > > > > > > In my understanding:
>> > > > > > > > > > > >
>> > > > > > > > > > > > 1) CreateTopic - broker created
>> /brokers/topics/<topic>
>> > > > > > > > > > > > (Not the controller picked up changes from zk and
>> > > > broadcasted
>> > > > > > > > > > > > LeaderAndIsr and UpdateMetadata)
>> > > > > > > > > > > >
>> > > > > > > > > > > > 2) AlterTopic - same as 1) - broker changed
>> assignment
>> > > data
>> > > > > > > > > > > > in zookeeper or created admin path for topic config
>> > > change
>> > > > > > > > > > > >
>> > > > > > > > > > > > 3) DeleteTopic - admin path /admin/delete_topics is
>> > > created
>> > > > > > > > > > > >
>> > > > > > > > > > > > 4) ReassignPartitions and PreferredReplica -
>> > > corresponding
>> > > > > > admin
>> > > > > > > > > > > > path is created
>> > > > > > > > > > > >
>> > > > > > > > > > > > Now what can be considered a completed operation
>> from
>> > the
>> > > > > > > client's
>> > > > > > > > > > > > perspective?
>> > > > > > > > > > > > 1) Topic is created once corresponding data is in zk
>> > > > > > > > > > > > (I remember there were some thoughts that it'd be
>> good
>> > to
>> > > > > > > consider
>> > > > > > > > > > > > topic created once all replicas receive information
>> > about
>> > > > it
>> > > > > > and
>> > > > > > > > thus
>> > > > > > > > > > > > clients can produce/consume from it, but as was
>> > discussed
>> > > > > this
>> > > > > > > > seems
>> > > > > > > > > > > > to be a hard thing to do)
>> > > > > > > > > > > >
>> > > > > > > > > > > > 2) Probably same as 1), so right after AlterTopic is
>> > > issued
>> > > > > > > > > > > >
>> > > > > > > > > > > > 3) The topic has been removed from /brokers/topics
>> > > > > > > > > > > >
>> > > > > > > > > > > > 4) ReassignPartitions and PrefferedReplica were
>> > discussed
>> > > > > > > earlier -
>> > > > > > > > > > > > in short the former is completed once partition
>> state
>> > > info
>> > > > in
>> > > > > > zk
>> > > > > > > > > > matches
>> > > > > > > > > > > > reassignment request and admin path is empty, the
>> > latter
>> > > -
>> > > > > once
>> > > > > > > > data
>> > > > > > > > > > > > in zk shows that head of assignned replicas of the
>> > > > partition
>> > > > > > and
>> > > > > > > > > leader
>> > > > > > > > > > > > is the same replica
>> > > > > > > > > > > >
>> > > > > > > > > > > > Thoughts?
>> > > > > > > > > > > >
>> > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > Andrii Biletskyi
>> > > > > > > > > > > >
>> > > > > > > > > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <
>> > > > > > > wangguoz@gmail.com
>> > > > > > > > >
>> > > > > > > > > > > wrote:
>> > > > > > > > > > > >
>> > > > > > > > > > > > > I think while loop is fine for supporting
>> blocking,
>> > > just
>> > > > > that
>> > > > > > > we
>> > > > > > > > > need
>> > > > > > > > > > > to
>> > > > > > > > > > > > > add back off to avoid bombarding brokers with
>> > > > DescribeTopic
>> > > > > > > > > requests.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > Also I have linked KAFKA-1125
>> > > > > > > > > > > > > <https://issues.apache.org/jira/browse/KAFKA-1125
>> >
>> > to
>> > > > your
>> > > > > > > > > proposal,
>> > > > > > > > > > > and
>> > > > > > > > > > > > > when KAFKA-1694 is done this ticket can also be
>> > closed.
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > Guozhang
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi
>> <
>> > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > > Great.
>> > > > > > > > > > > > > > I want to elaborate this a bit more, to see we
>> are
>> > on
>> > > > the
>> > > > > > > same
>> > > > > > > > > page
>> > > > > > > > > > > > > > concerning the client code.
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > So with all topic commands being async a client
>> > > > > > (AdminClient
>> > > > > > > in
>> > > > > > > > > our
>> > > > > > > > > > > > > > case or any other other client people would
>> like to
>> > > > > > > implement)
>> > > > > > > > to
>> > > > > > > > > > > > support
>> > > > > > > > > > > > > > a blocking operation (which seems to be a
>> natural
>> > > > > use-case
>> > > > > > > e.g.
>> > > > > > > > > for
>> > > > > > > > > > > > topic
>> > > > > > > > > > > > > > creation): would have to do:
>> > > > > > > > > > > > > > 1. issue CreateTopicRequest
>> > > > > > > > > > > > > > 2. if successful, in a "while" loop send
>> > > > > > DescribeTopicRequest
>> > > > > > > > and
>> > > > > > > > > > > > > > break the loop once all topics are returned in
>> > > response
>> > > > > (or
>> > > > > > > > upon
>> > > > > > > > > > > > > timeout).
>> > > > > > > > > > > > > > 3. if unsuccessful throw exception
>> > > > > > > > > > > > > > Would it be okay?
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > Andrii Biletskyi
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <
>> > > > > jun@confluent.io
>> > > > > > >
>> > > > > > > > > wrote:
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Andrii,
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > I think you are right. It seems that only
>> > > > > > > ReassignPartitions
>> > > > > > > > > > needs
>> > > > > > > > > > > a
>> > > > > > > > > > > > > > > separate verification request.
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > Jun
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii
>> > Biletskyi <
>> > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Guys,
>> > > > > > > > > > > > > > > > I like this idea too. Let's stick with that.
>> > I'll
>> > > > > > update
>> > > > > > > > KIP
>> > > > > > > > > > > > > > accordingly.
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > I was also thinking we can avoid adding
>> > dedicated
>> > > > > > status
>> > > > > > > > > check
>> > > > > > > > > > > > > > > > requests for topic commands. - We have
>> > everything
>> > > > in
>> > > > > > > > > > > DescribeTopic
>> > > > > > > > > > > > > > > > for that! E.g.:
>> > > > > > > > > > > > > > > > User issued CreateTopic - to check the
>> status
>> > > > client
>> > > > > > > sends
>> > > > > > > > > > > > > > DescribeTopic
>> > > > > > > > > > > > > > > > and checks whether is something returned for
>> > that
>> > > > > > topic.
>> > > > > > > > The
>> > > > > > > > > > same
>> > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > alteration, deletion.
>> > > > > > > > > > > > > > > > Btw, PreferredReplica status can be also
>> > checked
>> > > > with
>> > > > > > > > > > > > > > > DescribeTopicRequest
>> > > > > > > > > > > > > > > > (head of assigned replicas list == leader).
>> > > > > > > > > > > > > > > > For ReassignPartitions as discussed we'll
>> need
>> > to
>> > > > > have
>> > > > > > a
>> > > > > > > > > > separate
>> > > > > > > > > > > > > > > Verify...
>> > > > > > > > > > > > > > > > request.
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > Andrii Biletskyi
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang
>> Wang
>> > <
>> > > > > > > > > > > wangguoz@gmail.com
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > +1 on broker writing to ZK for async
>> > handling.
>> > > I
>> > > > > was
>> > > > > > > > > thinking
>> > > > > > > > > > > > that
>> > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > end state the admin requests would be
>> > > eventually
>> > > > > sent
>> > > > > > > to
>> > > > > > > > > > > > controller
>> > > > > > > > > > > > > > > > either
>> > > > > > > > > > > > > > > > > through re-routing or clients discovering
>> > them,
>> > > > > > instead
>> > > > > > > > of
>> > > > > > > > > > > > letting
>> > > > > > > > > > > > > > > > > controller listen on ZK admin path. But
>> > > thinking
>> > > > > > about
>> > > > > > > > it a
>> > > > > > > > > > > > second
>> > > > > > > > > > > > > > > time,
>> > > > > > > > > > > > > > > > I
>> > > > > > > > > > > > > > > > > think it is actually simpler to let
>> > controller
>> > > > > manage
>> > > > > > > > > > > > > > > > > incoming queued-up admin requests through
>> ZK.
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > Guozhang
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel
>> Koshy <
>> > > > > > > > > > > jjkoshy.w@gmail.com
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > +1 as well. I think it helps to keep the
>> > > > > rerouting
>> > > > > > > > > approach
>> > > > > > > > > > > > > > > orthogonal
>> > > > > > > > > > > > > > > > > > to this KIP.
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM
>> -0700,
>> > Jay
>> > > > > Kreps
>> > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > I'm +1 on Jun's suggestion as long as
>> it
>> > > can
>> > > > > work
>> > > > > > > for
>> > > > > > > > > all
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > > > requests.
>> > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun
>> Rao
>> > <
>> > > > > > > > > > jun@confluent.io
>> > > > > > > > > > > >
>> > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > Andrii,
>> > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > I think we agreed on the following.
>> > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > (a) Admin requests can be sent to
>> and
>> > > > handled
>> > > > > > by
>> > > > > > > > any
>> > > > > > > > > > > > broker.
>> > > > > > > > > > > > > > > > > > > > (b) Admin requests are processed
>> > > > > > asynchronously,
>> > > > > > > at
>> > > > > > > > > > least
>> > > > > > > > > > > > for
>> > > > > > > > > > > > > > > now.
>> > > > > > > > > > > > > > > > > > That is,
>> > > > > > > > > > > > > > > > > > > > when the client gets a response, it
>> > just
>> > > > > means
>> > > > > > > that
>> > > > > > > > > the
>> > > > > > > > > > > > > request
>> > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > initiated, but not necessarily
>> > completed.
>> > > > > Then,
>> > > > > > > > it's
>> > > > > > > > > up
>> > > > > > > > > > > to
>> > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > client
>> > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > issue another request to check the
>> > status
>> > > > for
>> > > > > > > > > > completion.
>> > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > To support (a), we were thinking of
>> > doing
>> > > > > > request
>> > > > > > > > > > > > forwarding
>> > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > controller (utilizing KAFKA-1912).
>> I am
>> > > > > making
>> > > > > > an
>> > > > > > > > > > > > alternative
>> > > > > > > > > > > > > > > > > proposal.
>> > > > > > > > > > > > > > > > > > > > Basically, the broker can just
>> write to
>> > > > > > ZooKeeper
>> > > > > > > > to
>> > > > > > > > > > > inform
>> > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > controller
>> > > > > > > > > > > > > > > > > > > > about the request. For example, to
>> > handle
>> > > > > > > > > > > > > > partitionReassignment,
>> > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > broker
>> > > > > > > > > > > > > > > > > > > > will just write the requested
>> > partitions
>> > > to
>> > > > > > > > > > > > > > > > > /admin/reassign_partitions
>> > > > > > > > > > > > > > > > > > > > (like what AdminUtils currently
>> does)
>> > and
>> > > > > then
>> > > > > > > > send a
>> > > > > > > > > > > > > response
>> > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > client. This shouldn't take long and
>> > the
>> > > > > > > > > implementation
>> > > > > > > > > > > > will
>> > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > simpler
>> > > > > > > > > > > > > > > > > > > > than forwarding the requests to the
>> > > > > controller
>> > > > > > > > > through
>> > > > > > > > > > > RPC.
>> > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > Jun
>> > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM,
>> Andrii
>> > > > > > > Biletskyi <
>> > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > Jun,
>> > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > I might be wrong but didn't we
>> agree
>> > we
>> > > > > will
>> > > > > > > let
>> > > > > > > > > any
>> > > > > > > > > > > > broker
>> > > > > > > > > > > > > > > from
>> > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > cluster handle *long-running*
>> admin
>> > > > > requests
>> > > > > > > (at
>> > > > > > > > > this
>> > > > > > > > > > > > time
>> > > > > > > > > > > > > > > > > > > > preferredReplica
>> > > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > reassignPartitions), via zk admin
>> > path.
>> > > > > Thus
>> > > > > > > > > > > CreateTopics
>> > > > > > > > > > > > > etc
>> > > > > > > > > > > > > > > > > should
>> > > > > > > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > > > > > sent
>> > > > > > > > > > > > > > > > > > > > > only to the controller.
>> > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
>> > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM,
>> Jun
>> > > > Rao <
>> > > > > > > > > > > > > jun@confluent.io>
>> > > > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > Joel, Andril,
>> > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > I think we agreed that those
>> admin
>> > > > > requests
>> > > > > > > can
>> > > > > > > > > be
>> > > > > > > > > > > > issued
>> > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > any
>> > > > > > > > > > > > > > > > > > > > broker.
>> > > > > > > > > > > > > > > > > > > > > > Because of that, there doesn't
>> seem
>> > > to
>> > > > > be a
>> > > > > > > > > strong
>> > > > > > > > > > > need
>> > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > know
>> > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > controller. So, perhaps we can
>> > > proceed
>> > > > by
>> > > > > > not
>> > > > > > > > > > making
>> > > > > > > > > > > > any
>> > > > > > > > > > > > > > > change
>> > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > format of TMR right now. When we
>> > > start
>> > > > > > using
>> > > > > > > > > create
>> > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > request
>> > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > producer, we will need a new
>> > version
>> > > of
>> > > > > TMR
>> > > > > > > > that
>> > > > > > > > > > > > doesn't
>> > > > > > > > > > > > > > > > trigger
>> > > > > > > > > > > > > > > > > > auto
>> > > > > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > creation. But that can be done
>> > later.
>> > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > As a first cut implementation, I
>> > > think
>> > > > > the
>> > > > > > > > broker
>> > > > > > > > > > can
>> > > > > > > > > > > > > just
>> > > > > > > > > > > > > > > > write
>> > > > > > > > > > > > > > > > > > to ZK
>> > > > > > > > > > > > > > > > > > > > > > directly for
>> > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > >
>> > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
>> > > > > > > > > > > > > > > > > > > > > > requests, instead of forwarding
>> > them
>> > > to
>> > > > > the
>> > > > > > > > > > > controller.
>> > > > > > > > > > > > > > This
>> > > > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > > > > > simplify
>> > > > > > > > > > > > > > > > > > > > > > the implementation on the broker
>> > > side.
>> > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > Jun
>> > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58
>> AM,
>> > > Joel
>> > > > > > Koshy
>> > > > > > > <
>> > > > > > > > > > > > > > > > > jjkoshy.w@gmail.com>
>> > > > > > > > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > (Thanks Andrii for the
>> summary)
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > For (1) yes we will circle
>> back
>> > on
>> > > > that
>> > > > > > > > shortly
>> > > > > > > > > > > after
>> > > > > > > > > > > > > > > syncing
>> > > > > > > > > > > > > > > > > up
>> > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > person. I think it is close to
>> > > > getting
>> > > > > > > > > committed
>> > > > > > > > > > > > > although
>> > > > > > > > > > > > > > > > > > development
>> > > > > > > > > > > > > > > > > > > > > > > for KAFKA-1927 can probably
>> begin
>> > > > > without
>> > > > > > > it.
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > There is one more item we
>> covered
>> > > at
>> > > > > the
>> > > > > > > > > hangout.
>> > > > > > > > > > > > i.e.,
>> > > > > > > > > > > > > > > > whether
>> > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > want to add the coordinator to
>> > the
>> > > > > topic
>> > > > > > > > > metadata
>> > > > > > > > > > > > > > response
>> > > > > > > > > > > > > > > or
>> > > > > > > > > > > > > > > > > > provide
>> > > > > > > > > > > > > > > > > > > > > > > a clearer
>> ClusterMetadataRequest.
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > There are two reasons I think
>> we
>> > > > should
>> > > > > > try
>> > > > > > > > and
>> > > > > > > > > > > avoid
>> > > > > > > > > > > > > > > adding
>> > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > field:
>> > > > > > > > > > > > > > > > > > > > > > > - It is irrelevant to topic
>> > > metadata
>> > > > > > > > > > > > > > > > > > > > > > > - If we finally do request
>> > > rerouting
>> > > > in
>> > > > > > > Kafka
>> > > > > > > > > > then
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > > field
>> > > > > > > > > > > > > > > > > > would
>> > > > > > > > > > > > > > > > > > > > add
>> > > > > > > > > > > > > > > > > > > > > > >   little to no value. (It
>> still
>> > > helps
>> > > > > to
>> > > > > > > > have a
>> > > > > > > > > > > > > separate
>> > > > > > > > > > > > > > > > > > > > > > >   ClusterMetadataRequest to
>> query
>> > > for
>> > > > > > > > > > cluster-wide
>> > > > > > > > > > > > > > > > information
>> > > > > > > > > > > > > > > > > > such
>> > > > > > > > > > > > > > > > > > > > as
>> > > > > > > > > > > > > > > > > > > > > > >   'which broker is the
>> > controller?'
>> > > > as
>> > > > > > Joe
>> > > > > > > > > > > > mentioned.)
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > I think it would be cleaner to
>> > have
>> > > > an
>> > > > > > > > explicit
>> > > > > > > > > > > > > > > > > > > > ClusterMetadataRequest
>> > > > > > > > > > > > > > > > > > > > > > > that you can send to any
>> broker
>> > in
>> > > > > order
>> > > > > > to
>> > > > > > > > > > obtain
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > controller
>> > > > > > > > > > > > > > > > > > > > (and
>> > > > > > > > > > > > > > > > > > > > > > > in the future possibly other
>> > > > > cluster-wide
>> > > > > > > > > > > > > information). I
>> > > > > > > > > > > > > > > > think
>> > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > main argument against doing
>> this
>> > > and
>> > > > > > > instead
>> > > > > > > > > > adding
>> > > > > > > > > > > > it
>> > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > metadata response was
>> > convenience -
>> > > > > i.e.,
>> > > > > > > you
>> > > > > > > > > > don't
>> > > > > > > > > > > > > have
>> > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > discover
>> > > > > > > > > > > > > > > > > > > > > > > the controller in advance.
>> > > However, I
>> > > > > > don't
>> > > > > > > > see
>> > > > > > > > > > > much
>> > > > > > > > > > > > > > actual
>> > > > > > > > > > > > > > > > > > > > > > > benefit/convenience in this
>> and
>> > in
>> > > > fact
>> > > > > > > think
>> > > > > > > > > it
>> > > > > > > > > > > is a
>> > > > > > > > > > > > > > > > > non-issue.
>> > > > > > > > > > > > > > > > > > Let
>> > > > > > > > > > > > > > > > > > > > > > > me know if I'm overlooking
>> > > something
>> > > > > > here.
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > As an example, say we need to
>> > > > initiate
>> > > > > > > > > partition
>> > > > > > > > > > > > > > > reassignment
>> > > > > > > > > > > > > > > > > by
>> > > > > > > > > > > > > > > > > > > > > > > issuing the new
>> > > > > ReassignPartitionsRequest
>> > > > > > > to
>> > > > > > > > > the
>> > > > > > > > > > > > > > controller
>> > > > > > > > > > > > > > > > > > (assume
>> > > > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > already have the desired
>> manual
>> > > > > partition
>> > > > > > > > > > > > assignment).
>> > > > > > > > > > > > > > If
>> > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > are to
>> > > > > > > > > > > > > > > > > > > > > > > augment topic metadata
>> response
>> > > then
>> > > > > the
>> > > > > > > flow
>> > > > > > > > > be
>> > > > > > > > > > > > > > something
>> > > > > > > > > > > > > > > > like
>> > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > :
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > - Issue topic metadata
>> request to
>> > > any
>> > > > > > > broker
>> > > > > > > > > (and
>> > > > > > > > > > > > > > discover
>> > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > >   controller
>> > > > > > > > > > > > > > > > > > > > > > > - Connect to controller if
>> > required
>> > > > > > (i.e.,
>> > > > > > > if
>> > > > > > > > > the
>> > > > > > > > > > > > > broker
>> > > > > > > > > > > > > > > > above
>> > > > > > > > > > > > > > > > > !=
>> > > > > > > > > > > > > > > > > > > > > > >   controller)
>> > > > > > > > > > > > > > > > > > > > > > > - Issue the partition
>> > reassignment
>> > > > > > request
>> > > > > > > to
>> > > > > > > > > the
>> > > > > > > > > > > > > > > controller.
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > With an explicit cluster
>> metadata
>> > > > > request
>> > > > > > > it
>> > > > > > > > > > would
>> > > > > > > > > > > > be:
>> > > > > > > > > > > > > > > > > > > > > > > - Issue cluster metadata
>> request
>> > to
>> > > > any
>> > > > > > > > broker
>> > > > > > > > > > > > > > > > > > > > > > > - Connect to controller if
>> > required
>> > > > > > (i.e.,
>> > > > > > > if
>> > > > > > > > > the
>> > > > > > > > > > > > > broker
>> > > > > > > > > > > > > > > > above
>> > > > > > > > > > > > > > > > > !=
>> > > > > > > > > > > > > > > > > > > > > > >   controller)
>> > > > > > > > > > > > > > > > > > > > > > > - Issue the partition
>> > reassignment
>> > > > > > request
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > So it seems to add little
>> > practical
>> > > > > value
>> > > > > > > and
>> > > > > > > > > > > bloats
>> > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > metadata
>> > > > > > > > > > > > > > > > > > > > > > > response with an irrelevant
>> > detail.
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > The other angle to this is the
>> > > > > following
>> > > > > > -
>> > > > > > > is
>> > > > > > > > > it
>> > > > > > > > > > a
>> > > > > > > > > > > > > matter
>> > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > naming?
>> > > > > > > > > > > > > > > > > > > > > > > Should we just rename topic
>> > > metadata
>> > > > > > > > > > > request/response
>> > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > just
>> > > > > > > > > > > > > > > > > > > > > > > MetadataRequest/Response and
>> add
>> > > > > cluster
>> > > > > > > > > metadata
>> > > > > > > > > > > to
>> > > > > > > > > > > > > it?
>> > > > > > > > > > > > > > By
>> > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > same
>> > > > > > > > > > > > > > > > > > > > > > > token should we also allow
>> > querying
>> > > > for
>> > > > > > the
>> > > > > > > > > > > consumer
>> > > > > > > > > > > > > > > > > coordinator
>> > > > > > > > > > > > > > > > > > (and
>> > > > > > > > > > > > > > > > > > > > > > > in future transaction
>> > coordinator)
>> > > as
>> > > > > > well?
>> > > > > > > > > This
>> > > > > > > > > > > > leads
>> > > > > > > > > > > > > > to a
>> > > > > > > > > > > > > > > > > > bloated
>> > > > > > > > > > > > > > > > > > > > > > > request which isn't very
>> > appealing
>> > > > and
>> > > > > > > > > altogether
>> > > > > > > > > > > > > > > confusing.
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > Joel
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at
>> > 09:34:12AM
>> > > > > -0700,
>> > > > > > > Jun
>> > > > > > > > > Rao
>> > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > Andri,
>> > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > Thanks for the summary.
>> > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > 1. I just realized that in
>> > order
>> > > to
>> > > > > > start
>> > > > > > > > > > working
>> > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > > > KAFKA-1927, we
>> > > > > > > > > > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > > > > > > > > need to merge the changes to
>> > > > > > > > > > OffsetCommitRequest
>> > > > > > > > > > > > > (from
>> > > > > > > > > > > > > > > > 0.8.2)
>> > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > trunk.
>> > > > > > > > > > > > > > > > > > > > > > > > This is planned to be done
>> as
>> > > part
>> > > > of
>> > > > > > > > > > KAFKA-1634.
>> > > > > > > > > > > > So,
>> > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > > need
>> > > > > > > > > > > > > > > > > > > > > > > Guozhang
>> > > > > > > > > > > > > > > > > > > > > > > > and Joel's help to wrap this
>> > up.
>> > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > 2. Thinking about this a bit
>> > > more,
>> > > > if
>> > > > > > the
>> > > > > > > > > > > semantic
>> > > > > > > > > > > > of
>> > > > > > > > > > > > > > > those
>> > > > > > > > > > > > > > > > > > "write"
>> > > > > > > > > > > > > > > > > > > > > > > > requests is async (i.e.,
>> after
>> > > the
>> > > > > > client
>> > > > > > > > > gets
>> > > > > > > > > > a
>> > > > > > > > > > > > > > > response,
>> > > > > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > > > > just
>> > > > > > > > > > > > > > > > > > > > > > means
>> > > > > > > > > > > > > > > > > > > > > > > > that the operation is
>> > initiated,
>> > > > but
>> > > > > > not
>> > > > > > > > > > > > necessarily
>> > > > > > > > > > > > > > > > > > completed), we
>> > > > > > > > > > > > > > > > > > > > > > don't
>> > > > > > > > > > > > > > > > > > > > > > > > really need to forward the
>> > > requests
>> > > > > to
>> > > > > > > the
>> > > > > > > > > > > > > controller.
>> > > > > > > > > > > > > > > > > > Instead, the
>> > > > > > > > > > > > > > > > > > > > > > > > receiving broker can just
>> write
>> > > the
>> > > > > > > > operation
>> > > > > > > > > > to
>> > > > > > > > > > > ZK
>> > > > > > > > > > > > > as
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > admin
>> > > > > > > > > > > > > > > > > > > > > > command
>> > > > > > > > > > > > > > > > > > > > > > > > line tool previously does.
>> This
>> > > > will
>> > > > > > > > simplify
>> > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > implementation.
>> > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > 8. There is another
>> > > implementation
>> > > > > > detail
>> > > > > > > > for
>> > > > > > > > > > > > > describe
>> > > > > > > > > > > > > > > > topic.
>> > > > > > > > > > > > > > > > > > > > > Ideally,
>> > > > > > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > want to read the topic
>> config
>> > > from
>> > > > > the
>> > > > > > > > broker
>> > > > > > > > > > > > cache,
>> > > > > > > > > > > > > > > > instead
>> > > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > > > > > > ZooKeeper.
>> > > > > > > > > > > > > > > > > > > > > > > > Currently, every broker
>> reads
>> > the
>> > > > > > > > topic-level
>> > > > > > > > > > > > config
>> > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > all
>> > > > > > > > > > > > > > > > > > > > topics.
>> > > > > > > > > > > > > > > > > > > > > > > > However, it ignores those
>> for
>> > > > topics
>> > > > > > not
>> > > > > > > > > hosted
>> > > > > > > > > > > on
>> > > > > > > > > > > > > > > itself.
>> > > > > > > > > > > > > > > > > So,
>> > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > may
>> > > > > > > > > > > > > > > > > > > > > > > need
>> > > > > > > > > > > > > > > > > > > > > > > > to change
>> TopicConfigManager a
>> > > bit
>> > > > so
>> > > > > > > that
>> > > > > > > > it
>> > > > > > > > > > > > caches
>> > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > configs
>> > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > all
>> > > > > > > > > > > > > > > > > > > > > > > > topics.
>> > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > Jun
>> > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13
>> > PM,
>> > > > > Andrii
>> > > > > > > > > > > Biletskyi <
>> > > > > > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly
>> >
>> > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > Guys,
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > Thanks for a great
>> > discussion!
>> > > > > > > > > > > > > > > > > > > > > > > > > Here are the actions
>> points:
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala
>> > > > requests
>> > > > > > > > > objects,
>> > > > > > > > > > > use
>> > > > > > > > > > > > > java
>> > > > > > > > > > > > > > > > > > protocol
>> > > > > > > > > > > > > > > > > > > > > > > definitions.
>> > > > > > > > > > > > > > > > > > > > > > > > >     A: Gwen kindly took
>> that
>> > > > > > > > (KAFKA-1927).
>> > > > > > > > > > It's
>> > > > > > > > > > > > > > > important
>> > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > speed
>> > > > > > > > > > > > > > > > > > > > > up
>> > > > > > > > > > > > > > > > > > > > > > > > > review procedure
>> > > > > > > > > > > > > > > > > > > > > > > > >          there since this
>> > > ticket
>> > > > > > blocks
>> > > > > > > > > other
>> > > > > > > > > > > > > > important
>> > > > > > > > > > > > > > > > > > changes.
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute
>> > > facility
>> > > > > vs
>> > > > > > > > client
>> > > > > > > > > > > > > > maintaining
>> > > > > > > > > > > > > > > > > > cluster
>> > > > > > > > > > > > > > > > > > > > > > state.
>> > > > > > > > > > > > > > > > > > > > > > > > >     A: Jay has added
>> pseudo
>> > > code
>> > > > to
>> > > > > > > > > > KAFKA-1912
>> > > > > > > > > > > -
>> > > > > > > > > > > > > need
>> > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > consider
>> > > > > > > > > > > > > > > > > > > > > > > whether
>> > > > > > > > > > > > > > > > > > > > > > > > > this will be
>> > > > > > > > > > > > > > > > > > > > > > > > >         easy to implement
>> as
>> > a
>> > > > > > > > server-side
>> > > > > > > > > > > > feature
>> > > > > > > > > > > > > > > > > (comments
>> > > > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > > > > > > > > welcomed!).
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > 3. Q: Controller field in
>> > wire
>> > > > > > > protocol.
>> > > > > > > > > > > > > > > > > > > > > > > > >     A: This might be
>> useful
>> > for
>> > > > > > > clients,
>> > > > > > > > > add
>> > > > > > > > > > > this
>> > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > TopicMetadataResponse
>> > > > > > > > > > > > > > > > > > > > > > > > > (already in KIP).
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic
>> > creation
>> > > > > from
>> > > > > > > TMR.
>> > > > > > > > > > > > > > > > > > > > > > > > >     A: I will add
>> proposed by
>> > > Jun
>> > > > > > > > solution
>> > > > > > > > > > > (using
>> > > > > > > > > > > > > > > > clientId
>> > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > that)
>> > > > > > > > > > > > > > > > > > > > > > > to the
>> > > > > > > > > > > > > > > > > > > > > > > > > KIP.
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > 5. Q: Bumping new
>> versions of
>> > > TMR
>> > > > > vs
>> > > > > > > > > grabbing
>> > > > > > > > > > > all
>> > > > > > > > > > > > > > > > protocol
>> > > > > > > > > > > > > > > > > > > > changes
>> > > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > one
>> > > > > > > > > > > > > > > > > > > > > > > > > version.
>> > > > > > > > > > > > > > > > > > > > > > > > >     A: It was decided to
>> try
>> > to
>> > > > > > gather
>> > > > > > > > all
>> > > > > > > > > > > > changes
>> > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > protocol
>> > > > > > > > > > > > > > > > > > > > > > (before
>> > > > > > > > > > > > > > > > > > > > > > > > > release).
>> > > > > > > > > > > > > > > > > > > > > > > > >         In case of TMR it
>> > worth
>> > > > > > > checking:
>> > > > > > > > > > > > > KAFKA-2020
>> > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > KIP-13
>> > > > > > > > > > > > > > > > > > > > > > > (quotas)
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed
>> to
>> > > > > > deserialize
>> > > > > > > > > > user's
>> > > > > > > > > > > > > input
>> > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > CLI
>> > > > > > > > > > > > > > > > > > tool.
>> > > > > > > > > > > > > > > > > > > > > > > > >     A: Use jackson for
>> that,
>> > > > /tools
>> > > > > > > > project
>> > > > > > > > > > is
>> > > > > > > > > > > a
>> > > > > > > > > > > > > > > separate
>> > > > > > > > > > > > > > > > > > jar so
>> > > > > > > > > > > > > > > > > > > > > > > shouldn't
>> > > > > > > > > > > > > > > > > > > > > > > > > be a big deal.
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > 7.  Q:
>> > VerifyReassingPartitions
>> > > > vs
>> > > > > > > > generic
>> > > > > > > > > > > status
>> > > > > > > > > > > > > > check
>> > > > > > > > > > > > > > > > > > command.
>> > > > > > > > > > > > > > > > > > > > > > > > >      A: For long-running
>> > > requests
>> > > > > > like
>> > > > > > > > > > reassign
>> > > > > > > > > > > > > > > > partitions
>> > > > > > > > > > > > > > > > > > > > > *progress*
>> > > > > > > > > > > > > > > > > > > > > > > check
>> > > > > > > > > > > > > > > > > > > > > > > > > request is useful,
>> > > > > > > > > > > > > > > > > > > > > > > > >          it makes sense to
>> > > > > introduce
>> > > > > > > it.
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > >  Please add, correct me
>> if I
>> > > > missed
>> > > > > > > > > > something.
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at
>> 6:20
>> > > PM,
>> > > > > > Andrii
>> > > > > > > > > > > > Biletskyi <
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> andrii.biletskyi@stealth.ly>
>> > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > > Joel,
>> > > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > > You are right, I removed
>> > > > > > > > ClusterMetadata
>> > > > > > > > > > > > because
>> > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > have
>> > > > > > > > > > > > > > > > > > > > > partially
>> > > > > > > > > > > > > > > > > > > > > > > > > > what we need in
>> > > TopicMetadata.
>> > > > > > Also,
>> > > > > > > as
>> > > > > > > > > Jay
>> > > > > > > > > > > > > pointed
>> > > > > > > > > > > > > > > out
>> > > > > > > > > > > > > > > > > > > > earlier,
>> > > > > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > > would like to have
>> > > "orthogonal"
>> > > > > > API,
>> > > > > > > > but
>> > > > > > > > > at
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > same
>> > > > > > > > > > > > > > > > time
>> > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > need
>> > > > > > > > > > > > > > > > > > > > > > > > > > to be backward
>> compatible.
>> > > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > > But I like your idea and
>> > even
>> > > > > have
>> > > > > > > some
>> > > > > > > > > > other
>> > > > > > > > > > > > > > > arguments
>> > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > option:
>> > > > > > > > > > > > > > > > > > > > > > > > > > There is also
>> > > > > DescribeTopicRequest
>> > > > > > > > which
>> > > > > > > > > > was
>> > > > > > > > > > > > > > proposed
>> > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > KIP,
>> > > > > > > > > > > > > > > > > > > > > > > > > > it returns topic
>> configs,
>> > > > > > partitions,
>> > > > > > > > > > > > replication
>> > > > > > > > > > > > > > > > factor
>> > > > > > > > > > > > > > > > > > plus
>> > > > > > > > > > > > > > > > > > > > > > > partition
>> > > > > > > > > > > > > > > > > > > > > > > > > > ISR, ASR,
>> > > > > > > > > > > > > > > > > > > > > > > > > > leader replica. The
>> later
>> > > part
>> > > > is
>> > > > > > > > really
>> > > > > > > > > > > > already
>> > > > > > > > > > > > > > > there
>> > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
>> > > > > > > > > > > > > > > > > > > > > > > > > > So again we'll have to
>> add
>> > > > stuff
>> > > > > to
>> > > > > > > > TMR,
>> > > > > > > > > > not
>> > > > > > > > > > > to
>> > > > > > > > > > > > > > > > duplicate
>> > > > > > > > > > > > > > > > > > some
>> > > > > > > > > > > > > > > > > > > > > info
>> > > > > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > > > > newly added requests.
>> > > However,
>> > > > > this
>> > > > > > > way
>> > > > > > > > > > we'll
>> > > > > > > > > > > > end
>> > > > > > > > > > > > > > up
>> > > > > > > > > > > > > > > > with
>> > > > > > > > > > > > > > > > > > > > > "monster"
>> > > > > > > > > > > > > > > > > > > > > > > > > > request which returns
>> > cluster
>> > > > > > > metadata,
>> > > > > > > > > > topic
>> > > > > > > > > > > > > > > > replication
>> > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > config
>> > > > > > > > > > > > > > > > > > > > > > > info
>> > > > > > > > > > > > > > > > > > > > > > > > > > plus partition
>> replication
>> > > > data.
>> > > > > > > Seems
>> > > > > > > > > > > logical
>> > > > > > > > > > > > to
>> > > > > > > > > > > > > > > split
>> > > > > > > > > > > > > > > > > > TMR to
>> > > > > > > > > > > > > > > > > > > > > > > > > > - ClusterMetadata
>> (brokers
>> > +
>> > > > > > > > controller,
>> > > > > > > > > > > maybe
>> > > > > > > > > > > > > smth
>> > > > > > > > > > > > > > > > else)
>> > > > > > > > > > > > > > > > > > > > > > > > > > - TopicMetadata (topic
>> > info +
>> > > > > > > partition
>> > > > > > > > > > > > details)
>> > > > > > > > > > > > > > > > > > > > > > > > > > But since current TMR is
>> > > > involved
>> > > > > > in
>> > > > > > > > lots
>> > > > > > > > > > of
>> > > > > > > > > > > > > places
>> > > > > > > > > > > > > > > > > > (including
>> > > > > > > > > > > > > > > > > > > > > > > network
>> > > > > > > > > > > > > > > > > > > > > > > > > > client,
>> > > > > > > > > > > > > > > > > > > > > > > > > > as I understand) this
>> might
>> > > be
>> > > > > very
>> > > > > > > > > serious
>> > > > > > > > > > > > > change
>> > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > > > > > > > probably
>> > > > > > > > > > > > > > > > > > > > > > > makes
>> > > > > > > > > > > > > > > > > > > > > > > > > > sense to stick with
>> current
>> > > > > > approach.
>> > > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
>> > > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at
>> > 5:29
>> > > > PM,
>> > > > > > Joel
>> > > > > > > > > > Koshy <
>> > > > > > > > > > > > > > > > > > > > jjkoshy.w@gmail.com
>> > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> I may be missing some
>> > > context
>> > > > > but
>> > > > > > > > > > hopefully
>> > > > > > > > > > > > this
>> > > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > > also be
>> > > > > > > > > > > > > > > > > > > > > > > covered
>> > > > > > > > > > > > > > > > > > > > > > > > > >> today: I thought the
>> > earlier
>> > > > > > > proposal
>> > > > > > > > > > where
>> > > > > > > > > > > > > there
>> > > > > > > > > > > > > > > was
>> > > > > > > > > > > > > > > > an
>> > > > > > > > > > > > > > > > > > > > > explicit
>> > > > > > > > > > > > > > > > > > > > > > > > > >> ClusterMetadata request
>> > was
>> > > > > > clearer
>> > > > > > > > and
>> > > > > > > > > > > > > explicit.
>> > > > > > > > > > > > > > > > During
>> > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > course
>> > > > > > > > > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> this thread I think the
>> > > > > conclusion
>> > > > > > > was
>> > > > > > > > > > that
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > main
>> > > > > > > > > > > > > > > > > need
>> > > > > > > > > > > > > > > > > > was
>> > > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> controller information
>> and
>> > > > that
>> > > > > > can
>> > > > > > > be
>> > > > > > > > > > > rolled
>> > > > > > > > > > > > > into
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > metadata
>> > > > > > > > > > > > > > > > > > > > > > > > > >> response but that
>> seems a
>> > > bit
>> > > > > > > > irrelevant
>> > > > > > > > > > to
>> > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > metadata.
>> > > > > > > > > > > > > > > > > > > > > FWIW I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> think the full
>> broker-list
>> > > is
>> > > > > also
>> > > > > > > > > > > irrelevant
>> > > > > > > > > > > > to
>> > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > metadata,
>> > > > > > > > > > > > > > > > > > > > > > but
>> > > > > > > > > > > > > > > > > > > > > > > > > >> it is already there
>> and in
>> > > > use.
>> > > > > I
>> > > > > > > > think
>> > > > > > > > > > > there
>> > > > > > > > > > > > is
>> > > > > > > > > > > > > > > still
>> > > > > > > > > > > > > > > > > > room
>> > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > an
>> > > > > > > > > > > > > > > > > > > > > > > > > >> explicit
>> ClusterMetadata
>> > > > request
>> > > > > > > since
>> > > > > > > > > > there
>> > > > > > > > > > > > may
>> > > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > other
>> > > > > > > > > > > > > > > > > > > > > > > > > >> cluster-level
>> information
>> > > that
>> > > > > we
>> > > > > > > may
>> > > > > > > > > want
>> > > > > > > > > > > to
>> > > > > > > > > > > > > add
>> > > > > > > > > > > > > > > over
>> > > > > > > > > > > > > > > > > > time
>> > > > > > > > > > > > > > > > > > > > (and
>> > > > > > > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> have nothing to do with
>> > > topic
>> > > > > > > > metadata).
>> > > > > > > > > > > > > > > > > > > > > > > > > >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at
>> > > > > 02:45:30PM
>> > > > > > > > > +0200,
>> > > > > > > > > > > > Andrii
>> > > > > > > > > > > > > > > > > Biletskyi
>> > > > > > > > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > Jun,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > 101. Okay, if you say
>> > that
>> > > > > such
>> > > > > > > use
>> > > > > > > > > case
>> > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > important. I
>> > > > > > > > > > > > > > > > > > > > also
>> > > > > > > > > > > > > > > > > > > > > > > think
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > using clientId for
>> these
>> > > > > > purposes
>> > > > > > > is
>> > > > > > > > > > fine
>> > > > > > > > > > > -
>> > > > > > > > > > > > if
>> > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > already
>> > > > > > > > > > > > > > > > > > > > have
>> > > > > > > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> field
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > as part of all Wire
>> > > protocol
>> > > > > > > > messages,
>> > > > > > > > > > why
>> > > > > > > > > > > > not
>> > > > > > > > > > > > > > use
>> > > > > > > > > > > > > > > > > that.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > I will update KIP-4
>> page
>> > > if
>> > > > > > nobody
>> > > > > > > > has
>> > > > > > > > > > > other
>> > > > > > > > > > > > > > ideas
>> > > > > > > > > > > > > > > > > > (which
>> > > > > > > > > > > > > > > > > > > > may
>> > > > > > > > > > > > > > > > > > > > > > > come up
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > during the call
>> today).
>> > > > > > > > > > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll
>> update
>> > > the
>> > > > > KIP
>> > > > > > > > > > > > accordingly.
>> > > > > > > > > > > > > I
>> > > > > > > > > > > > > > > > think
>> > > > > > > > > > > > > > > > > > we can
>> > > > > > > > > > > > > > > > > > > > > add
>> > > > > > > > > > > > > > > > > > > > > > > new,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > fine-grained error
>> codes
>> > > if
>> > > > > some
>> > > > > > > > error
>> > > > > > > > > > > code
>> > > > > > > > > > > > > > > received
>> > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > specific
>> > > > > > > > > > > > > > > > > > > > > > > case
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > won't give enough
>> > context
>> > > to
>> > > > > > > return
>> > > > > > > > a
>> > > > > > > > > > > > > > descriptive
>> > > > > > > > > > > > > > > > > error
>> > > > > > > > > > > > > > > > > > > > > message
>> > > > > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> user.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > Look forward to
>> > discussing
>> > > > all
>> > > > > > > > > > outstanding
>> > > > > > > > > > > > > > issues
>> > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > detail
>> > > > > > > > > > > > > > > > > > > > > > today
>> > > > > > > > > > > > > > > > > > > > > > > > > during
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > the call.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
>> > > > > > > > > > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015
>> at
>> > > > 10:59
>> > > > > > PM,
>> > > > > > > > Jun
>> > > > > > > > > > Rao
>> > > > > > > > > > > <
>> > > > > > > > > > > > > > > > > > jun@confluent.io
>> > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 101. There may be a
>> > use
>> > > > case
>> > > > > > > where
>> > > > > > > > > you
>> > > > > > > > > > > > only
>> > > > > > > > > > > > > > want
>> > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > topics
>> > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> created
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > manually by admins.
>> > > > > Currently,
>> > > > > > > you
>> > > > > > > > > can
>> > > > > > > > > > > do
>> > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > by
>> > > > > > > > > > > > > > > > > > > > disabling
>> > > > > > > > > > > > > > > > > > > > > > auto
>> > > > > > > > > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > creation and issue
>> > topic
>> > > > > > > creation
>> > > > > > > > > from
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > TopicCommand.
>> > > > > > > > > > > > > > > > > > > > If
>> > > > > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> disable auto
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > topic creation
>> > > completely
>> > > > on
>> > > > > > the
>> > > > > > > > > > broker
>> > > > > > > > > > > > and
>> > > > > > > > > > > > > > > don't
>> > > > > > > > > > > > > > > > > > have a
>> > > > > > > > > > > > > > > > > > > > way
>> > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> distinguish
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > between topic
>> creation
>> > > > > > requests
>> > > > > > > > from
>> > > > > > > > > > the
>> > > > > > > > > > > > > > regular
>> > > > > > > > > > > > > > > > > > clients
>> > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> admin, we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > can't support
>> manual
>> > > topic
>> > > > > > > > creation
>> > > > > > > > > > any
>> > > > > > > > > > > > > more.
>> > > > > > > > > > > > > > I
>> > > > > > > > > > > > > > > > was
>> > > > > > > > > > > > > > > > > > > > thinking
>> > > > > > > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> another
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > way of
>> distinguishing
>> > > the
>> > > > > > > clients
>> > > > > > > > > > making
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > creation
>> > > > > > > > > > > > > > > > > > > > > > > requests
>> > > > > > > > > > > > > > > > > > > > > > > > > >> is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > using clientId. For
>> > > > example,
>> > > > > > the
>> > > > > > > > > admin
>> > > > > > > > > > > > tool
>> > > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > set
>> > > > > > > > > > > > > > > > > > it to
>> > > > > > > > > > > > > > > > > > > > > > > something
>> > > > > > > > > > > > > > > > > > > > > > > > > >> like
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin and the
>> broker
>> > can
>> > > > > treat
>> > > > > > > > that
>> > > > > > > > > > > > clientId
>> > > > > > > > > > > > > > > > > > specially.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Also, there is a
>> > related
>> > > > > > > > discussion
>> > > > > > > > > in
>> > > > > > > > > > > > > > > KAFKA-2020.
>> > > > > > > > > > > > > > > > > > > > > Currently,
>> > > > > > > > > > > > > > > > > > > > > > > we do
>> > > > > > > > > > > > > > > > > > > > > > > > > >> the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > following in
>> > > > > > > > TopicMetadataResponse:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 1. If leader is not
>> > > > > available,
>> > > > > > > we
>> > > > > > > > > set
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > > > partition
>> > > > > > > > > > > > > > > > > > level
>> > > > > > > > > > > > > > > > > > > > > > error
>> > > > > > > > > > > > > > > > > > > > > > > code
>> > > > > > > > > > > > > > > > > > > > > > > > > >> to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 2. If a non-leader
>> > > replica
>> > > > > is
>> > > > > > > not
>> > > > > > > > > > > > available,
>> > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > take
>> > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > replica
>> > > > > > > > > > > > > > > > > > > > > > > > > out
>> > > > > > > > > > > > > > > > > > > > > > > > > >> of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > the assigned
>> replica
>> > > list
>> > > > > and
>> > > > > > > isr
>> > > > > > > > in
>> > > > > > > > > > the
>> > > > > > > > > > > > > > > response.
>> > > > > > > > > > > > > > > > > As
>> > > > > > > > > > > > > > > > > > an
>> > > > > > > > > > > > > > > > > > > > > > > indication
>> > > > > > > > > > > > > > > > > > > > > > > > > >> for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > doing that, we set
>> the
>> > > > > > partition
>> > > > > > > > > level
>> > > > > > > > > > > > error
>> > > > > > > > > > > > > > > code
>> > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > This has a few
>> > problems.
>> > > > > > First,
>> > > > > > > > > > > > > > > > ReplicaNotAvailable
>> > > > > > > > > > > > > > > > > > > > probably
>> > > > > > > > > > > > > > > > > > > > > > > > > >> shouldn't be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > an error, at least
>> for
>> > > the
>> > > > > > > normal
>> > > > > > > > > > > > > > > > producer/consumer
>> > > > > > > > > > > > > > > > > > > > clients
>> > > > > > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > just
>> > > > > > > > > > > > > > > > > > > > > > > > > >> want
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to find out the
>> > leader.
>> > > > > > Second,
>> > > > > > > it
>> > > > > > > > > can
>> > > > > > > > > > > > > happen
>> > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > both
>> > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > leader
>> > > > > > > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > another replica are
>> > not
>> > > > > > > available
>> > > > > > > > at
>> > > > > > > > > > the
>> > > > > > > > > > > > > same
>> > > > > > > > > > > > > > > > time.
>> > > > > > > > > > > > > > > > > > There
>> > > > > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > > > no
>> > > > > > > > > > > > > > > > > > > > > > > > > error
>> > > > > > > > > > > > > > > > > > > > > > > > > >> code
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to indicate both.
>> > Third,
>> > > > > even
>> > > > > > > if a
>> > > > > > > > > > > replica
>> > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > not
>> > > > > > > > > > > > > > > > > > > > available,
>> > > > > > > > > > > > > > > > > > > > > > > it's
>> > > > > > > > > > > > > > > > > > > > > > > > > >> still
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > useful to return
>> its
>> > > > replica
>> > > > > > id
>> > > > > > > > > since
>> > > > > > > > > > > some
>> > > > > > > > > > > > > > > clients
>> > > > > > > > > > > > > > > > > > (e.g.
>> > > > > > > > > > > > > > > > > > > > > admin
>> > > > > > > > > > > > > > > > > > > > > > > tool)
>> > > > > > > > > > > > > > > > > > > > > > > > > >> may
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > still make use of
>> it.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > One way to address
>> > this
>> > > > > issue
>> > > > > > is
>> > > > > > > > to
>> > > > > > > > > > > always
>> > > > > > > > > > > > > > > return
>> > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > replica
>> > > > > > > > > > > > > > > > > > > > > > > id for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > leader, assigned
>> > > replicas,
>> > > > > and
>> > > > > > > isr
>> > > > > > > > > > > > > regardless
>> > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > whether
>> > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> corresponding
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > broker is live or
>> not.
>> > > > Since
>> > > > > > we
>> > > > > > > > also
>> > > > > > > > > > > > return
>> > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > list
>> > > > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > > > live
>> > > > > > > > > > > > > > > > > > > > > > > > > brokers,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > client can figure
>> out
>> > > > > whether
>> > > > > > a
>> > > > > > > > > leader
>> > > > > > > > > > > or
>> > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > replica
>> > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > live
>> > > > > > > > > > > > > > > > > > > > > or
>> > > > > > > > > > > > > > > > > > > > > > > not
>> > > > > > > > > > > > > > > > > > > > > > > > > >> and act
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > accordingly. This
>> way,
>> > > we
>> > > > > > don't
>> > > > > > > > need
>> > > > > > > > > > to
>> > > > > > > > > > > > set
>> > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > partition
>> > > > > > > > > > > > > > > > > > > > > > level
>> > > > > > > > > > > > > > > > > > > > > > > > > error
>> > > > > > > > > > > > > > > > > > > > > > > > > >> code
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > when the leader or
>> a
>> > > > replica
>> > > > > > is
>> > > > > > > > not
>> > > > > > > > > > > > > available.
>> > > > > > > > > > > > > > > > This
>> > > > > > > > > > > > > > > > > > > > doesn't
>> > > > > > > > > > > > > > > > > > > > > > > change
>> > > > > > > > > > > > > > > > > > > > > > > > > >> the wire
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > protocol, but does
>> > > change
>> > > > > the
>> > > > > > > > > > semantics.
>> > > > > > > > > > > > > Since
>> > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > > > > evolving
>> > > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> protocol
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > of
>> > TopicMetadataRequest
>> > > > > here,
>> > > > > > we
>> > > > > > > > can
>> > > > > > > > > > > > > > potentially
>> > > > > > > > > > > > > > > > > > piggyback
>> > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > change.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 102.1 For those
>> types
>> > of
>> > > > > > errors
>> > > > > > > > due
>> > > > > > > > > to
>> > > > > > > > > > > > > invalid
>> > > > > > > > > > > > > > > > > input,
>> > > > > > > > > > > > > > > > > > > > > > shouldn't
>> > > > > > > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> just
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > guard it at
>> parameter
>> > > > > > validation
>> > > > > > > > > time
>> > > > > > > > > > > and
>> > > > > > > > > > > > > > throw
>> > > > > > > > > > > > > > > > > > > > > > > > > >>
>> InvalidArgumentException
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > without even
>> sending
>> > the
>> > > > > > request
>> > > > > > > > to
>> > > > > > > > > > the
>> > > > > > > > > > > > > > broker?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Jun
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16,
>> 2015
>> > at
>> > > > > 10:37
>> > > > > > > AM,
>> > > > > > > > > > Andrii
>> > > > > > > > > > > > > > > > Biletskyi <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > andrii.biletskyi@stealth.ly
>> > > > > >
>> > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Jun,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Answering your
>> > > > questions:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 101. If I
>> understand
>> > > you
>> > > > > > > > > correctly,
>> > > > > > > > > > > you
>> > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > saying
>> > > > > > > > > > > > > > > > > > > > future
>> > > > > > > > > > > > > > > > > > > > > > > producer
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > versions
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > (which
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will be ported to
>> > > > TMR_V1)
>> > > > > > > won't
>> > > > > > > > be
>> > > > > > > > > > > able
>> > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > automatically
>> > > > > > > > > > > > > > > > > > > > > > > create
>> > > > > > > > > > > > > > > > > > > > > > > > > >> topic (if
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > unconditionally
>> > remove
>> > > > > topic
>> > > > > > > > > > creation
>> > > > > > > > > > > > from
>> > > > > > > > > > > > > > > > there).
>> > > > > > > > > > > > > > > > > > But
>> > > > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > need to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > preserve logic.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Ok, about your
>> > > proposal:
>> > > > > I'm
>> > > > > > > > not a
>> > > > > > > > > > big
>> > > > > > > > > > > > fan
>> > > > > > > > > > > > > > > too,
>> > > > > > > > > > > > > > > > > > when it
>> > > > > > > > > > > > > > > > > > > > > > comes
>> > > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > differentiating
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > clients directly
>> in
>> > > > > protocol
>> > > > > > > > > schema.
>> > > > > > > > > > > And
>> > > > > > > > > > > > > > also
>> > > > > > > > > > > > > > > > I'm
>> > > > > > > > > > > > > > > > > > not
>> > > > > > > > > > > > > > > > > > > > > sure I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> understand
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > at
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > all why
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
>> > > > auto.create.topics.enable
>> > > > > > is a
>> > > > > > > > > > server
>> > > > > > > > > > > > side
>> > > > > > > > > > > > > > > > > > > > configuration.
>> > > > > > > > > > > > > > > > > > > > > > Can
>> > > > > > > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > deprecate
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > this setting
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in future
>> versions,
>> > > add
>> > > > > this
>> > > > > > > > > setting
>> > > > > > > > > > > to
>> > > > > > > > > > > > > > > producer
>> > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > based
>> > > > > > > > > > > > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> upon
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > receiving
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > UnknownTopic
>> create
>> > > > topic
>> > > > > > > > > explicitly
>> > > > > > > > > > > by
>> > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > separate
>> > > > > > > > > > > > > > > > > > > > > producer
>> > > > > > > > > > > > > > > > > > > > > > > call
>> > > > > > > > > > > > > > > > > > > > > > > > > via
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > adminClient?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes.
>> It's
>> > > > > because
>> > > > > > > we
>> > > > > > > > > want
>> > > > > > > > > > > to
>> > > > > > > > > > > > > > > support
>> > > > > > > > > > > > > > > > > > batching
>> > > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > at
>> > > > > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> same
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > time we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > want to give
>> > > descriptive
>> > > > > > error
>> > > > > > > > > > > messages
>> > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > clients.
>> > > > > > > > > > > > > > > > > > > > Since
>> > > > > > > > > > > > > > > > > > > > > > > > > >> AdminClient
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > holds the context
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to construct such
>> > > > messages
>> > > > > > > (e.g.
>> > > > > > > > > > > > > AdminClient
>> > > > > > > > > > > > > > > > layer
>> > > > > > > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > > > > > > know
>> > > > > > > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
>> InvalidArgumentsCode
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > means two cases:
>> > > either
>> > > > > > > invalid
>> > > > > > > > > > > number -
>> > > > > > > > > > > > > > e.g.
>> > > > > > > > > > > > > > > > -1;
>> > > > > > > > > > > > > > > > > or
>> > > > > > > > > > > > > > > > > > > > > > > > > >> replication-factor
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > was
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > provided while
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > partitions
>> argument
>> > > > > wasn't)
>> > > > > > -
>> > > > > > > I
>> > > > > > > > > > > wrapped
>> > > > > > > > > > > > > > > > responses
>> > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > Exceptions.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> But I'm
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > open to any
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > other ideas, this
>> > was
>> > > > just
>> > > > > > > > initial
>> > > > > > > > > > > > > version.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I
>> agree.
>> > > > I'll
>> > > > > > > change
>> > > > > > > > > > that
>> > > > > > > > > > > to
>> > > > > > > > > > > > > > > > probably
>> > > > > > > > > > > > > > > > > > some
>> > > > > > > > > > > > > > > > > > > > > other
>> > > > > > > > > > > > > > > > > > > > > > > dto.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13,
>> 2015
>> > > at
>> > > > > 7:16
>> > > > > > > PM,
>> > > > > > > > > Jun
>> > > > > > > > > > > > Rao <
>> > > > > > > > > > > > > > > > > > > > > jun@confluent.io>
>> > > > > > > > > > > > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Andrii,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 101. That's
>> what I
>> > > was
>> > > > > > > > thinking
>> > > > > > > > > > too,
>> > > > > > > > > > > > but
>> > > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > > may
>> > > > > > > > > > > > > > > > > > not be
>> > > > > > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> simple. In
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > TopicMetadataRequest_V1,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we can let it
>> not
>> > > > > trigger
>> > > > > > > auto
>> > > > > > > > > > topic
>> > > > > > > > > > > > > > > creation.
>> > > > > > > > > > > > > > > > > > Then,
>> > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> producer
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > side,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > if it gets an
>> > > > > > > > > > UnknownTopicException,
>> > > > > > > > > > > > it
>> > > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > > > explicitly
>> > > > > > > > > > > > > > > > > > > > > > > issue a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> createTopicRequest
>> > > for
>> > > > > > auto
>> > > > > > > > > topic
>> > > > > > > > > > > > > > creation.
>> > > > > > > > > > > > > > > On
>> > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > consumer
>> > > > > > > > > > > > > > > > > > > > > > > > > side,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > will
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > never issue
>> > > > > > > > createTopicRequest.
>> > > > > > > > > > This
>> > > > > > > > > > > > > works
>> > > > > > > > > > > > > > > > when
>> > > > > > > > > > > > > > > > > > auto
>> > > > > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> creation is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > enabled on the
>> > > broker
>> > > > > > side.
>> > > > > > > > > > > However, I
>> > > > > > > > > > > > > am
>> > > > > > > > > > > > > > > not
>> > > > > > > > > > > > > > > > > > sure how
>> > > > > > > > > > > > > > > > > > > > > > > things
>> > > > > > > > > > > > > > > > > > > > > > > > > >> will work
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > when auto topic
>> > > > creation
>> > > > > > is
>> > > > > > > > > > disabled
>> > > > > > > > > > > > on
>> > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > broker
>> > > > > > > > > > > > > > > > > > > > side.
>> > > > > > > > > > > > > > > > > > > > > > In
>> > > > > > > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> case,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > want to have a
>> way
>> > > to
>> > > > > > > manually
>> > > > > > > > > > > create
>> > > > > > > > > > > > a
>> > > > > > > > > > > > > > > topic,
>> > > > > > > > > > > > > > > > > > > > > potentially
>> > > > > > > > > > > > > > > > > > > > > > > > > through
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > commands.
>> However,
>> > > > then
>> > > > > we
>> > > > > > > > need
>> > > > > > > > > a
>> > > > > > > > > > > way
>> > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > distinguish
>> > > > > > > > > > > > > > > > > > > > > > > > > >> createTopicRequest
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > issued from the
>> > > > producer
>> > > > > > > > clients
>> > > > > > > > > > and
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > > admin
>> > > > > > > > > > > > > > > > > > tools.
>> > > > > > > > > > > > > > > > > > > > > May
>> > > > > > > > > > > > > > > > > > > > > > > be we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> can
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > new field in
>> > > > > > > > createTopicRequest
>> > > > > > > > > > and
>> > > > > > > > > > > > set
>> > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > > > > differently
>> > > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> producer
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > client and the
>> > admin
>> > > > > > client.
>> > > > > > > > > > > However,
>> > > > > > > > > > > > I
>> > > > > > > > > > > > > am
>> > > > > > > > > > > > > > > not
>> > > > > > > > > > > > > > > > > > sure if
>> > > > > > > > > > > > > > > > > > > > > > > that's
>> > > > > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> best
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > approach.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 2. Yes,
>> > refactoring
>> > > > > > existing
>> > > > > > > > > > > requests
>> > > > > > > > > > > > > is a
>> > > > > > > > > > > > > > > > > > non-trivial
>> > > > > > > > > > > > > > > > > > > > > > > amount of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> work.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > posted some
>> > comments
>> > > > in
>> > > > > > > > > > KAFKA-1927.
>> > > > > > > > > > > We
>> > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > > probably
>> > > > > > > > > > > > > > > > > > > > > have
>> > > > > > > > > > > > > > > > > > > > > > > to fix
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > first, before
>> > adding
>> > > > the
>> > > > > > new
>> > > > > > > > > logic
>> > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > KAFKA-1694.
>> > > > > > > > > > > > > > > > > > > > > > > Otherwise, the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > changes
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > will be too
>> big.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102. About the
>> > > > > > AdminClient:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a
>> bit
>> > > > weird
>> > > > > > that
>> > > > > > > > we
>> > > > > > > > > > > return
>> > > > > > > > > > > > > > > > exception
>> > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > api. It
>> > > > > > > > > > > > > > > > > > > > > > > > > >> seems
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we should
>> either
>> > > > return
>> > > > > > > error
>> > > > > > > > > code
>> > > > > > > > > > > or
>> > > > > > > > > > > > > > throw
>> > > > > > > > > > > > > > > an
>> > > > > > > > > > > > > > > > > > > > exception
>> > > > > > > > > > > > > > > > > > > > > > > when
>> > > > > > > > > > > > > > > > > > > > > > > > > >> getting
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > response state.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.2. We
>> probably
>> > > > > > shouldn't
>> > > > > > > > > > > > explicitly
>> > > > > > > > > > > > > > use
>> > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > request
>> > > > > > > > > > > > > > > > > > > > > > > object
>> > > > > > > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > > > >> the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > api.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Not every
>> request
>> > > > > > evolution
>> > > > > > > > > > requires
>> > > > > > > > > > > > an
>> > > > > > > > > > > > > > api
>> > > > > > > > > > > > > > > > > > change.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Jun
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13,
>> > 2015
>> > > > at
>> > > > > > 4:08
>> > > > > > > > AM,
>> > > > > > > > > > > > Andrii
>> > > > > > > > > > > > > > > > > Biletskyi
>> > > > > > > > > > > > > > > > > > <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > > > andrii.biletskyi@stealth.ly
>> > > > > > > >
>> > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Jun,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks for
>> you
>> > > > > comments.
>> > > > > > > > > Answers
>> > > > > > > > > > > > > inline:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 100. There
>> are a
>> > > few
>> > > > > > > fields
>> > > > > > > > > such
>> > > > > > > > > > > as
>> > > > > > > > > > > > > > > > > > > > ReplicaAssignment,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > ReassignPartitionRequest,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
>> > > > > > PartitionsSerialized
>> > > > > > > > > that
>> > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > represented
>> > > > > > > > > > > > > > > > > > as a
>> > > > > > > > > > > > > > > > > > > > > > > string,
>> > > > > > > > > > > > > > > > > > > > > > > > > but
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
>> > > > structures
>> > > > > > in
>> > > > > > > > > json.
>> > > > > > > > > > > > Could
>> > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > flatten
>> > > > > > > > > > > > > > > > > > > > them
>> > > > > > > > > > > > > > > > > > > > > > out
>> > > > > > > > > > > > > > > > > > > > > > > > > >> directly in
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
>> > > > definition
>> > > > > as
>> > > > > > > > > > > > > arrays/records?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, now with
>> > > Admin
>> > > > > > Client
>> > > > > > > > > this
>> > > > > > > > > > > > looks
>> > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > bit
>> > > > > > > > > > > > > > > > > > weird.
>> > > > > > > > > > > > > > > > > > > > My
>> > > > > > > > > > > > > > > > > > > > > > > initial
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > motivation
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > was:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > ReassignPartitionCommand
>> > > > > > > > > accepts
>> > > > > > > > > > > > input
>> > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > json,
>> > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > want
>> > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> remain
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > tools'
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > interfaces
>> > > > unchanged,
>> > > > > > > where
>> > > > > > > > > > > > possible.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > If we port
>> it to
>> > > > > > > > deserialized
>> > > > > > > > > > > > format,
>> > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > CLI
>> > > > > > > > > > > > > > > > > > (/tools
>> > > > > > > > > > > > > > > > > > > > > > > project)
>> > > > > > > > > > > > > > > > > > > > > > > > > >> we will
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > have
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to add some
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json library
>> > since
>> > > > > > /tools
>> > > > > > > is
>> > > > > > > > > > > written
>> > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > java
>> > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > we'll
>> > > > > > > > > > > > > > > > > > > > > > > need to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > deserialize
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json file
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > provided by a
>> > > user.
>> > > > > Can
>> > > > > > we
>> > > > > > > > > > quickly
>> > > > > > > > > > > > > agree
>> > > > > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > > what
>> > > > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > library
>> > > > > > > > > > > > > > > > > > > > > > > > > >> should
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > (Jackson,
>> GSON,
>> > > > > > whatever)?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
>> > > > > > > > TopicMetadataRequest
>> > > > > > > > > > v1
>> > > > > > > > > > > > > still
>> > > > > > > > > > > > > > > > > trigger
>> > > > > > > > > > > > > > > > > > auto
>> > > > > > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> creation?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > This
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a
>> bit
>> > > > weird
>> > > > > > now
>> > > > > > > > that
>> > > > > > > > > > we
>> > > > > > > > > > > > > have a
>> > > > > > > > > > > > > > > > > > separate
>> > > > > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> creation api.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought
>> > > about
>> > > > > how
>> > > > > > > the
>> > > > > > > > > new
>> > > > > > > > > > > > > > > > > > createTopicRequest
>> > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > TopicMetadataRequest
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be
>> > used
>> > > in
>> > > > > the
>> > > > > > > > > > > > > > producer/consumer
>> > > > > > > > > > > > > > > > > > client,
>> > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > addition
>> > > > > > > > > > > > > > > > > > > > > > > > > >> to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For
>> > > > example,
>> > > > > > > > ideally,
>> > > > > > > > > > we
>> > > > > > > > > > > > > don't
>> > > > > > > > > > > > > > > want
>> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > from
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to
>> > > > trigger
>> > > > > > auto
>> > > > > > > > > topic
>> > > > > > > > > > > > > > creation.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > I agree, this
>> > > > strange
>> > > > > > > logic
>> > > > > > > > > > should
>> > > > > > > > > > > > be
>> > > > > > > > > > > > > > > fixed.
>> > > > > > > > > > > > > > > > > > I'm not
>> > > > > > > > > > > > > > > > > > > > > > > confident
>> > > > > > > > > > > > > > > > > > > > > > > > > >> in
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > correct me if
>> > I'm
>> > > > > wrong,
>> > > > > > > but
>> > > > > > > > > it
>> > > > > > > > > > > > > doesn't
>> > > > > > > > > > > > > > > look
>> > > > > > > > > > > > > > > > > > like a
>> > > > > > > > > > > > > > > > > > > > > hard
>> > > > > > > > > > > > > > > > > > > > > > > thing
>> > > > > > > > > > > > > > > > > > > > > > > > > >> to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > do, I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > think we can
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > leverage
>> > > AdminClient
>> > > > > for
>> > > > > > > > that
>> > > > > > > > > in
>> > > > > > > > > > > > > > Producer
>> > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > unconditionally
>> > > > > > > > > > > > > > > > > > > > > > > > > >> remove
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > creation from
>> > the
>> > > > > > > > > > > > > > TopicMetadataRequest_V1.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 2. I think
>> Jay
>> > > meant
>> > > > > > > getting
>> > > > > > > > > rid
>> > > > > > > > > > > of
>> > > > > > > > > > > > > > scala
>> > > > > > > > > > > > > > > > > > classes
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
>> > > > > > > > > HeartbeatRequestAndHeader
>> > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >>
>> > HeartbeatResponseAndHeader.
>> > > We
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > did
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a
>> stop-gap
>> > > > thing
>> > > > > > when
>> > > > > > > > > > adding
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > new
>> > > > > > > > > > > > > > > > > > requests
>> > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> consumers.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However,
>> the
>> > > long
>> > > > > term
>> > > > > > > > plan
>> > > > > > > > > is
>> > > > > > > > > > > to
>> > > > > > > > > > > > > get
>> > > > > > > > > > > > > > > rid
>> > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > all
>> > > > > > > > > > > > > > > > > > > > > those
>> > > > > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> just
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
>> > > > > request/response
>> > > > > > in
>> > > > > > > > the
>> > > > > > > > > > > > client.
>> > > > > > > > > > > > > > > Since
>> > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > KIP
>> > > > > > > > > > > > > > > > > > > > > > > proposes
>> > > > > > > > > > > > > > > > > > > > > > > > > >> to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant
>> > > number
>> > > > > of
>> > > > > > > new
>> > > > > > > > > > > > requests,
>> > > > > > > > > > > > > > > > perhaps
>> > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > should
>> > > > > > > > > > > > > > > > > > > > > > > bite
>> > > > > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up
>> the
>> > > > > existing
>> > > > > > > > scala
>> > > > > > > > > > > > requests
>> > > > > > > > > > > > > > > first
>> > > > > > > > > > > > > > > > > > before
>> > > > > > > > > > > > > > > > > > > > > > > adding new
>> > > > > > > > > > > > > > > > > > > > > > > > > >> ones?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks
>> like
>> > I
>> > > > > > > > > misunderstood
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > point
>> > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Okay, I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > will
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > rework that.
>> The
>> > > > only
>> > > > > > > thing
>> > > > > > > > is
>> > > > > > > > > > > that
>> > > > > > > > > > > > I
>> > > > > > > > > > > > > > > don't
>> > > > > > > > > > > > > > > > > see
>> > > > > > > > > > > > > > > > > > any
>> > > > > > > > > > > > > > > > > > > > > > > example
>> > > > > > > > > > > > > > > > > > > > > > > > > how
>> > > > > > > > > > > > > > > > > > > > > > > > > >> it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > was
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > done
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > for at
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > least one
>> > existing
>> > > > > > > protocol
>> > > > > > > > > > > message.
>> > > > > > > > > > > > > > Thus,
>> > > > > > > > > > > > > > > > as
>> > > > > > > > > > > > > > > > > I
>> > > > > > > > > > > > > > > > > > > > > > > understand, I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> have to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > think
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > how we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are going to
>> do
>> > > it.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Re porting
>> all
>> > > > > existing
>> > > > > > > > RQ/RP
>> > > > > > > > > in
>> > > > > > > > > > > > this
>> > > > > > > > > > > > > > > patch.
>> > > > > > > > > > > > > > > > > > Sounds
>> > > > > > > > > > > > > > > > > > > > > > > > > reasonable,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> but
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > if
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > it's
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > an
>> *obligatory*
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > requirement
>> to
>> > > have
>> > > > > > Admin
>> > > > > > > > KIP
>> > > > > > > > > > > done,
>> > > > > > > > > > > > > I'm
>> > > > > > > > > > > > > > > > afraid
>> > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > > > > > > > > be a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> serious
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > blocker for
>> us.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > There are 13
>> > > > protocol
>> > > > > > > > messages
>> > > > > > > > > > and
>> > > > > > > > > > > > all
>> > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > would
>> > > > > > > > > > > > > > > > > > > > > > require
>> > > > > > > > > > > > > > > > > > > > > > > not
>> > > > > > > > > > > > > > > > > > > > > > > > > >> only
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > unit
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > tests but
>> quite
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > intensive
>> manual
>> > > > > > testing,
>> > > > > > > > no?
>> > > > > > > > > > I'm
>> > > > > > > > > > > > > afraid
>> > > > > > > > > > > > > > > I'm
>> > > > > > > > > > > > > > > > > > not the
>> > > > > > > > > > > > > > > > > > > > > > > right guy
>> > > > > > > > > > > > > > > > > > > > > > > > > >> to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > cover
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > pretty much
>> all
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka core
>> > > internals
>> > > > > :).
>> > > > > > > Let
>> > > > > > > > > me
>> > > > > > > > > > > know
>> > > > > > > > > > > > > > your
>> > > > > > > > > > > > > > > > > > thoughts
>> > > > > > > > > > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> item. Btw
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > there
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is a ticket
>> to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > follow-up
>> this
>> > > > issue (
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > >
>> https://issues.apache.org/jira/browse/KAFKA-2006
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > ).
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Andrii
>> Biletskyi
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar
>> 13,
>> > > 2015
>> > > > > at
>> > > > > > > 6:40
>> > > > > > > > > AM,
>> > > > > > > > > > > Jun
>> > > > > > > > > > > > > > Rao <
>> > > > > > > > > > > > > > > > > > > > > > > jun@confluent.io>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > A few more
>> > > > comments.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 100. There
>> > are a
>> > > > few
>> > > > > > > > fields
>> > > > > > > > > > such
>> > > > > > > > > > > > as
>> > > > > > > > > > > > > > > > > > > > > ReplicaAssignment,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > ReassignPartitionRequest,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
>> > > > > > PartitionsSerialized
>> > > > > > > > > that
>> > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > represented
>> > > > > > > > > > > > > > > > > > as a
>> > > > > > > > > > > > > > > > > > > > > > > string,
>> > > > > > > > > > > > > > > > > > > > > > > > > but
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
>> > > > structures
>> > > > > > in
>> > > > > > > > > json.
>> > > > > > > > > > > > Could
>> > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > flatten
>> > > > > > > > > > > > > > > > > > > > them
>> > > > > > > > > > > > > > > > > > > > > > out
>> > > > > > > > > > > > > > > > > > > > > > > > > >> directly in
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
>> > > > definition
>> > > > > as
>> > > > > > > > > > > > > arrays/records?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
>> > > > > > > > > TopicMetadataRequest
>> > > > > > > > > > > v1
>> > > > > > > > > > > > > > still
>> > > > > > > > > > > > > > > > > > trigger
>> > > > > > > > > > > > > > > > > > > > auto
>> > > > > > > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > creation?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > This
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a
>> bit
>> > > > weird
>> > > > > > now
>> > > > > > > > that
>> > > > > > > > > > we
>> > > > > > > > > > > > > have a
>> > > > > > > > > > > > > > > > > > separate
>> > > > > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> creation api.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought
>> > > about
>> > > > > how
>> > > > > > > the
>> > > > > > > > > new
>> > > > > > > > > > > > > > > > > > createTopicRequest
>> > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > TopicMetadataRequest
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be
>> > used
>> > > in
>> > > > > the
>> > > > > > > > > > > > > > producer/consumer
>> > > > > > > > > > > > > > > > > > client,
>> > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > addition
>> > > > > > > > > > > > > > > > > > > > > > > > > >> to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For
>> > > > example,
>> > > > > > > > ideally,
>> > > > > > > > > > we
>> > > > > > > > > > > > > don't
>> > > > > > > > > > > > > > > want
>> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > from
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to
>> > > > trigger
>> > > > > > auto
>> > > > > > > > > topic
>> > > > > > > > > > > > > > creation.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 2. I think
>> Jay
>> > > > meant
>> > > > > > > > getting
>> > > > > > > > > > rid
>> > > > > > > > > > > > of
>> > > > > > > > > > > > > > > scala
>> > > > > > > > > > > > > > > > > > classes
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
>> > > > > > > > > HeartbeatRequestAndHeader
>> > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >>
>> > HeartbeatResponseAndHeader.
>> > > We
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > did
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a
>> stop-gap
>> > > > thing
>> > > > > > when
>> > > > > > > > > > adding
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > new
>> > > > > > > > > > > > > > > > > > requests
>> > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> consumers.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However,
>> the
>> > > long
>> > > > > term
>> > > > > > > > plan
>> > > > > > > > > is
>> > > > > > > > > > > to
>> > > > > > > > > > > > > get
>> > > > > > > > > > > > > > > rid
>> > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > all
>> > > > > > > > > > > > > > > > > > > > > those
>> > > > > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> just
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
>> > > > > request/response
>> > > > > > in
>> > > > > > > > the
>> > > > > > > > > > > > client.
>> > > > > > > > > > > > > > > Since
>> > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > KIP
>> > > > > > > > > > > > > > > > > > > > > > > proposes
>> > > > > > > > > > > > > > > > > > > > > > > > > >> to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant
>> > > number
>> > > > > of
>> > > > > > > new
>> > > > > > > > > > > > requests,
>> > > > > > > > > > > > > > > > perhaps
>> > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > should
>> > > > > > > > > > > > > > > > > > > > > > > bite
>> > > > > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up
>> the
>> > > > > existing
>> > > > > > > > scala
>> > > > > > > > > > > > requests
>> > > > > > > > > > > > > > > first
>> > > > > > > > > > > > > > > > > > before
>> > > > > > > > > > > > > > > > > > > > > > > adding new
>> > > > > > > > > > > > > > > > > > > > > > > > > >> ones?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Jun
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar
>> > 12,
>> > > > 2015
>> > > > > > at
>> > > > > > > > 3:37
>> > > > > > > > > > PM,
>> > > > > > > > > > > > > > Andrii
>> > > > > > > > > > > > > > > > > > Biletskyi
>> > > > > > > > > > > > > > > > > > > > <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > andrii.biletskyi@stealth.ly
>> > > > > > > > > >
>> > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > As said
>> > above
>> > > -
>> > > > I
>> > > > > > list
>> > > > > > > > > again
>> > > > > > > > > > > all
>> > > > > > > > > > > > > > > > comments
>> > > > > > > > > > > > > > > > > > from
>> > > > > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > thread
>> > > > > > > > > > > > > > > > > > > > > > > > > >> so we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can see
>> > what's
>> > > > > left
>> > > > > > > and
>> > > > > > > > > > > finalize
>> > > > > > > > > > > > > all
>> > > > > > > > > > > > > > > > > pending
>> > > > > > > > > > > > > > > > > > > > > issues.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments
>> > from
>> > > > Jay:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This
>> is
>> > > much
>> > > > > > needed
>> > > > > > > > > > > > > > functionality,
>> > > > > > > > > > > > > > > > but
>> > > > > > > > > > > > > > > > > > there
>> > > > > > > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > > > > > > > > lot
>> > > > > > > > > > > > > > > > > > > > > > > > > >> of the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > so
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > let's
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > really
>> think
>> > > > these
>> > > > > > > > > protocols
>> > > > > > > > > > > > > > through.
>> > > > > > > > > > > > > > > We
>> > > > > > > > > > > > > > > > > > really
>> > > > > > > > > > > > > > > > > > > > > want
>> > > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > end
>> > > > > > > > > > > > > > > > > > > > > > > > > >> up
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > with a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > set
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > of well
>> > > > > thought-out,
>> > > > > > > > > > > orthoganol
>> > > > > > > > > > > > > > apis.
>> > > > > > > > > > > > > > > > For
>> > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > reason
>> > > > > > > > > > > > > > > > > > > > > > > I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> think it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > really
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> important to
>> > > > think
>> > > > > > > > through
>> > > > > > > > > > the
>> > > > > > > > > > > > end
>> > > > > > > > > > > > > > > state
>> > > > > > > > > > > > > > > > > > even if
>> > > > > > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> includes
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > APIs
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > won't
>> > > implement
>> > > > in
>> > > > > > the
>> > > > > > > > > first
>> > > > > > > > > > > > > phase.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
>> > Definitely
>> > > > > behind
>> > > > > > > > this.
>> > > > > > > > > > > Would
>> > > > > > > > > > > > > > > > > appreciate
>> > > > > > > > > > > > > > > > > > if
>> > > > > > > > > > > > > > > > > > > > > there
>> > > > > > > > > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > > > > > > > > >> concrete
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > comments
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > how this
>> can
>> > > be
>> > > > > > > > improved.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's
>> > > please
>> > > > > > please
>> > > > > > > > > > please
>> > > > > > > > > > > > wait
>> > > > > > > > > > > > > > > until
>> > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > have
>> > > > > > > > > > > > > > > > > > > > > > > switched
>> > > > > > > > > > > > > > > > > > > > > > > > > >> the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > server
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > over
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to the
>> new
>> > > java
>> > > > > > > protocol
>> > > > > > > > > > > > > > definitions.
>> > > > > > > > > > > > > > > If
>> > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > add
>> > > > > > > > > > > > > > > > > > > > > > upteen
>> > > > > > > > > > > > > > > > > > > > > > > > > more
>> > > > > > > > > > > > > > > > > > > > > > > > > >> ad
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > hoc
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > scala
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > objects
>> that
>> > > is
>> > > > > just
>> > > > > > > > > > > generating
>> > > > > > > > > > > > > more
>> > > > > > > > > > > > > > > > work
>> > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> conversion we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > know
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > have to
>> do.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed
>> in
>> > > the
>> > > > > > latest
>> > > > > > > > > > patch -
>> > > > > > > > > > > > > > removed
>> > > > > > > > > > > > > > > > > scala
>> > > > > > > > > > > > > > > > > > > > > > protocol
>> > > > > > > > > > > > > > > > > > > > > > > > > >> classes.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This
>> > > proposal
>> > > > > > > > > introduces
>> > > > > > > > > > a
>> > > > > > > > > > > > new
>> > > > > > > > > > > > > > type
>> > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > > > optional
>> > > > > > > > > > > > > > > > > > > > > > > > > >> parameter.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > This
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> inconsistent
>> > > > with
>> > > > > > > > > everything
>> > > > > > > > > > > > else
>> > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > protocol
>> > > > > > > > > > > > > > > > > > > > > > > where we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> use -1
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > or
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > some
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > other
>> marker
>> > > > > value.
>> > > > > > > You
>> > > > > > > > > > could
>> > > > > > > > > > > > > argue
>> > > > > > > > > > > > > > > > either
>> > > > > > > > > > > > > > > > > > way
>> > > > > > > > > > > > > > > > > > > > but
>> > > > > > > > > > > > > > > > > > > > > > > let's
>> > > > > > > > > > > > > > > > > > > > > > > > > >> stick
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > with
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > for
>> > > consistency.
>> > > > > For
>> > > > > > > > > clients
>> > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > implemented
>> > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > protocol
>> > > > > > > > > > > > > > > > > > > > > > > > > >> in a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > better
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > way
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than our
>> > scala
>> > > > > code
>> > > > > > > > these
>> > > > > > > > > > > basic
>> > > > > > > > > > > > > > > > primitives
>> > > > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > > > > hard
>> > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> change.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed
>> in
>> > > the
>> > > > > > latest
>> > > > > > > > > > patch -
>> > > > > > > > > > > > > > removed
>> > > > > > > > > > > > > > > > > > MaybeOf
>> > > > > > > > > > > > > > > > > > > > > type
>> > > > > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> changed
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > protocol
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> accordingly.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 4.
>> > > > > ClusterMetadata:
>> > > > > > > This
>> > > > > > > > > > seems
>> > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > duplicate
>> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > which
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers,
>> > > topics,
>> > > > > and
>> > > > > > > > > > > > partitions. I
>> > > > > > > > > > > > > > > think
>> > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > should
>> > > > > > > > > > > > > > > > > > > > > > > rename
>> > > > > > > > > > > > > > > > > > > > > > > > > >> that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > request
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > ClusterMetadataRequest
>> > > > > > > > (or
>> > > > > > > > > > > just
>> > > > > > > > > > > > > > > > > > MetadataRequest)
>> > > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> include the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > id
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> controller.
>> > Or
>> > > > are
>> > > > > > > there
>> > > > > > > > > > other
>> > > > > > > > > > > > > > things
>> > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > could
>> > > > > > > > > > > > > > > > > > > > add
>> > > > > > > > > > > > > > > > > > > > > > > here?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I
>> agree.
>> > > > > Updated
>> > > > > > > the
>> > > > > > > > > KIP.
>> > > > > > > > > > > > Let's
>> > > > > > > > > > > > > > > > extends
>> > > > > > > > > > > > > > > > > > > > > > > TopicMetadata
>> > > > > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > version 2
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > include
>> > > > > controller.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We
>> have a
>> > > > > > tendency
>> > > > > > > to
>> > > > > > > > > try
>> > > > > > > > > > > to
>> > > > > > > > > > > > > > make a
>> > > > > > > > > > > > > > > > lot
>> > > > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > > > > > requests
>> > > > > > > > > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> can
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > only
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > go
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> particular
>> > > > nodes.
>> > > > > > This
>> > > > > > > > > adds
>> > > > > > > > > > a
>> > > > > > > > > > > > lot
>> > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > burden
>> > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > client
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > implementations
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > (it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > sounds
>> easy
>> > > but
>> > > > > each
>> > > > > > > > > > discovery
>> > > > > > > > > > > > can
>> > > > > > > > > > > > > > > fail
>> > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > many
>> > > > > > > > > > > > > > > > > > > > > > parts
>> > > > > > > > > > > > > > > > > > > > > > > so
>> > > > > > > > > > > > > > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> ends
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > up
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > being a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > full
>> state
>> > > > machine
>> > > > > > to
>> > > > > > > do
>> > > > > > > > > > > > right). I
>> > > > > > > > > > > > > > > think
>> > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > should
>> > > > > > > > > > > > > > > > > > > > > > > > > consider
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > making
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > admin
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > commands
>> and
>> > > > > ideally
>> > > > > > > as
>> > > > > > > > > many
>> > > > > > > > > > > of
>> > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > other
>> > > > > > > > > > > > > > > > > > apis
>> > > > > > > > > > > > > > > > > > > > as
>> > > > > > > > > > > > > > > > > > > > > > > possible
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > available
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > on
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > all
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers
>> and
>> > > just
>> > > > > > > > redirect
>> > > > > > > > > to
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > controller
>> > > > > > > > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > broker
>> > > > > > > > > > > > > > > > > > > > > > > > > >> side.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Perhaps
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > there
>> would
>> > > be a
>> > > > > > > general
>> > > > > > > > > way
>> > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > encapsulate
>> > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > re-routing
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > behavior.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a
>> > very
>> > > > > > > > interesting
>> > > > > > > > > > > idea,
>> > > > > > > > > > > > > but
>> > > > > > > > > > > > > > > > seems
>> > > > > > > > > > > > > > > > > > there
>> > > > > > > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > > > > > > some
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > concerns
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > about
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > feature
>> > (like
>> > > > > > > > performance
>> > > > > > > > > > > > > > > > considerations,
>> > > > > > > > > > > > > > > > > > how
>> > > > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > complicate
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > server
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I believe
>> > this
>> > > > > > > shouldn't
>> > > > > > > > > be
>> > > > > > > > > > a
>> > > > > > > > > > > > > > blocker.
>> > > > > > > > > > > > > > > > If
>> > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > feature is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > implemented
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > at
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > some
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point it
>> > won't
>> > > > > > affect
>> > > > > > > > > Admin
>> > > > > > > > > > > > > changes
>> > > > > > > > > > > > > > -
>> > > > > > > > > > > > > > > at
>> > > > > > > > > > > > > > > > > > least
>> > > > > > > > > > > > > > > > > > > > no
>> > > > > > > > > > > > > > > > > > > > > > > changes
>> > > > > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > public
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > API
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
>> > > > required.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We
>> should
>> > > > > > probably
>> > > > > > > > > > > normalize
>> > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > key
>> > > > > > > > > > > > > > > > > > value
>> > > > > > > > > > > > > > > > > > > > > pairs
>> > > > > > > > > > > > > > > > > > > > > > > used
>> > > > > > > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > configs
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
>> > > embedding a
>> > > > > new
>> > > > > > > > > > > formatting.
>> > > > > > > > > > > > > So
>> > > > > > > > > > > > > > > two
>> > > > > > > > > > > > > > > > > > strings
>> > > > > > > > > > > > > > > > > > > > > > rather
>> > > > > > > > > > > > > > > > > > > > > > > > > than
>> > > > > > > > > > > > > > > > > > > > > > > > > >> one
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > with
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > an
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > internal
>> > > equals
>> > > > > > sign.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed
>> in
>> > > the
>> > > > > > latest
>> > > > > > > > > > patch -
>> > > > > > > > > > > > > > > > normalized
>> > > > > > > > > > > > > > > > > > > > configs
>> > > > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> changed
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > protocol
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> accordingly.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the
>> > > > > > > postcondition
>> > > > > > > > of
>> > > > > > > > > > > these
>> > > > > > > > > > > > > > APIs
>> > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > command has
>> > > > > > > > > > > > > > > > > > > > > > > > > >> begun
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > or
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
>> command
>> > > has
>> > > > > been
>> > > > > > > > > > > completed?
>> > > > > > > > > > > > It
>> > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > > lot
>> > > > > > > > > > > > > > > > > > more
>> > > > > > > > > > > > > > > > > > > > > > > usable if
>> > > > > > > > > > > > > > > > > > > > > > > > > >> the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > been
>> > completed
>> > > > so
>> > > > > > you
>> > > > > > > > know
>> > > > > > > > > > > that
>> > > > > > > > > > > > if
>> > > > > > > > > > > > > > you
>> > > > > > > > > > > > > > > > > > create a
>> > > > > > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> then
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > publish
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it you
>> won't
>> > > get
>> > > > > an
>> > > > > > > > > > exception
>> > > > > > > > > > > > > about
>> > > > > > > > > > > > > > > > there
>> > > > > > > > > > > > > > > > > > being
>> > > > > > > > > > > > > > > > > > > > no
>> > > > > > > > > > > > > > > > > > > > > > > such
>> > > > > > > > > > > > > > > > > > > > > > > > > >> topic.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: For
>> long
>> > > > > running
>> > > > > > > > > requests
>> > > > > > > > > > > > (like
>> > > > > > > > > > > > > > > > > reassign
>> > > > > > > > > > > > > > > > > > > > > > > partitions) -
>> > > > > > > > > > > > > > > > > > > > > > > > > >> the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > post
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> condition is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
>> has
>> > > > begun
>> > > > > -
>> > > > > > so
>> > > > > > > > we
>> > > > > > > > > > > don't
>> > > > > > > > > > > > > > block
>> > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > client.
>> > > > > > > > > > > > > > > > > > > > > In
>> > > > > > > > > > > > > > > > > > > > > > > case
>> > > > > > > > > > > > > > > > > > > > > > > > > >> of your
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > example -
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
>> > > commands,
>> > > > > this
>> > > > > > > > will
>> > > > > > > > > be
>> > > > > > > > > > > > > > > refactored
>> > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > commands
>> > > > > > > > > > > > > > > > > > > > > > > > > >> will
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > executed
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> immediately,
>> > > > since
>> > > > > > the
>> > > > > > > > > > > > Controller
>> > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > serve
>> > > > > > > > > > > > > > > > > > > > Admin
>> > > > > > > > > > > > > > > > > > > > > > > > > requests
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> (follow-up
>> > > > ticket
>> > > > > > > > > > KAFKA-1777).
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 8.
>> Describe
>> > > > topic
>> > > > > > and
>> > > > > > > > list
>> > > > > > > > > > > > topics
>> > > > > > > > > > > > > > > > > duplicate
>> > > > > > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > > > > > lot
>> > > > > > > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > > > > > > stuff
>> > > > > > > > > > > > > > > > > > > > > > > > > >> in the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > metadata
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request.
>> Is
>> > > > there
>> > > > > a
>> > > > > > > > reason
>> > > > > > > > > > to
>> > > > > > > > > > > > give
>> > > > > > > > > > > > > > > back
>> > > > > > > > > > > > > > > > > > topics
>> > > > > > > > > > > > > > > > > > > > > > marked
>> > > > > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > deletion? I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > feel
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like if
>> we
>> > > just
>> > > > > make
>> > > > > > > the
>> > > > > > > > > > > > > > > post-condition
>> > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > delete
>> > > > > > > > > > > > > > > > > > > > > > > > > >> command be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic is
>> > > deleted
>> > > > > > that
>> > > > > > > > will
>> > > > > > > > > > get
>> > > > > > > > > > > > rid
>> > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > need
>> > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> right?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > And
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be much
>> more
>> > > > > > > intuitive.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed
>> in
>> > > the
>> > > > > > latest
>> > > > > > > > > > patch -
>> > > > > > > > > > > > > > removed
>> > > > > > > > > > > > > > > > > > topics
>> > > > > > > > > > > > > > > > > > > > > marked
>> > > > > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> deletion
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > ListTopicsRequest.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 9.
>> Should we
>> > > > > > consider
>> > > > > > > > > > batching
>> > > > > > > > > > > > > these
>> > > > > > > > > > > > > > > > > > requests?
>> > > > > > > > > > > > > > > > > > > > We
>> > > > > > > > > > > > > > > > > > > > > > have
>> > > > > > > > > > > > > > > > > > > > > > > > > >> generally
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > tried
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > allow
>> > multiple
>> > > > > > > > operations
>> > > > > > > > > to
>> > > > > > > > > > > be
>> > > > > > > > > > > > > > > batched.
>> > > > > > > > > > > > > > > > > My
>> > > > > > > > > > > > > > > > > > > > > > suspicion
>> > > > > > > > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > without
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we will
>> get
>> > a
>> > > > lot
>> > > > > of
>> > > > > > > > code
>> > > > > > > > > > that
>> > > > > > > > > > > > > does
>> > > > > > > > > > > > > > > > > > something
>> > > > > > > > > > > > > > > > > > > > like
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > for(topic:
>> > > > > > > > > > > > > > > adminClient.listTopics())
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > >  adminClient.describeTopic(topic)
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this code
>> > will
>> > > > > work
>> > > > > > > > great
>> > > > > > > > > > when
>> > > > > > > > > > > > you
>> > > > > > > > > > > > > > > test
>> > > > > > > > > > > > > > > > > on 5
>> > > > > > > > > > > > > > > > > > > > > topics
>> > > > > > > > > > > > > > > > > > > > > > > but
>> > > > > > > > > > > > > > > > > > > > > > > > > not
>> > > > > > > > > > > > > > > > > > > > > > > > > >> do as
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > well
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > if
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > you have
>> > 50k.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
>> Updated
>> > the
>> > > > > KIP -
>> > > > > > > > > please
>> > > > > > > > > > > > check
>> > > > > > > > > > > > > > > "Topic
>> > > > > > > > > > > > > > > > > > Admin
>> > > > > > > > > > > > > > > > > > > > > > Schema"
>> > > > > > > > > > > > > > > > > > > > > > > > > >> section.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I
>> think
>> > we
>> > > > > > should
>> > > > > > > > also
>> > > > > > > > > > > > discuss
>> > > > > > > > > > > > > > how
>> > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > want
>> > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > expose a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > programmatic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > JVM
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > client
>> api
>> > for
>> > > > > these
>> > > > > > > > > > > operations.
>> > > > > > > > > > > > > > > > Currently
>> > > > > > > > > > > > > > > > > > > > people
>> > > > > > > > > > > > > > > > > > > > > > > rely on
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > AdminUtils
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > which
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > is
>> totally
>> > > > > sketchy.
>> > > > > > I
>> > > > > > > > > think
>> > > > > > > > > > we
>> > > > > > > > > > > > > > > probably
>> > > > > > > > > > > > > > > > > need
>> > > > > > > > > > > > > > > > > > > > > another
>> > > > > > > > > > > > > > > > > > > > > > > > > client
>> > > > > > > > > > > > > > > > > > > > > > > > > >> under
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clients/
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > that
>> exposes
>> > > > > > > > > administrative
>> > > > > > > > > > > > > > > > functionality.
>> > > > > > > > > > > > > > > > > > We
>> > > > > > > > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > > > > > > > need
>> > > > > > > > > > > > > > > > > > > > > > > > > >> this just
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > properly
>> > test
>> > > > the
>> > > > > > new
>> > > > > > > > > apis,
>> > > > > > > > > > I
>> > > > > > > > > > > > > > suspect.
>> > > > > > > > > > > > > > > > We
>> > > > > > > > > > > > > > > > > > should
>> > > > > > > > > > > > > > > > > > > > > > > figure
>> > > > > > > > > > > > > > > > > > > > > > > > > out
>> > > > > > > > > > > > > > > > > > > > > > > > > >> that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > API.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
>> Updated
>> > the
>> > > > > KIP -
>> > > > > > > > > please
>> > > > > > > > > > > > check
>> > > > > > > > > > > > > > > "Admin
>> > > > > > > > > > > > > > > > > > Client"
>> > > > > > > > > > > > > > > > > > > > > > > section
>> > > > > > > > > > > > > > > > > > > > > > > > > >> with an
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > initial
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > API
>> > proposal.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The
>> > other
>> > > > > > > > information
>> > > > > > > > > > that
>> > > > > > > > > > > > > would
>> > > > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > > really
>> > > > > > > > > > > > > > > > > > > > > > useful
>> > > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > get
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > would
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> information
>> > > > about
>> > > > > > > > > > > > partitions--how
>> > > > > > > > > > > > > > much
>> > > > > > > > > > > > > > > > > data
>> > > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> partition,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > what
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
>> segment
>> > > > > offsets,
>> > > > > > > > what
>> > > > > > > > > is
>> > > > > > > > > > > the
>> > > > > > > > > > > > > > > log-end
>> > > > > > > > > > > > > > > > > > offset
>> > > > > > > > > > > > > > > > > > > > > > (i.e.
>> > > > > > > > > > > > > > > > > > > > > > > last
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > offset),
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > what
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
>> > compaction
>> > > > > > point,
>> > > > > > > > > etc. I
>> > > > > > > > > > > > think
>> > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > done
>> > > > > > > > > > > > > > > > > > > > right
>> > > > > > > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> would be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> successor to
>> > > the
>> > > > > > very
>> > > > > > > > > > awkward
>> > > > > > > > > > > > > > > > > OffsetRequest
>> > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > have
>> > > > > > > > > > > > > > > > > > > > > > > today.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I
>> removed
>> > > > > > > > > > > > > > > ConsumerGroupOffsetsRequest
>> > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > latest
>> > > > > > > > > > > > > > > > > > > > > > > > > >> patch. I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > believe
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
>> should
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be
>> resolved
>> > > in a
>> > > > > > > > separate
>> > > > > > > > > > KIP
>> > > > > > > > > > > /
>> > > > > > > > > > > > > jira
>> > > > > > > > > > > > > > > > > ticket.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 12.
>> > Generally
>> > > we
>> > > > > can
>> > > > > > > do
>> > > > > > > > > good
>> > > > > > > > > > > > error
>> > > > > > > > > > > > > > > > > handling
>> > > > > > > > > > > > > > > > > > > > > without
>> > > > > > > > > > > > > > > > > > > > > > > > > needing
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > custom
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> server-side
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > messages.
>> > I.e.
>> > > > > > > generally
>> > > > > > > > > the
>> > > > > > > > > > > > > client
>> > > > > > > > > > > > > > > has
>> > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > context
>> > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > know
>> > > > > > > > > > > > > > > > > > > > > > > > > >> that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > if
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > got
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > an error
>> > that
>> > > > the
>> > > > > > > topic
>> > > > > > > > > > > doesn't
>> > > > > > > > > > > > > > exist
>> > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > say
>> > > > > > > > > > > > > > > > > > > > > "Topic
>> > > > > > > > > > > > > > > > > > > > > > X
>> > > > > > > > > > > > > > > > > > > > > > > > > >> doesn't
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > exist"
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
>> "error
>> > > code
>> > > > > 14"
>> > > > > > > (or
>> > > > > > > > > > > > > whatever).
>> > > > > > > > > > > > > > > > Maybe
>> > > > > > > > > > > > > > > > > > there
>> > > > > > > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > > > > > > > > specific
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > cases
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > where
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this is
>> > hard?
>> > > If
>> > > > > we
>> > > > > > > want
>> > > > > > > > > to
>> > > > > > > > > > > add
>> > > > > > > > > > > > > > > > > server-side
>> > > > > > > > > > > > > > > > > > > > error
>> > > > > > > > > > > > > > > > > > > > > > > messages
>> > > > > > > > > > > > > > > > > > > > > > > > > >> we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > really
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > do
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > need to
>> do
>> > > this
>> > > > > in a
>> > > > > > > > > > > consistent
>> > > > > > > > > > > > > way
>> > > > > > > > > > > > > > > > across
>> > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > protocol.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
>> Updated
>> > the
>> > > > > KIP -
>> > > > > > > > > please
>> > > > > > > > > > > > check
>> > > > > > > > > > > > > > > > > "Protocol
>> > > > > > > > > > > > > > > > > > > > > Errors"
>> > > > > > > > > > > > > > > > > > > > > > > > > >> section. I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > added
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > comprehensive,
>> > > > > > > > > fine-grained
>> > > > > > > > > > > list
>> > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > error
>> > > > > > > > > > > > > > > > > > codes.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments
>> > from
>> > > > > > > Guozhang:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 13.
>> Describe
>> > > > topic
>> > > > > > > > > request:
>> > > > > > > > > > it
>> > > > > > > > > > > > > would
>> > > > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > > great to
>> > > > > > > > > > > > > > > > > > > > > go
>> > > > > > > > > > > > > > > > > > > > > > > beyond
>> > > > > > > > > > > > > > > > > > > > > > > > > >> just
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > batching
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
>> name
>> > > regex
>> > > > > for
>> > > > > > > > this
>> > > > > > > > > > > > request.
>> > > > > > > > > > > > > > For
>> > > > > > > > > > > > > > > > > > example,
>> > > > > > > > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > > > > > > > very
>> > > > > > > > > > > > > > > > > > > > > > > > > >> common use
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > case
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the topic
>> > > > command
>> > > > > is
>> > > > > > > to
>> > > > > > > > > list
>> > > > > > > > > > > all
>> > > > > > > > > > > > > > > topics
>> > > > > > > > > > > > > > > > > > whose
>> > > > > > > > > > > > > > > > > > > > > config
>> > > > > > > > > > > > > > > > > > > > > > > A's
>> > > > > > > > > > > > > > > > > > > > > > > > > >> value is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > B.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > With
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
>> name
>> > > regex
>> > > > > > then
>> > > > > > > we
>> > > > > > > > > > have
>> > > > > > > > > > > to
>> > > > > > > > > > > > > > first
>> > > > > > > > > > > > > > > > > > retrieve
>> > > > > > > > > > > > > > > > > > > > > > > __all__
>> > > > > > > > > > > > > > > > > > > > > > > > > >> topics's
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> description
>> > > info
>> > > > > and
>> > > > > > > > then
>> > > > > > > > > > > filter
>> > > > > > > > > > > > > at
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > client
>> > > > > > > > > > > > > > > > > > > > > end,
>> > > > > > > > > > > > > > > > > > > > > > > which
>> > > > > > > > > > > > > > > > > > > > > > > > > >> will
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > huge
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > burden on
>> > ZK.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > AND
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 14.
>> Config
>> > > K-Vs
>> > > > in
>> > > > > > > > create
>> > > > > > > > > > > topic:
>> > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > related
>> > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> previous
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > point;
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > maybe we
>> can
>> > > add
>> > > > > > > another
>> > > > > > > > > > > > metadata
>> > > > > > > > > > > > > > K-V
>> > > > > > > > > > > > > > > or
>> > > > > > > > > > > > > > > > > > just a
>> > > > > > > > > > > > > > > > > > > > > > > metadata
>> > > > > > > > > > > > > > > > > > > > > > > > > >> string
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > along
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > side
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > with
>> config
>> > > K-V
>> > > > in
>> > > > > > > > create
>> > > > > > > > > > > topic
>> > > > > > > > > > > > > like
>> > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > did
>> > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > offset
>> > > > > > > > > > > > > > > > > > > > > > > > > >> commit
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > request.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > This
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > field
>> can be
>> > > > quite
>> > > > > > > > useful
>> > > > > > > > > in
>> > > > > > > > > > > > > storing
>> > > > > > > > > > > > > > > > > > information
>> > > > > > > > > > > > > > > > > > > > > > like
>> > > > > > > > > > > > > > > > > > > > > > > > > >> "owner" of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > who issue
>> > the
>> > > > > create
>> > > > > > > > > > command,
>> > > > > > > > > > > > etc,
>> > > > > > > > > > > > > > > which
>> > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > quite
>> > > > > > > > > > > > > > > > > > > > > > > > > important
>> > > > > > > > > > > > > > > > > > > > > > > > > >> for a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> multi-tenant
>> > > > > > setting.
>> > > > > > > > Then
>> > > > > > > > > > in
>> > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > describe
>> > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > request
>> > > > > > > > > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> can
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > also
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > batch
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > on regex
>> of
>> > > the
>> > > > > > > metadata
>> > > > > > > > > > > field.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: As
>> > > discussed
>> > > > it
>> > > > > > is
>> > > > > > > > very
>> > > > > > > > > > > > > > interesting
>> > > > > > > > > > > > > > > > but
>> > > > > > > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> implemented
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > later
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > after
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we have
>> some
>> > > > basic
>> > > > > > > > > > > functionality
>> > > > > > > > > > > > > > > there.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today
>> > all
>> > > > the
>> > > > > > > admin
>> > > > > > > > > > > > operations
>> > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > async in
>> > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > sense
>> > > > > > > > > > > > > > > > > > > > > > > > > >> that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > return
>> once
>> > it
>> > > > is
>> > > > > > > > written
>> > > > > > > > > in
>> > > > > > > > > > > ZK,
>> > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > is why
>> > > > > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > need
>> > > > > > > > > > > > > > > > > > > > > > > > > >> extra
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> verification
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
>> > > > > > > > > > > > > testUtil.waitForTopicCreated()
>> > > > > > > > > > > > > > /
>> > > > > > > > > > > > > > > > > verify
>> > > > > > > > > > > > > > > > > > > > > > partition
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reassignment
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request,
>> > etc.
>> > > > With
>> > > > > > > admin
>> > > > > > > > > > > > requests
>> > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > could
>> > > > > > > > > > > > > > > > > > add a
>> > > > > > > > > > > > > > > > > > > > > > flag
>> > > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> enable /
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > disable
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> synchronous
>> > > > > > requests;
>> > > > > > > > when
>> > > > > > > > > > it
>> > > > > > > > > > > is
>> > > > > > > > > > > > > > > turned
>> > > > > > > > > > > > > > > > > on,
>> > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > response
>> > > > > > > > > > > > > > > > > > > > > > > > > >> will not
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > return
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > until the
>> > > > request
>> > > > > > has
>> > > > > > > > been
>> > > > > > > > > > > > > > completed.
>> > > > > > > > > > > > > > > > And
>> > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > async
>> > > > > > > > > > > > > > > > > > > > > > > > > >> requests we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > can
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > add a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > "token"
>> > field
>> > > in
>> > > > > the
>> > > > > > > > > > response,
>> > > > > > > > > > > > and
>> > > > > > > > > > > > > > > then
>> > > > > > > > > > > > > > > > > only
>> > > > > > > > > > > > > > > > > > > > need
>> > > > > > > > > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > > > > > > > > > > general
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > "admin
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> verification
>> > > > > > request"
>> > > > > > > > with
>> > > > > > > > > > the
>> > > > > > > > > > > > > given
>> > > > > > > > > > > > > > > > token
>> > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > check
>> > > > > > > > > > > > > > > > > > > > > > > if the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> async
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > request
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > has been
>> > > > > completed.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I see
>> > your
>> > > > > point.
>> > > > > > > My
>> > > > > > > > > idea
>> > > > > > > > > > > was
>> > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > provide
>> > > > > > > > > > > > > > > > > > > > > specific
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Verify...Request
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > per
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > each
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long
>> running
>> > > > > > request,
>> > > > > > > > > where
>> > > > > > > > > > > > > needed.
>> > > > > > > > > > > > > > We
>> > > > > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > > > do it
>> > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > way
>> > > > > > > > > > > > > > > > > > > > > > > > > you
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > suggest.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > The
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > only
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > concern
>> is
>> > > that
>> > > > > > > > > introducing
>> > > > > > > > > > a
>> > > > > > > > > > > > > token
>> > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > again
>> > > > > > > > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > > > > > > make
>> > > > > > > > > > > > > > > > > > > > > > > > > >> schema
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > We
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to do
>> > similar
>> > > > > thing
>> > > > > > > > > > > introducing
>> > > > > > > > > > > > > > single
>> > > > > > > > > > > > > > > > > > > > > AdminRequest
>> > > > > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > all
>> > > > > > > > > > > > > > > > > > > > > > > > > >> topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > commands
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > but
>> rejected
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this idea
>> > > > because
>> > > > > we
>> > > > > > > > > wanted
>> > > > > > > > > > to
>> > > > > > > > > > > > > have
>> > > > > > > > > > > > > > > > schema
>> > > > > > > > > > > > > > > > > > > > > defined.
>> > > > > > > > > > > > > > > > > > > > > > So
>> > > > > > > > > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > more a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > choice
>> > > between:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > a) have
>> > fixed
>> > > > > schema
>> > > > > > > but
>> > > > > > > > > > > > introduce
>> > > > > > > > > > > > > > > each
>> > > > > > > > > > > > > > > > > > time new
>> > > > > > > > > > > > > > > > > > > > > > > > > >> Verify...Request
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> long-running
>> > > > > > requests
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > b) use
>> one
>> > > > request
>> > > > > > for
>> > > > > > > > > > > > > verification
>> > > > > > > > > > > > > > > but
>> > > > > > > > > > > > > > > > > > > > generalize
>> > > > > > > > > > > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > > > > > > > > > with
>> > > > > > > > > > > > > > > > > > > > > > > > > >> token
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine
>> > with
>> > > > > > whatever
>> > > > > > > > > > > decision
>> > > > > > > > > > > > > > > > community
>> > > > > > > > > > > > > > > > > > come
>> > > > > > > > > > > > > > > > > > > > to.
>> > > > > > > > > > > > > > > > > > > > > > > Just
>> > > > > > > > > > > > > > > > > > > > > > > > > let
>> > > > > > > > > > > > > > > > > > > > > > > > > >> me
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > know
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > your
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comment
>> from
>> > > > Gwen:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 16.
>> > > Specifically
>> > > > > for
>> > > > > > > > > > > ownership,
>> > > > > > > > > > > > I
>> > > > > > > > > > > > > > > think
>> > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > plan
>> > > > > > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > > > > to add
>> > > > > > > > > > > > > > > > > > > > > > > > > >> ACL
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sounds
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like you
>> are
>> > > > > > > describing
>> > > > > > > > > ACL)
>> > > > > > > > > > > via
>> > > > > > > > > > > > > an
>> > > > > > > > > > > > > > > > > external
>> > > > > > > > > > > > > > > > > > > > > system
>> > > > > > > > > > > > > > > > > > > > > > > > > (Argus,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Sentry).
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I
>> remember
>> > > > KIP-11
>> > > > > > > > > described
>> > > > > > > > > > > > this,
>> > > > > > > > > > > > > > but
>> > > > > > > > > > > > > > > I
>> > > > > > > > > > > > > > > > > > can't
>> > > > > > > > > > > > > > > > > > > > find
>> > > > > > > > > > > > > > > > > > > > > > > the KIP
>> > > > > > > > > > > > > > > > > > > > > > > > > >> any
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > longer.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay,
>> no
>> > > > > problem.
>> > > > > > > Not
>> > > > > > > > > > sure
>> > > > > > > > > > > > > though
>> > > > > > > > > > > > > > > how
>> > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > > > > > going
>> > > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> handle
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > it.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Wait
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
>> > > > committed
>> > > > > > > first
>> > > > > > > > > and
>> > > > > > > > > > > > > include
>> > > > > > > > > > > > > > > > > changes
>> > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadata from
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > later
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > one?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I
>> > > added
>> > > > > this
>> > > > > > > > note
>> > > > > > > > > to
>> > > > > > > > > > > > "Open
>> > > > > > > > > > > > > > > > > > Questions"
>> > > > > > > > > > > > > > > > > > > > > > section
>> > > > > > > > > > > > > > > > > > > > > > > so
>> > > > > > > > > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> don't
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > miss
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii
>> > > Biletskyi
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri,
>> Mar
>> > > 13,
>> > > > > 2015
>> > > > > > > at
>> > > > > > > > > > 12:34
>> > > > > > > > > > > > AM,
>> > > > > > > > > > > > > > > Andrii
>> > > > > > > > > > > > > > > > > > > > > Biletskyi <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > andrii.biletskyi@stealth.ly
>> > > > > > > > > > >
>> > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Today I
>> > > > uploaded
>> > > > > > the
>> > > > > > > > > patch
>> > > > > > > > > > > > that
>> > > > > > > > > > > > > > > covers
>> > > > > > > > > > > > > > > > > > some of
>> > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> discussed
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > agreed
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > items:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
>> removed
>> > > > > MaybeOf
>> > > > > > > > > optional
>> > > > > > > > > > > > type
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
>> switched
>> > > to
>> > > > > java
>> > > > > > > > > > protocol
>> > > > > > > > > > > > > > > > definitions
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
>> > simplified
>> > > > > > > messages
>> > > > > > > > > > > > > (normalized
>> > > > > > > > > > > > > > > > > configs,
>> > > > > > > > > > > > > > > > > > > > > removed
>> > > > > > > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> marked
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
>> deletion)
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I also
>> > > updated
>> > > > > the
>> > > > > > > > KIP-4
>> > > > > > > > > > > with
>> > > > > > > > > > > > > > > > respective
>> > > > > > > > > > > > > > > > > > > > changes
>> > > > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> wrote down
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > my
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
>> proposal
>> > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > pending
>> > > items:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch
>> > > Admin
>> > > > > > > > Operations
>> > > > > > > > > > ->
>> > > > > > > > > > > > > > updated
>> > > > > > > > > > > > > > > > Wire
>> > > > > > > > > > > > > > > > > > > > > Protocol
>> > > > > > > > > > > > > > > > > > > > > > > schema
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > proposal
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
>> Remove
>> > > > > > > > ClusterMetadata
>> > > > > > > > > > ->
>> > > > > > > > > > > > > > changed
>> > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > extend
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
>> TopicMetadataRequest
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin
>> > > Client
>> > > > > ->
>> > > > > > > > > updated
>> > > > > > > > > > my
>> > > > > > > > > > > > > > initial
>> > > > > > > > > > > > > > > > > > proposal
>> > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > reflect
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > batching
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Error
>> > > codes
>> > > > ->
>> > > > > > > > > proposed
>> > > > > > > > > > > > > > > fine-grained
>> > > > > > > > > > > > > > > > > > error
>> > > > > > > > > > > > > > > > > > > > > code
>> > > > > > > > > > > > > > > > > > > > > > > > > instead
>> > > > > > > > > > > > > > > > > > > > > > > > > >> of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
>> > > > > AdminRequestFailed
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I will
>> > also
>> > > > > send a
>> > > > > > > > > > separate
>> > > > > > > > > > > > > email
>> > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > cover all
>> > > > > > > > > > > > > > > > > > > > > > > comments
>> > > > > > > > > > > > > > > > > > > > > > > > > >> from
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > thread.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii
>> > > > Biletskyi
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu,
>> > Mar
>> > > > 12,
>> > > > > > 2015
>> > > > > > > > at
>> > > > > > > > > > 9:26
>> > > > > > > > > > > > PM,
>> > > > > > > > > > > > > > > Gwen
>> > > > > > > > > > > > > > > > > > Shapira
>> > > > > > > > > > > > > > > > > > > > <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > gshapira@cloudera.com
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Found
>> > > KIP-11
>> > > > (
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >>
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> It
>> > actually
>> > > > > > > specifies
>> > > > > > > > > > > changes
>> > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > Metadata
>> > > > > > > > > > > > > > > > > > > > > > > protocol,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> so
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > making
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sure
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> both
>> KIPs
>> > > are
>> > > > > > > > > consistent
>> > > > > > > > > > in
>> > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > regard
>> > > > > > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > > > > > > > good.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> On
>> Thu,
>> > Mar
>> > > > 12,
>> > > > > > > 2015
>> > > > > > > > at
>> > > > > > > > > > > 12:21
>> > > > > > > > > > > > > PM,
>> > > > > > > > > > > > > > > > Gwen
>> > > > > > > > > > > > > > > > > > > > Shapira
>> > > > > > > > > > > > > > > > > > > > > <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > gshapira@cloudera.com
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
>> > > > Specifically
>> > > > > > for
>> > > > > > > > > > > > ownership, I
>> > > > > > > > > > > > > > > think
>> > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > plan
>> > > > > > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > add
>> > > > > > > > > > > > > > > > > > > > > > > > > >> ACL
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > sounds
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > like
>> > you
>> > > > are
>> > > > > > > > > describing
>> > > > > > > > > > > > ACL)
>> > > > > > > > > > > > > > via
>> > > > > > > > > > > > > > > an
>> > > > > > > > > > > > > > > > > > > > external
>> > > > > > > > > > > > > > > > > > > > > > > system
>> > > > > > > > > > > > > > > > > > > > > > > > > >> (Argus,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I
>> > > remember
>> > > > > > KIP-11
>> > > > > > > > > > > described
>> > > > > > > > > > > > > > this,
>> > > > > > > > > > > > > > > > > but I
>> > > > > > > > > > > > > > > > > > > > can't
>> > > > > > > > > > > > > > > > > > > > > > > find
>> > > > > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> KIP
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > any
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > longer.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
>> > > > Regardless, I
>> > > > > > > think
>> > > > > > > > > > KIP-4
>> > > > > > > > > > > > > > focuses
>> > > > > > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > > > > > getting
>> > > > > > > > > > > > > > > > > > > > > > > > > >> information
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > already
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
>> exists
>> > > from
>> > > > > > Kafka
>> > > > > > > > > > > brokers,
>> > > > > > > > > > > > > not
>> > > > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > > > adding
>> > > > > > > > > > > > > > > > > > > > > > > information
>> > > > > > > > > > > > > > > > > > > > > > > > > >> that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > perhaps
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
>> should
>> > > > exist
>> > > > > > but
>> > > > > > > > > > doesn't
>> > > > > > > > > > > > yet?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On
>> Thu,
>> > > Mar
>> > > > > 12,
>> > > > > > > > 2015
>> > > > > > > > > at
>> > > > > > > > > > > > 6:37
>> > > > > > > > > > > > > > AM,
>> > > > > > > > > > > > > > > > > > Guozhang
>> > > > > > > > > > > > > > > > > > > > > Wang
>> > > > > > > > > > > > > > > > > > > > > > <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > wangguoz@gmail.com>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> Folks,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> Just
>> > > want
>> > > > to
>> > > > > > > > > > elaborate a
>> > > > > > > > > > > > bit
>> > > > > > > > > > > > > > > more
>> > > > > > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > create-topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > metadata
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> batching
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > describe-topic
>> > > > > > > > based
>> > > > > > > > > > on
>> > > > > > > > > > > > > > config /
>> > > > > > > > > > > > > > > > > > metadata
>> > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > my
>> > > > > > > > > > > > > > > > > > > > > > > > > >> previous
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > email
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > as
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on
>> > > > > KAFKA-1694.
>> > > > > > > The
>> > > > > > > > > > main
>> > > > > > > > > > > > > > > motivation
>> > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > have
>> > > > > > > > > > > > > > > > > > > > > > > some
>> > > > > > > > > > > > > > > > > > > > > > > > > >> sort of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > management
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > mechanisms,
>> > > > > > > which
>> > > > > > > > I
>> > > > > > > > > > > think
>> > > > > > > > > > > > is
>> > > > > > > > > > > > > > > quite
>> > > > > > > > > > > > > > > > > > > > important
>> > > > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > multi-tenant
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > /
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > architecture:
>> > > > > > > > today
>> > > > > > > > > > > anyone
>> > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > > create
>> > > > > > > > > > > > > > > > > > > > topics
>> > > > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> shared
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Kafka
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> cluster,
>> > > but
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> there
>> > is
>> > > > no
>> > > > > > > > concept
>> > > > > > > > > or
>> > > > > > > > > > > > > > > "ownership"
>> > > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > > > topics
>> > > > > > > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > created
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > by
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> different
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> users.
>> > > For
>> > > > > > > > example,
>> > > > > > > > > at
>> > > > > > > > > > > > > > LinkedIn
>> > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > basically
>> > > > > > > > > > > > > > > > > > > > > > > > > >> distinguish
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > owners
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> some
>> > > > casual
>> > > > > > > topic
>> > > > > > > > > name
>> > > > > > > > > > > > > prefix,
>> > > > > > > > > > > > > > > > which
>> > > > > > > > > > > > > > > > > > is a
>> > > > > > > > > > > > > > > > > > > > > bit
>> > > > > > > > > > > > > > > > > > > > > > > > > awkward
>> > > > > > > > > > > > > > > > > > > > > > > > > >> and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > does
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > fly
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we
>> > scale
>> > > > our
>> > > > > > > > > > customers.
>> > > > > > > > > > > It
>> > > > > > > > > > > > > > would
>> > > > > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > > great
>> > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > use
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > describe-topics
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > such
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > Describe
>> > > > all
>> > > > > > > > topics
>> > > > > > > > > > that
>> > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > created
>> > > > > > > > > > > > > > > > > > by me.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > Describe
>> > > > all
>> > > > > > > > topics
>> > > > > > > > > > > whose
>> > > > > > > > > > > > > > > > retention
>> > > > > > > > > > > > > > > > > > time
>> > > > > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > > > > > > overriden
>> > > > > > > > > > > > > > > > > > > > > > > > > >> to X.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > Describe
>> > > > all
>> > > > > > > > topics
>> > > > > > > > > > > whose
>> > > > > > > > > > > > > > > writable
>> > > > > > > > > > > > > > > > > > group
>> > > > > > > > > > > > > > > > > > > > > > include
>> > > > > > > > > > > > > > > > > > > > > > > > > user
>> > > > > > > > > > > > > > > > > > > > > > > > > >> Y
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> related
>> > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > > authorization),
>> > > > > > > > > etc..
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One
>> > > > possible
>> > > > > > way
>> > > > > > > > to
>> > > > > > > > > > > > achieve
>> > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > add a
>> > > > > > > > > > > > > > > > > > > > > > > > > metadata
>> > > > > > > > > > > > > > > > > > > > > > > > > >> file
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > in
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > create-topic
>> > > > > > > > > request,
>> > > > > > > > > > > > whose
>> > > > > > > > > > > > > > > value
>> > > > > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > > > > also
>> > > > > > > > > > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> written ZK
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > as
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > create
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> topic;
>> > > > then
>> > > > > > > > > > > > describe-topics
>> > > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > > > choose to
>> > > > > > > > > > > > > > > > > > > > > > batch
>> > > > > > > > > > > > > > > > > > > > > > > > > topics
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > based
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > on
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 1)
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > name
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> regex,
>> > > 2)
>> > > > > > config
>> > > > > > > > K-V
>> > > > > > > > > > > > > matching,
>> > > > > > > > > > > > > > > 3)
>> > > > > > > > > > > > > > > > > > metadata
>> > > > > > > > > > > > > > > > > > > > > > > regex,
>> > > > > > > > > > > > > > > > > > > > > > > > > etc.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > Thoughts?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > Guozhang
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On
>> > Thu,
>> > > > Mar
>> > > > > 5,
>> > > > > > > > 2015
>> > > > > > > > > at
>> > > > > > > > > > > > 4:37
>> > > > > > > > > > > > > > PM,
>> > > > > > > > > > > > > > > > > > Guozhang
>> > > > > > > > > > > > > > > > > > > > > Wang
>> > > > > > > > > > > > > > > > > > > > > > <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > wangguoz@gmail.com>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > Thanks
>> > > > for
>> > > > > > the
>> > > > > > > > > > updated
>> > > > > > > > > > > > > wiki.
>> > > > > > > > > > > > > > A
>> > > > > > > > > > > > > > > > few
>> > > > > > > > > > > > > > > > > > > > comments
>> > > > > > > > > > > > > > > > > > > > > > > below:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1.
>> > > Error
>> > > > > > > > > description
>> > > > > > > > > > in
>> > > > > > > > > > > > > > > > response: I
>> > > > > > > > > > > > > > > > > > think
>> > > > > > > > > > > > > > > > > > > > > if
>> > > > > > > > > > > > > > > > > > > > > > > some
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > errorCode
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> indicate
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > several
>> > > > > > > different
>> > > > > > > > > > error
>> > > > > > > > > > > > > cases
>> > > > > > > > > > > > > > > > then
>> > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > should
>> > > > > > > > > > > > > > > > > > > > > > > really
>> > > > > > > > > > > > > > > > > > > > > > > > > >> change
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> multiple
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > codes.
>> > > In
>> > > > > > > general
>> > > > > > > > > the
>> > > > > > > > > > > > > > errorCode
>> > > > > > > > > > > > > > > > > > itself
>> > > > > > > > > > > > > > > > > > > > > would
>> > > > > > > > > > > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> precise
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > sufficient
>> > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > describing
>> > > > > > the
>> > > > > > > > > server
>> > > > > > > > > > > > side
>> > > > > > > > > > > > > > > > errors.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2.
>> > > > Describe
>> > > > > > > topic
>> > > > > > > > > > > > request:
>> > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > > would
>> > > > > > > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > > > > > > great
>> > > > > > > > > > > > > > > > > > > > > > > to go
>> > > > > > > > > > > > > > > > > > > > > > > > > >> beyond
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > just
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> batching
>> > on
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> topic
>> > > > name
>> > > > > > > regex
>> > > > > > > > > for
>> > > > > > > > > > > this
>> > > > > > > > > > > > > > > > request.
>> > > > > > > > > > > > > > > > > > For
>> > > > > > > > > > > > > > > > > > > > > > > example, a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> very
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > common
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > use
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> case
>> of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> the
>> > > topic
>> > > > > > > command
>> > > > > > > > > is
>> > > > > > > > > > to
>> > > > > > > > > > > > > list
>> > > > > > > > > > > > > > > all
>> > > > > > > > > > > > > > > > > > topics
>> > > > > > > > > > > > > > > > > > > > > whose
>> > > > > > > > > > > > > > > > > > > > > > > > > config
>> > > > > > > > > > > > > > > > > > > > > > > > > >> A's
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > value
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > B.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> topic
>> > > > name
>> > > > > > > regex
>> > > > > > > > > then
>> > > > > > > > > > > we
>> > > > > > > > > > > > > have
>> > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > first
>> > > > > > > > > > > > > > > > > > > > > > retrieve
>> > > > > > > > > > > > > > > > > > > > > > > > > >> __all__
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topics's
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > description
>> > > > > > > info
>> > > > > > > > > and
>> > > > > > > > > > > then
>> > > > > > > > > > > > > > > filter
>> > > > > > > > > > > > > > > > at
>> > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > client
>> > > > > > > > > > > > > > > > > > > > > > > end,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> which
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > burden
>> > > on
>> > > > > ZK.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3.
>> > > Config
>> > > > > > K-Vs
>> > > > > > > in
>> > > > > > > > > > > create
>> > > > > > > > > > > > > > topic:
>> > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > > > > related to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > previous
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point;
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> maybe
>> > > we
>> > > > > can
>> > > > > > > add
>> > > > > > > > > > > another
>> > > > > > > > > > > > > > > metadata
>> > > > > > > > > > > > > > > > > > K-V or
>> > > > > > > > > > > > > > > > > > > > > > just a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> metadata
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > string
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > along
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> with
>> > > > config
>> > > > > > K-V
>> > > > > > > > in
>> > > > > > > > > > > create
>> > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > like
>> > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > did
>> > > > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> offset
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > commit
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> request.
>> > > This
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> field
>> > > can
>> > > > > be
>> > > > > > > > quite
>> > > > > > > > > > > useful
>> > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > storing
>> > > > > > > > > > > > > > > > > > > > > > > information
>> > > > > > > > > > > > > > > > > > > > > > > > > like
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > "owner"
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> who
>> > > issue
>> > > > > the
>> > > > > > > > > create
>> > > > > > > > > > > > > command,
>> > > > > > > > > > > > > > > > etc,
>> > > > > > > > > > > > > > > > > > which
>> > > > > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > > > > quite
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > important
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > > multi-tenant
>> > > > > > > > > setting.
>> > > > > > > > > > > > Then
>> > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > describe
>> > > > > > > > > > > > > > > > > > > > > > > topic
>> > > > > > > > > > > > > > > > > > > > > > > > > >> request
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > can
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > also
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on
>> > > regex
>> > > > of
>> > > > > > the
>> > > > > > > > > > > metadata
>> > > > > > > > > > > > > > field.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4.
>> > > Today
>> > > > > all
>> > > > > > > the
>> > > > > > > > > > admin
>> > > > > > > > > > > > > > > operations
>> > > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > > > async
>> > > > > > > > > > > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> sense
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > return
>> > > > once
>> > > > > > it
>> > > > > > > is
>> > > > > > > > > > > written
>> > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > ZK,
>> > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > > > > why we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> need
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > extra
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > > verification
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> like
>> > > > > > > > > > > > > > > > > testUtil.waitForTopicCreated() /
>> > > > > > > > > > > > > > > > > > > > > verify
>> > > > > > > > > > > > > > > > > > > > > > > > > >> partition
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> reassignment
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > request,
>> > > > > etc.
>> > > > > > > > With
>> > > > > > > > > > > admin
>> > > > > > > > > > > > > > > requests
>> > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > could
>> > > > > > > > > > > > > > > > > > > > > > add
>> > > > > > > > > > > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> flag to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > enable
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > /
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> disable
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > synchronous
>> > > > > > > > > requests;
>> > > > > > > > > > > > when
>> > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > turned
>> > > > > > > > > > > > > > > > > > > > on,
>> > > > > > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> response
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> return
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> until
>> > > the
>> > > > > > > request
>> > > > > > > > > has
>> > > > > > > > > > > > been
>> > > > > > > > > > > > > > > > > > completed. And
>> > > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > async
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > requests
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > "token"
>> > > > > field
>> > > > > > > in
>> > > > > > > > > the
>> > > > > > > > > > > > > > response,
>> > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > then
>> > > > > > > > > > > > > > > > > > > > > only
>> > > > > > > > > > > > > > > > > > > > > > > need a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > general
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > "admin
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > > verification
>> > > > > > > > > request"
>> > > > > > > > > > > > with
>> > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > given
>> > > > > > > > > > > > > > > > > > > > token
>> > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > check
>> > > > > > > > > > > > > > > > > > > > > > > > > >> if the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > async
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> request
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> has
>> > > been
>> > > > > > > > completed.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> 5. +1
>> > > for
>> > > > > > > > extending
>> > > > > > > > > > > > > Metadata
>> > > > > > > > > > > > > > > > > request
>> > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > include
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > controller /
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > coordinator
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > > information,
>> > > > > > > and
>> > > > > > > > > then
>> > > > > > > > > > > we
>> > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > remove
>> > > > > > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > > > ClusterMetadata
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > requests.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > Guozhang
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On
>> > Tue,
>> > > > Mar
>> > > > > > 3,
>> > > > > > > > 2015
>> > > > > > > > > > at
>> > > > > > > > > > > > > 10:23
>> > > > > > > > > > > > > > > AM,
>> > > > > > > > > > > > > > > > > Joel
>> > > > > > > > > > > > > > > > > > > > > Koshy <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > jjkoshy.w@gmail.com
>> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > Thanks
>> > > > for
>> > > > > > > > sending
>> > > > > > > > > > > that
>> > > > > > > > > > > > > out
>> > > > > > > > > > > > > > > Joe
>> > > > > > > > > > > > > > > > -
>> > > > > > > > > > > > > > > > > I
>> > > > > > > > > > > > > > > > > > > > don't
>> > > > > > > > > > > > > > > > > > > > > > > think I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> will be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > able
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> it
>> > > > today,
>> > > > > so
>> > > > > > > if
>> > > > > > > > > > notes
>> > > > > > > > > > > > can
>> > > > > > > > > > > > > be
>> > > > > > > > > > > > > > > > sent
>> > > > > > > > > > > > > > > > > > out
>> > > > > > > > > > > > > > > > > > > > > > > afterward
>> > > > > > > > > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > would
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > great.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> On
>> > > Mon,
>> > > > > Mar
>> > > > > > > 02,
>> > > > > > > > > 2015
>> > > > > > > > > > > at
>> > > > > > > > > > > > > > > > 09:16:13AM
>> > > > > > > > > > > > > > > > > > > > -0800,
>> > > > > > > > > > > > > > > > > > > > > > Gwen
>> > > > > > > > > > > > > > > > > > > > > > > > > >> Shapira
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > > Thanks
>> > > > > for
>> > > > > > > > > sending
>> > > > > > > > > > > > this
>> > > > > > > > > > > > > > out
>> > > > > > > > > > > > > > > > Joe.
>> > > > > > > > > > > > > > > > > > > > Looking
>> > > > > > > > > > > > > > > > > > > > > > > forward
>> > > > > > > > > > > > > > > > > > > > > > > > > >> to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > chatting
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > with
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > > everyone
>> > > > > :)
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > On
>> > > > Mon,
>> > > > > > Mar
>> > > > > > > 2,
>> > > > > > > > > > 2015
>> > > > > > > > > > > at
>> > > > > > > > > > > > > > 6:46
>> > > > > > > > > > > > > > > > AM,
>> > > > > > > > > > > > > > > > > > Joe
>> > > > > > > > > > > > > > > > > > > > > Stein
>> > > > > > > > > > > > > > > > > > > > > > <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > joe.stein@stealth.ly>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > > > Hey, I
>> > > > > > > just
>> > > > > > > > > sent
>> > > > > > > > > > > > out a
>> > > > > > > > > > > > > > > > google
>> > > > > > > > > > > > > > > > > > > > hangout
>> > > > > > > > > > > > > > > > > > > > > > > invite
>> > > > > > > > > > > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> all
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > pmc,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > committers
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > > > > > everyone I
>> > > > > > > > > found
>> > > > > > > > > > > > > working
>> > > > > > > > > > > > > > > on
>> > > > > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > > > KIP.
>> > > > > > > > > > > > > > > > > > > > If
>> > > > > > > > > > > > > > > > > > > > > I
>> > > > > > > > > > > > > > > > > > > > > > > missed
>> > > > > > > > > > > > > > > > > > > > > > > > > >> anyone
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > please
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > > let
>> > > > me
>> > > > > > > know
>> > > > > > > > > and
>> > > > > > > > > > > can
>> > > > > > > > > > > > > > update
>> > > > > > > > > > > > > > > > it,
>> > > > > > > > > > > > > > > > > > np.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > We
>> > > > > > should
>> > > > > > > do
>> > > > > > > > > > this
>> > > > > > > > > > > > > every
>> > > > > > > > > > > > > > > > > Tuesday
>> > > > > > > > > > > > > > > > > > @
>> > > > > > > > > > > > > > > > > > > > 2pm
>> > > > > > > > > > > > > > > > > > > > > > > Eastern
>> > > > > > > > > > > > > > > > > > > > > > > > > >> Time.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Maybe
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > INFRA
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > > help
>> > > > > to
>> > > > > > > > make a
>> > > > > > > > > > > > google
>> > > > > > > > > > > > > > > > account
>> > > > > > > > > > > > > > > > > > so we
>> > > > > > > > > > > > > > > > > > > > > can
>> > > > > > > > > > > > > > > > > > > > > > > manage
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > better?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > To
>> > > > > > discuss
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >>
>> > > > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > in
>> > > > > > > progress
>> > > > > > > > > and
>> > > > > > > > > > > > > related
>> > > > > > > > > > > > > > > JIRA
>> > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > are
>> > > > > > > > > > > > > > > > > > > > > > > > > >> interdependent
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > common
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > ~
>> > > > Joe
>> > > > > > > Stein
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > On
>> > > > > Tue,
>> > > > > > > Feb
>> > > > > > > > > 24,
>> > > > > > > > > > > 2015
>> > > > > > > > > > > > > at
>> > > > > > > > > > > > > > > 2:59
>> > > > > > > > > > > > > > > > > > PM, Jay
>> > > > > > > > > > > > > > > > > > > > > > > Kreps <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
>> > > > > jay.kreps@gmail.com
>> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > Let's
>> > > > > > > stay
>> > > > > > > > on
>> > > > > > > > > > > > Google
>> > > > > > > > > > > > > > > > hangouts
>> > > > > > > > > > > > > > > > > > that
>> > > > > > > > > > > > > > > > > > > > > will
>> > > > > > > > > > > > > > > > > > > > > > > also
>> > > > > > > > > > > > > > > > > > > > > > > > > >> record
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > make
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > > sessions
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > > > available
>> > > > > > > > on
>> > > > > > > > > > > > youtube.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > -Jay
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > On
>> > > > > Tue,
>> > > > > > > Feb
>> > > > > > > > > 24,
>> > > > > > > > > > > > 2015
>> > > > > > > > > > > > > at
>> > > > > > > > > > > > > > > > 11:49
>> > > > > > > > > > > > > > > > > > AM,
>> > > > > > > > > > > > > > > > > > > > > Jeff
>> > > > > > > > > > > > > > > > > > > > > > > > > Holoman
>> > > > > > > > > > > > > > > > > > > > > > > > > >> <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > > > > > > > > > jholoman@cloudera.com
>> > > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > Jay /
>> > > > > > > Joe
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > We're
>> > > > > > > > happy
>> > > > > > > > > > to
>> > > > > > > > > > > > send
>> > > > > > > > > > > > > > > out a
>> > > > > > > > > > > > > > > > > > Webex
>> > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > this
>> > > > > > > > > > > > > > > > > > > > > > > > > >> purpose.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > We
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> record
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> the
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > > > sessions
>> > > > > > > > if
>> > > > > > > > > > > there
>> > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > interest and
>> > > > > > > > > > > > > > > > > > > > > > > publish
>> > > > > > > > > > > > > > > > > > > > > > > > > >> them
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > out.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > > Thanks
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > Jeff
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > On
>> > > > > > Tue,
>> > > > > > > > Feb
>> > > > > > > > > > 24,
>> > > > > > > > > > > > > 2015
>> > > > > > > > > > > > > > at
>> > > > > > > > > > > > > > > > > > 11:28 AM,
>> > > > > > > > > > > > > > > > > > > > > Jay
>> > > > > > > > > > > > > > > > > > > > > > > > > Kreps <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > > > > > > jay.kreps@gmail.com>
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > Let's
>> > > > > > > > try
>> > > > > > > > > > to
>> > > > > > > > > > > > get
>> > > > > > > > > > > > > > the
>> > > > > > > > > > > > > > > > > > technical
>> > > > > > > > > > > > > > > > > > > > > > > hang-ups
>> > > > > > > > > > > > > > > > > > > > > > > > > >> sorted
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > out,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > though.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > really
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > think
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > there
>> > > > > > > > is
>> > > > > > > > > > some
>> > > > > > > > > > > > > > benefit
>> > > > > > > > > > > > > > > > to
>> > > > > > > > > > > > > > > > > > live
>> > > > > > > > > > > > > > > > > > > > > > > discussion
>> > > > > > > > > > > > > > > > > > > > > > > > > vs
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > writing. I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > am
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > > hopeful
>> > > > > that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > if
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > we
>> > > > > > > post
>> > > > > > > > > > > > > > instructions
>> > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > give
>> > > > > > > > > > > > > > > > > > > > > > > ourselves a
>> > > > > > > > > > > > > > > > > > > > > > > > > >> few
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > attempts
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> get
>> > it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > > > working.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > > Tuesday
>> > > > > > > > > at
>> > > > > > > > > > > that
>> > > > > > > > > > > > > > time
>> > > > > > > > > > > > > > > > > would
>> > > > > > > > > > > > > > > > > > work
>> > > > > > > > > > > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> me...any
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > objections?
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > -Jay
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > On
>> > > > > > > Tue,
>> > > > > > > > > Feb
>> > > > > > > > > > > 24,
>> > > > > > > > > > > > > > 2015
>> > > > > > > > > > > > > > > at
>> > > > > > > > > > > > > > > > > > 8:18
>> > > > > > > > > > > > > > > > > > > > AM,
>> > > > > > > > > > > > > > > > > > > > > > Joe
>> > > > > > > > > > > > > > > > > > > > > > > > > Stein
>> > > > > > > > > > > > > > > > > > > > > > > > > >> <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > > > > > > joe.stein@stealth.ly
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > > Weekly
>> > > > > > > > > > > would
>> > > > > > > > > > > > be
>> > > > > > > > > > > > > > > great
>> > > > > > > > > > > > > > > > > > maybe
>> > > > > > > > > > > > > > > > > > > > > like
>> > > > > > > > > > > > > > > > > > > > > > > every
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Tuesday ~
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 1pm
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > ET
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > /
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> PT
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > ????
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > I
>> > > > > > > > don't
>> > > > > > > > > > > mind
>> > > > > > > > > > > > > > google
>> > > > > > > > > > > > > > > > > > hangout
>> > > > > > > > > > > > > > > > > > > > but
>> > > > > > > > > > > > > > > > > > > > > > > there
>> > > > > > > > > > > > > > > > > > > > > > > > > is
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > always
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > some
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> issue
>> or
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > > > whatever
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > so
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > we
>> > > > > > > > know
>> > > > > > > > > > the
>> > > > > > > > > > > > > > apache
>> > > > > > > > > > > > > > > > irc
>> > > > > > > > > > > > > > > > > > > > channel
>> > > > > > > > > > > > > > > > > > > > > > > works.
>> > > > > > > > > > > > > > > > > > > > > > > > > We
>> > > > > > > > > > > > > > > > > > > > > > > > > >> can
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > start
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > there
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> see
>> > > how
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > goes?
>> > > > > > > > > We
>> > > > > > > > > > > can
>> > > > > > > > > > > > > pull
>> > > > > > > > > > > > > > > > > > transcripts
>> > > > > > > > > > > > > > > > > > > > > too
>> > > > > > > > > > > > > > > > > > > > > > > and
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > associate
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> tickets
>> > if
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> need
>> > > be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > makes
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > it
>> > > > > > > > > > helpful
>> > > > > > > > > > > > for
>> > > > > > > > > > > > > > > > things.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > ~
>> > > > > > > > > > Joestein
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > On
>> > > > > > > > Tue,
>> > > > > > > > > > Feb
>> > > > > > > > > > > > 24,
>> > > > > > > > > > > > > > > 2015
>> > > > > > > > > > > > > > > > at
>> > > > > > > > > > > > > > > > > > 11:10
>> > > > > > > > > > > > > > > > > > > > > AM,
>> > > > > > > > > > > > > > > > > > > > > > > Jay
>> > > > > > > > > > > > > > > > > > > > > > > > > >> Kreps <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > > > > > > > > jay.kreps@gmail.com
>> > > > > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > We'd
>> > > > > > > > > > > talked
>> > > > > > > > > > > > > > about
>> > > > > > > > > > > > > > > > > > doing a
>> > > > > > > > > > > > > > > > > > > > > > Google
>> > > > > > > > > > > > > > > > > > > > > > > > > >> Hangout to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > chat
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > about
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> What
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > about
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > > > > > generalizing
>> > > > > > > > > > > > > > > that a
>> > > > > > > > > > > > > > > > > > little
>> > > > > > > > > > > > > > > > > > > > > > > > > further...I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > actually
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > think
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > would
>> > > be
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > good
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > for
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > > > everyone
>> > > > > > > > > > > > > > > spending a
>> > > > > > > > > > > > > > > > > > > > > reasonable
>> > > > > > > > > > > > > > > > > > > > > > > chunk
>> > > > > > > > > > > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > their
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > week
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > stuff
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > maybe
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > sync
>> > > > > > > > > up
>> > > > > > > > > > > > once
>> > > > > > > > > > > > > a
>> > > > > > > > > > > > > > > > week.
>> > > > > > > > > > > > > > > > > I
>> > > > > > > > > > > > > > > > > > > > think
>> > > > > > > > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > > > > > > could
>> > > > > > > > > > > > > > > > > > > > > > > > > >> use
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > time
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > talk
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > > through
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > > design
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > > stuff,
>> > > > > > > > > > > make
>> > > > > > > > > > > > > > sure
>> > > > > > > > > > > > > > > we
>> > > > > > > > > > > > > > > > > > are on
>> > > > > > > > > > > > > > > > > > > > > top
>> > > > > > > > > > > > > > > > > > > > > > of
>> > > > > > > > > > > > > > > > > > > > > > > > > code
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > reviews,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > talk
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> through
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> any
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > > tricky
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > > issues,
>> > > > > > > > > > > > etc.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > We
>> > > > > > > > > can
>> > > > > > > > > > > make
>> > > > > > > > > > > > > it
>> > > > > > > > > > > > > > > > > publicly
>> > > > > > > > > > > > > > > > > > > > > > > available so
>> > > > > > > > > > > > > > > > > > > > > > > > > >> that
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > any
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > one
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > along
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > who
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > > likes.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > Any
>> > > > > > > > > > > > interest
>> > > > > > > > > > > > > in
>> > > > > > > > > > > > > > > > doing
>> > > > > > > > > > > > > > > > > > this?
>> > > > > > > > > > > > > > > > > > > > > If
>> > > > > > > > > > > > > > > > > > > > > > so
>> > > > > > > > > > > > > > > > > > > > > > > > > I'll
>> > > > > > > > > > > > > > > > > > > > > > > > > >> try
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > set
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > it
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > up
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > > starting
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > next
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > week.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > -Jay
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > On
>> > > > > > > > > Tue,
>> > > > > > > > > > > Feb
>> > > > > > > > > > > > > 24,
>> > > > > > > > > > > > > > > > 2015
>> > > > > > > > > > > > > > > > > at
>> > > > > > > > > > > > > > > > > > > > 3:57
>> > > > > > > > > > > > > > > > > > > > > > AM,
>> > > > > > > > > > > > > > > > > > > > > > > > > Andrii
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > <
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
>> > > > > > > > > > > > > > > > > > > > > > > wrote:
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > >
>> > > > > > > > Hi
>> > > > > > > > > > all,
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > >
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > >
>> > > > > > > > > I've
>> > > > > > > > > > > > > updated
>> > > > > > > > > > > > > > > KIP
>> > > > > > > > > > > > > > > > > > page,
>> > > > > > > > > > > > > > > > > > > > > fixed
>> > > > > > > > > > > > > > > > > > > > > > /
>> > > > > > > > > > > > > > > > > > > > > > > > > >> aligned
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > document
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
>> > structure.
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > Also I
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
>> > >>
>> > > > >
>> > > > > > added
>> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
>> > >> >
>> > > > > >
>> > > > > > > >
>> > > > > > > > > some
>> > > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > ...
>> > > > > > > > > > > > > > >
>> > > > > > > > > > > > > > > [Message clipped]
>> > > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > >
>> > > > > > > > > > > > > --
>> > > > > > > > > > > > > -- Guozhang
>> > > > > > > > > > > > >
>> > > > > > > > > > > >
>> > > > > > > > > > >
>> > > > > > > > > >
>> > > > > > > > >
>> > > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > --
>> > > > > > > -- Guozhang
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> >
>> >
>> > --
>> > Thanks,
>> > Neha
>> >
>>
>
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
201. I think it's probably easier to consolidate on DTR. If we evolve TMR,
the producer will be using the latest version of TMR. However, the latest
version of TMR won't trigger auto topic creation. This means that we need
to change the producer to use CreateTopicRequest. If we consolidate on DTR.
We can fix the producer logic later.

205. Also, there was a remaining item on how to get the controller info. We
can either add the controller id to the response of DTR or create a new
cluster metadata request/response. The cluster meta response will then
include the detailed information about every broker and the controller. I
think the latter is probably better. For example, when we add the
multi-port support (for security), the cluster meta response can include
all ports for a broker. In DTR, we only need to return one broker port to
the client.

Thanks,

Jun

On Tue, Mar 24, 2015 at 12:26 PM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Hi all,
>
> A short summary of our discussion:
>
> 201. DescribeTopicRequest(DTR) vs TopicMetadataRequest(TMR)
>
> *Issue*: Topic description should include topic-level configuration,
> other information is available in TMR - leader, ISR, AR.
> So now there are two options - either add full-fledged new DTR
> that will include TMR + all needed information (which will be used then
> instead TMR) or extend TMR to include topic-level configuration.
>
> *Resolution*: it was decided to vote on it: either a) *extend TMR* or b)
> *add
> DTR*.
> I'd go with a) - we will have to roll out TMR_V1 but it's better than
> having
> "dead" requests if we add DTR and leave TMR unused in future versions
>
> 202. CreateTopicRequest sync/async semantics
>
> *Issue*: What should be a post-condition of Create/Alter/Delete
> requests - the command has been completed or command has been
> initiated?
> Even though it looks reasonable that users would typically expect to
> have topic created upon CreateTopicRequest the corner case is that
> topic can be deleted right after creation, so received CreateTopicResponse
> cannot be a 100% guarantee topic is created and available, even if
> CreateTopicRequest is synchronous. Also, making request blocking on
> server may be a hard thing to do right now (check Jun's email from Mar 21).
>
> *Resolution*: for now it makes sense to start with async semantics for
> Create/Alter/Delete.
>
> 203. VerifyReassignPartition vs leverage DescribeTopicRequest
>
> *Issue*: DescribeTopicRequest (or TopicMetadaRequest) should be enough,
> since controller doesn't remove partitions from admin/reassign_partitions
> until
> the assignment actually matches target assignment so partition reassignment
> cannot fail for specific partition in the current implementation.
> Everything else
> ("Completed" and "In Progress" state) can be checked with assigned
> partitions
> field from DescribeTopicRequest.
>
> *Resolution*: remove VerifyReassignPartitionsRequest
>
> 204. Also it was discussed that after we finalize question 201 and port
> java protocol definition (KAFKA-1927) I will split patch to separate
> pieces,
> first will cover Wire Protocol changes, since we have a common vision on
> it.
>
> Please let me know your thoughts on q. 201!
>
> Thanks,
> Andrii Biletskyi
>
>
>
>
> On Tue, Mar 24, 2015 at 7:57 PM, Neha Narkhede <ne...@confluent.io> wrote:
>
> > I'd like to resurface the discussion of sync vs async topic creation.
> > Possibly, we can discuss that in today's KIP meeting. I am worried about
> > pushing the burden of handling async topic creation on the clients. This
> is
> > one of the most frequently asked questions on the mailing list and I hope
> > we find a way to address it as we separate the metadata request and topic
> > creation. Since topic creation doesn't take that long, there should be a
> > really easy way for clients to express their preference for a sync topic
> > creation request where the receipt of a response suggests completion of
> > topic creation.
> >
> > The behavior that I'm expecting is that if I'm a client and I choose to
> > create a topic synchronously, after I receive a response, my first send
> > request for that topic should *not* fail with some error.
> >
> > On Tue, Mar 24, 2015 at 8:02 AM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Andrii,
> > >
> > >
> > > 111.4 Yes, we can discuss this in the meeting today.
> > >
> > > 112. The question is when will a PartitionReassignment fail. Currently,
> > it
> > > can fail if the input is incorrect (e.g, invalid partition/replica).
> > > However, this can be detected when the ReassignPartitionRequest is
> > issued.
> > > Once the process of partition reassignment is started, it will not
> fail.
> > > The controller just waits until the process completes. We can improve
> the
> > > status reporting of partition reassignment. However, I am not sure if
> > that
> > > needs to be tied to this KIP.
> > >
> > > 115.1 Yes, that makes sense. I misunderstood it. I thought
> > > PreferredReplicaLeaderElectionInProgress and
> ReassignPartitionsInProgress
> > > are used in the status check.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Tue, Mar 24, 2015 at 3:46 AM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Jun,
> > > >
> > > > Thanks for such an accurate review!
> > > >
> > > > Most of your remarks I didn't fix in the previous change because
> > > > I thought we'd finalize them today, during the call. Anyway, I've
> > > > fixed them now.
> > > > My comments to ensure I covered everything:
> > > >
> > > > 110. Fixed. Removed global error code.
> > > >
> > > > 111.1-3. Removed redundant TopicName, Partitions and
> ReplicationFactor
> > > > 111.4 Yes, now we are in the situation when TopicMetadataResponse
> > > > contains everything we need except topic level configs. I'm
> > > > okay with both solutions, probably makes sense to discuss and pick
> > > > the preferable one.
> > > >
> > > > 112. Need to think about it a bit more. Currently we may distinguish
> > > > 3 states of specific partition being reassigned:
> > > > a) Completed - absent in /reassign_partitions zk path and
> > DescribeTopic/
> > > > TopicMetadata shows expected assignment
> > > > 2) In progress - present in /reassign_partitions
> > > > 3) Failed - absent in /reassign_partitions but
> > > DescribeTopic/TopicMetadata
> > > > replicas field doesn't correspond expected
> > > > Now, if use only DescribeTopic to check reassignment status - how can
> > we
> > > > distinguish states "In Progress" and "Failed"?
> > > > Also, there are some concerns that current reassignment status
> provided
> > > > by ReaasignPartitionsCommand (and the same was intended in KIP-4) is
> > > > very spurious, non-informative. Probably it makes sense to add
> > additional
> > > > data there (thus separate request may be useful), I'm considering
> > > different
> > > > options now - btw, comments here are highly appreciated!
> > > >
> > > > 113-114. Okay, made it compliant with topic commands. About "The
> error
> > > code
> > > > will
> > > > then be per topic" - just a note: currently
> (ReassignPartitionsCommand)
> > > > validation
> > > > is done for the whole reassignment string, so in case of errors for
> > > > specific
> > > > partitions the reassignment is NOT started at all. Both options look
> > fine
> > > > for me,
> > > > it's just the logic will be slightly changed.
> > > >
> > > > 115.1 Removed NotControllerReceivedAdminCommand. But why should we
> > > > remove PreferredReplicaLeaderElectionInProgress and
> > > > ReassignPartitionsInProgress?
> > > > On Reassign/Preferred..Request we create admin zk path, we can of
> > course
> > > > update the path, but wouldn't it be safer if we simply refuse to
> start
> > > new
> > > > reassignment in case one is in progress?
> > > >
> > > > 115.2 Yes, paragraph Protocol Errors covers only errors that are
> > proposed
> > > > to be added. I didn't want to list all errors that are to be added
> > since
> > > > anyway
> > > > the final version will be clear after the development is completed.
> > > >
> > > > 100. Fixed, normalized json string in Create/Alter.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Tue, Mar 24, 2015 at 12:58 AM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Andrii,
> > > > >
> > > > > I looked at the latest wiki for this KIP. I have a few more
> comments.
> > > > >
> > > > > 110. In CreateTopicResponse, AlterTopicResponse,
> DeleteTopicResponse
> > > and
> > > > > DescribeTopicResponse, we probably don't need the global error
> code.
> > > Our
> > > > > current convention is to just use the same global error code in
> each
> > > > topic.
> > > > > This simplifies the error checking on the client side.
> > > > >
> > > > > 111. DescribeTopicResponse:
> > > > > 111.1 Our protocol definition doesn't support optional fields. So,
> we
> > > > can't
> > > > > make Leader an optional field.
> > > > > 111.2 Do we really need the field Partitions and ReplicationFactor
> in
> > > > > TopicConfigDetails?
> > > > > The former is basically the size of the TopicPartitionDetails array
> > and
> > > > the
> > > > > latter can be found from the size of the Replica array.
> > > > > 111.3 Do we need TopicName in TopicDescription since it's already
> in
> > > > > DescribeTopicResponse?
> > > > > 111.4 Finally, DescribeTopicResponse is very similar to
> > > > > TopicMetadataResponse. So, it probably makes sense just to keep one
> > of
> > > > them
> > > > > in the future. Should we just use DescribeTopicRequest/Response to
> > > > replace
> > > > > TopicMetadataRequest/Response in the producer/consumer client in
> the
> > > > future
> > > > > (since DescribeTopicRequest doesn't trigger auto topic creation)?
> If
> > > so,
> > > > we
> > > > > will need to add the broker list in DescribeTopicResponse.
> > > > >
> > > > > 112. Thinking about this a bit more, I don't think we need a
> separate
> > > > > VerifyReassignPartitionRequest/Response.
> > > > > We can just use DescribeTopic to get the assigned replicas and
> check
> > if
> > > > > they match the target replica assignment in the client. The
> > controller
> > > > > propagates the metadata change after the reassignment completes for
> > > each
> > > > > partition.
> > > > >
> > > > > 113. ReassignPartitionRequest: For consistency, we probably want to
> > > nest
> > > > > the partition data under topic. So instead of
> > > > >   ReassignPartitionRequest => [Topic PartitionId [ReplicaId]]
> > > > > we can have sth like
> > > > >   ReassignPartitionRequest => [Topic [PartitionId [ReplicaId]]]
> > > > > The error code will then be per topic.
> > > > >
> > > > > 114. PreferredReplicaLeaderElectionRequest: Same as the above.
> > Instead
> > > of
> > > > >   PreferredReplicaLeaderElectionRequest => [Topic PartitionId]
> > > > > we can have
> > > > >   PreferredReplicaLeaderElectionRequest => [Topic [PartitionId]]
> > > > > Again, the error code will be per topic.
> > > > >
> > > > > 115. ErrorCode:
> > > > > 115.1 NotControllerReceivedAdminRequest,
> > > > > PreferredReplicaLeaderElectionInProgress
> > > > > and ReassignPartitionsInProgress are likely not needed any more.
> > > > > 115.2 We probably need to add UnknownTopicOrPartitionCode since
> it's
> > a
> > > > > valid error code for some of the requests (e.g. describe, alter,
> etc)
> > > > when
> > > > > a non-existing topic is specified.
> > > > >
> > > > >
> > > > > Previous comments.
> > > > > 100. ReplicaAssignment in CreateTopicRequest and AlterTopicRequest
> > are
> > > > > still a json string. We need to flatten them.
> > > > >
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Sat, Mar 21, 2015 at 2:03 AM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Guozhang,
> > > > > >
> > > > > > I'm not sure I understand how can we use those tools in CLI.
> > > > > > First of all, those are *Test*Utils, e.g.
> > > waitUntilMetadataIsPropagated
> > > > > > (which might be very useful for us) requires
> > servers:Seq[KafkaServer]
> > > > > > as an argument:
> > > > > >
> > > > > > def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer],
> topic:
> > > > > String,
> > > > > > partition: Int, timeout: Long = 5000L): Int
> > > > > >
> > > > > > how can we have access it at runtime in one of the brokers?
> > > > > >
> > > > > > Secondly, how can user directly call these tools if zookeeper
> might
> > > be
> > > > > > not accessible (which is used in the tools, right?) at all - e.g.
> > > > behind
> > > > > > the VPC
> > > > > > in AWS. Furthermore, I think with KIP-4 we are trying to create
> an
> > > > > > abstraction and
> > > > > > a single point of zookeeper interactions, to eliminate direct
> calls
> > > to
> > > > ZK
> > > > > > either to get
> > > > > > some cluster information or to change something.
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > > On Sat, Mar 21, 2015 at 3:37 AM, Guozhang Wang <
> wangguoz@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > Andrii,
> > > > > > >
> > > > > > > Actually the checking logic Jun mentioned is already
> implemented
> > as
> > > > > > > TestUtils.waitUntilXXX (LeaderIsElected, MetadataIsPropagated,
> > > > etc...)
> > > > > I
> > > > > > > think we can extend these functions as CLI tools like
> > TopicCommand
> > > so
> > > > > > that
> > > > > > > users re-implementing such endpoint can directly call something
> > > like
> > > > > > > java.tools.WaitUntilXXX (of course this requires them to have
> > javac
> > > > > > > installed, which should be a reasonable requirement?)
> > > > > > >
> > > > > > > Guozhang
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >
> > > > > > > > Jun,
> > > > > > > >
> > > > > > > > Not that I was saying we need to make requests blocking on
> > > server,
> > > > > > > > it was just to emphasize that with async requests a client
> > > > > > > implementations
> > > > > > > > may be a little bit more than just issue request - get the
> > > > response.
> > > > > > > > Thanks for the explanation, I understand now that we can go
> > with
> > > > > agreed
> > > > > > > > solution though it may not be perfect.
> > > > > > > > I believe this was one of the last controversial questions
> from
> > > the
> > > > > > list.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Andrii Biletskyi
> > > > > > > >
> > > > > > > > On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > >
> > > > > > > > > Andrii,
> > > > > > > > >
> > > > > > > > > A few points.
> > > > > > > > >
> > > > > > > > > 1. Create/Alter can typically complete quickly. So, it's
> > > possible
> > > > > to
> > > > > > > make
> > > > > > > > > the request block until it's completed. However, currently,
> > > doing
> > > > > > this
> > > > > > > at
> > > > > > > > > the broker is a bit involved. To make Create block, we will
> > > need
> > > > to
> > > > > > add
> > > > > > > > > some callbacks in KafkaController. This is possible.
> However,
> > > the
> > > > > > > > > controller logic currently is pretty completed. It would
> > > probably
> > > > > be
> > > > > > > > better
> > > > > > > > > if we clean it up first before adding more complexity to
> it.
> > > > Alter
> > > > > is
> > > > > > > > even
> > > > > > > > > trickier. Adding partition is currently handled through
> > > > > > > KafkaController.
> > > > > > > > So
> > > > > > > > > it can be dealt with in a similar way. However, Alter
> config
> > is
> > > > > done
> > > > > > > > > completely differently. It doesn't go through the
> controller.
> > > > > > Instead,
> > > > > > > > each
> > > > > > > > > broker listens to ZooKeeper directly. So, it's not clear if
> > > there
> > > > > is
> > > > > > an
> > > > > > > > > easy way on the broker to figure out whether a config is
> > > applied
> > > > on
> > > > > > > every
> > > > > > > > > broker.
> > > > > > > > >
> > > > > > > > > 2. Delete can potentially take long if a replica to be
> > deleted
> > > is
> > > > > > > > offline.
> > > > > > > > > PreferredLeader/PartitionReassign can also take long. So,
> we
> > > > can't
> > > > > > > really
> > > > > > > > > make those requests block on the broker.
> > > > > > > > >
> > > > > > > > > As you can see, at this moment it's not easy to make all
> > admin
> > > > > > requests
> > > > > > > > > block on the broker. So, if we want the blocking feature in
> > the
> > > > > admin
> > > > > > > > > utility in the short term, doing the completion check at
> the
> > > > admin
> > > > > > > client
> > > > > > > > > is probably an easier route, even though it may not be
> ideal.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >
> > > > > > > > > > Jun,
> > > > > > > > > >
> > > > > > > > > > I see your point. But wouldn't that lead to a "fat"
> client
> > > > > > > > > implementations?
> > > > > > > > > > Suppose someone would like to implement client for Admin
> > Wire
> > > > > > > protocol.
> > > > > > > > > > Not only people will have to code quite complicated logic
> > > like
> > > > > > "send
> > > > > > > > > > describe
> > > > > > > > > > request to each broker" (again state machin?) but it will
> > > also
> > > > > mean
> > > > > > > > > people
> > > > > > > > > > must understand internal kafka logic related to topic
> > storage
> > > > and
> > > > > > how
> > > > > > > > > > information is propageted from the controller to brokers.
> > > > > > > > > > I see this like a dilemma between having a concise Wire
> > > > Protocol
> > > > > > and
> > > > > > > > > > self-sufficient API to make client implementations
> simple.
> > > > > > > > > > I don't have a win-win solution though.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Andrii Biletskyi
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <
> > jun@confluent.io>
> > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > For 1), 2) and 3), blocking would probably mean that
> the
> > > new
> > > > > > > metadata
> > > > > > > > > is
> > > > > > > > > > > propagated to every broker. To achieve that, the client
> > can
> > > > > keep
> > > > > > > > > issuing
> > > > > > > > > > > the describe topic request to every broker until it
> sees
> > > the
> > > > > new
> > > > > > > > > metadata
> > > > > > > > > > > in the response.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Hm, actually the ticket you linked, Guozhang, brings
> as
> > > > back
> > > > > > > > > > > > to the problem what should be considered a
> > post-condition
> > > > for
> > > > > > > > > > > > each of the admin commands.
> > > > > > > > > > > > In my understanding:
> > > > > > > > > > > >
> > > > > > > > > > > > 1) CreateTopic - broker created
> /brokers/topics/<topic>
> > > > > > > > > > > > (Not the controller picked up changes from zk and
> > > > broadcasted
> > > > > > > > > > > > LeaderAndIsr and UpdateMetadata)
> > > > > > > > > > > >
> > > > > > > > > > > > 2) AlterTopic - same as 1) - broker changed
> assignment
> > > data
> > > > > > > > > > > > in zookeeper or created admin path for topic config
> > > change
> > > > > > > > > > > >
> > > > > > > > > > > > 3) DeleteTopic - admin path /admin/delete_topics is
> > > created
> > > > > > > > > > > >
> > > > > > > > > > > > 4) ReassignPartitions and PreferredReplica -
> > > corresponding
> > > > > > admin
> > > > > > > > > > > > path is created
> > > > > > > > > > > >
> > > > > > > > > > > > Now what can be considered a completed operation from
> > the
> > > > > > > client's
> > > > > > > > > > > > perspective?
> > > > > > > > > > > > 1) Topic is created once corresponding data is in zk
> > > > > > > > > > > > (I remember there were some thoughts that it'd be
> good
> > to
> > > > > > > consider
> > > > > > > > > > > > topic created once all replicas receive information
> > about
> > > > it
> > > > > > and
> > > > > > > > thus
> > > > > > > > > > > > clients can produce/consume from it, but as was
> > discussed
> > > > > this
> > > > > > > > seems
> > > > > > > > > > > > to be a hard thing to do)
> > > > > > > > > > > >
> > > > > > > > > > > > 2) Probably same as 1), so right after AlterTopic is
> > > issued
> > > > > > > > > > > >
> > > > > > > > > > > > 3) The topic has been removed from /brokers/topics
> > > > > > > > > > > >
> > > > > > > > > > > > 4) ReassignPartitions and PrefferedReplica were
> > discussed
> > > > > > > earlier -
> > > > > > > > > > > > in short the former is completed once partition state
> > > info
> > > > in
> > > > > > zk
> > > > > > > > > > matches
> > > > > > > > > > > > reassignment request and admin path is empty, the
> > latter
> > > -
> > > > > once
> > > > > > > > data
> > > > > > > > > > > > in zk shows that head of assignned replicas of the
> > > > partition
> > > > > > and
> > > > > > > > > leader
> > > > > > > > > > > > is the same replica
> > > > > > > > > > > >
> > > > > > > > > > > > Thoughts?
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <
> > > > > > > wangguoz@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > I think while loop is fine for supporting blocking,
> > > just
> > > > > that
> > > > > > > we
> > > > > > > > > need
> > > > > > > > > > > to
> > > > > > > > > > > > > add back off to avoid bombarding brokers with
> > > > DescribeTopic
> > > > > > > > > requests.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Also I have linked KAFKA-1125
> > > > > > > > > > > > > <https://issues.apache.org/jira/browse/KAFKA-1125>
> > to
> > > > your
> > > > > > > > > proposal,
> > > > > > > > > > > and
> > > > > > > > > > > > > when KAFKA-1694 is done this ticket can also be
> > closed.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Great.
> > > > > > > > > > > > > > I want to elaborate this a bit more, to see we
> are
> > on
> > > > the
> > > > > > > same
> > > > > > > > > page
> > > > > > > > > > > > > > concerning the client code.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > So with all topic commands being async a client
> > > > > > (AdminClient
> > > > > > > in
> > > > > > > > > our
> > > > > > > > > > > > > > case or any other other client people would like
> to
> > > > > > > implement)
> > > > > > > > to
> > > > > > > > > > > > support
> > > > > > > > > > > > > > a blocking operation (which seems to be a natural
> > > > > use-case
> > > > > > > e.g.
> > > > > > > > > for
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > creation): would have to do:
> > > > > > > > > > > > > > 1. issue CreateTopicRequest
> > > > > > > > > > > > > > 2. if successful, in a "while" loop send
> > > > > > DescribeTopicRequest
> > > > > > > > and
> > > > > > > > > > > > > > break the loop once all topics are returned in
> > > response
> > > > > (or
> > > > > > > > upon
> > > > > > > > > > > > > timeout).
> > > > > > > > > > > > > > 3. if unsuccessful throw exception
> > > > > > > > > > > > > > Would it be okay?
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <
> > > > > jun@confluent.io
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I think you are right. It seems that only
> > > > > > > ReassignPartitions
> > > > > > > > > > needs
> > > > > > > > > > > a
> > > > > > > > > > > > > > > separate verification request.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii
> > Biletskyi <
> > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > I like this idea too. Let's stick with that.
> > I'll
> > > > > > update
> > > > > > > > KIP
> > > > > > > > > > > > > > accordingly.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I was also thinking we can avoid adding
> > dedicated
> > > > > > status
> > > > > > > > > check
> > > > > > > > > > > > > > > > requests for topic commands. - We have
> > everything
> > > > in
> > > > > > > > > > > DescribeTopic
> > > > > > > > > > > > > > > > for that! E.g.:
> > > > > > > > > > > > > > > > User issued CreateTopic - to check the status
> > > > client
> > > > > > > sends
> > > > > > > > > > > > > > DescribeTopic
> > > > > > > > > > > > > > > > and checks whether is something returned for
> > that
> > > > > > topic.
> > > > > > > > The
> > > > > > > > > > same
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > alteration, deletion.
> > > > > > > > > > > > > > > > Btw, PreferredReplica status can be also
> > checked
> > > > with
> > > > > > > > > > > > > > > DescribeTopicRequest
> > > > > > > > > > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > > > > > > > > > For ReassignPartitions as discussed we'll
> need
> > to
> > > > > have
> > > > > > a
> > > > > > > > > > separate
> > > > > > > > > > > > > > > Verify...
> > > > > > > > > > > > > > > > request.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang
> Wang
> > <
> > > > > > > > > > > wangguoz@gmail.com
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > +1 on broker writing to ZK for async
> > handling.
> > > I
> > > > > was
> > > > > > > > > thinking
> > > > > > > > > > > > that
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > end state the admin requests would be
> > > eventually
> > > > > sent
> > > > > > > to
> > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > either
> > > > > > > > > > > > > > > > > through re-routing or clients discovering
> > them,
> > > > > > instead
> > > > > > > > of
> > > > > > > > > > > > letting
> > > > > > > > > > > > > > > > > controller listen on ZK admin path. But
> > > thinking
> > > > > > about
> > > > > > > > it a
> > > > > > > > > > > > second
> > > > > > > > > > > > > > > time,
> > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > think it is actually simpler to let
> > controller
> > > > > manage
> > > > > > > > > > > > > > > > > incoming queued-up admin requests through
> ZK.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel
> Koshy <
> > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > +1 as well. I think it helps to keep the
> > > > > rerouting
> > > > > > > > > approach
> > > > > > > > > > > > > > > orthogonal
> > > > > > > > > > > > > > > > > > to this KIP.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700,
> > Jay
> > > > > Kreps
> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > I'm +1 on Jun's suggestion as long as
> it
> > > can
> > > > > work
> > > > > > > for
> > > > > > > > > all
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > requests.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun
> Rao
> > <
> > > > > > > > > > jun@confluent.io
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > (a) Admin requests can be sent to and
> > > > handled
> > > > > > by
> > > > > > > > any
> > > > > > > > > > > > broker.
> > > > > > > > > > > > > > > > > > > > (b) Admin requests are processed
> > > > > > asynchronously,
> > > > > > > at
> > > > > > > > > > least
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > now.
> > > > > > > > > > > > > > > > > > That is,
> > > > > > > > > > > > > > > > > > > > when the client gets a response, it
> > just
> > > > > means
> > > > > > > that
> > > > > > > > > the
> > > > > > > > > > > > > request
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > initiated, but not necessarily
> > completed.
> > > > > Then,
> > > > > > > > it's
> > > > > > > > > up
> > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > issue another request to check the
> > status
> > > > for
> > > > > > > > > > completion.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > To support (a), we were thinking of
> > doing
> > > > > > request
> > > > > > > > > > > > forwarding
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > controller (utilizing KAFKA-1912). I
> am
> > > > > making
> > > > > > an
> > > > > > > > > > > > alternative
> > > > > > > > > > > > > > > > > proposal.
> > > > > > > > > > > > > > > > > > > > Basically, the broker can just write
> to
> > > > > > ZooKeeper
> > > > > > > > to
> > > > > > > > > > > inform
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > > > > about the request. For example, to
> > handle
> > > > > > > > > > > > > > partitionReassignment,
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > > will just write the requested
> > partitions
> > > to
> > > > > > > > > > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > > > > > > > > > (like what AdminUtils currently does)
> > and
> > > > > then
> > > > > > > > send a
> > > > > > > > > > > > > response
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > client. This shouldn't take long and
> > the
> > > > > > > > > implementation
> > > > > > > > > > > > will
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > simpler
> > > > > > > > > > > > > > > > > > > > than forwarding the requests to the
> > > > > controller
> > > > > > > > > through
> > > > > > > > > > > RPC.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM,
> Andrii
> > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Jun,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > I might be wrong but didn't we
> agree
> > we
> > > > > will
> > > > > > > let
> > > > > > > > > any
> > > > > > > > > > > > broker
> > > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > cluster handle *long-running* admin
> > > > > requests
> > > > > > > (at
> > > > > > > > > this
> > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > reassignPartitions), via zk admin
> > path.
> > > > > Thus
> > > > > > > > > > > CreateTopics
> > > > > > > > > > > > > etc
> > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM,
> Jun
> > > > Rao <
> > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I think we agreed that those
> admin
> > > > > requests
> > > > > > > can
> > > > > > > > > be
> > > > > > > > > > > > issued
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > > > broker.
> > > > > > > > > > > > > > > > > > > > > > Because of that, there doesn't
> seem
> > > to
> > > > > be a
> > > > > > > > > strong
> > > > > > > > > > > need
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > controller. So, perhaps we can
> > > proceed
> > > > by
> > > > > > not
> > > > > > > > > > making
> > > > > > > > > > > > any
> > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > format of TMR right now. When we
> > > start
> > > > > > using
> > > > > > > > > create
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > producer, we will need a new
> > version
> > > of
> > > > > TMR
> > > > > > > > that
> > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > creation. But that can be done
> > later.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > As a first cut implementation, I
> > > think
> > > > > the
> > > > > > > > broker
> > > > > > > > > > can
> > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > write
> > > > > > > > > > > > > > > > > > to ZK
> > > > > > > > > > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > > > > > > > > > requests, instead of forwarding
> > them
> > > to
> > > > > the
> > > > > > > > > > > controller.
> > > > > > > > > > > > > > This
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > > > > > > > > > the implementation on the broker
> > > side.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM,
> > > Joel
> > > > > > Koshy
> > > > > > > <
> > > > > > > > > > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > For (1) yes we will circle back
> > on
> > > > that
> > > > > > > > shortly
> > > > > > > > > > > after
> > > > > > > > > > > > > > > syncing
> > > > > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > person. I think it is close to
> > > > getting
> > > > > > > > > committed
> > > > > > > > > > > > > although
> > > > > > > > > > > > > > > > > > development
> > > > > > > > > > > > > > > > > > > > > > > for KAFKA-1927 can probably
> begin
> > > > > without
> > > > > > > it.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > There is one more item we
> covered
> > > at
> > > > > the
> > > > > > > > > hangout.
> > > > > > > > > > > > i.e.,
> > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > want to add the coordinator to
> > the
> > > > > topic
> > > > > > > > > metadata
> > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > > > > > > a clearer
> ClusterMetadataRequest.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > There are two reasons I think
> we
> > > > should
> > > > > > try
> > > > > > > > and
> > > > > > > > > > > avoid
> > > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > > > > > > > > > - It is irrelevant to topic
> > > metadata
> > > > > > > > > > > > > > > > > > > > > > > - If we finally do request
> > > rerouting
> > > > in
> > > > > > > Kafka
> > > > > > > > > > then
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > field
> > > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > >   little to no value. (It still
> > > helps
> > > > > to
> > > > > > > > have a
> > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > > > > >   ClusterMetadataRequest to
> query
> > > for
> > > > > > > > > > cluster-wide
> > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > > > > >   'which broker is the
> > controller?'
> > > > as
> > > > > > Joe
> > > > > > > > > > > > mentioned.)
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > I think it would be cleaner to
> > have
> > > > an
> > > > > > > > explicit
> > > > > > > > > > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > that you can send to any broker
> > in
> > > > > order
> > > > > > to
> > > > > > > > > > obtain
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > > > > in the future possibly other
> > > > > cluster-wide
> > > > > > > > > > > > > information). I
> > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > main argument against doing
> this
> > > and
> > > > > > > instead
> > > > > > > > > > adding
> > > > > > > > > > > > it
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > metadata response was
> > convenience -
> > > > > i.e.,
> > > > > > > you
> > > > > > > > > > don't
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > discover
> > > > > > > > > > > > > > > > > > > > > > > the controller in advance.
> > > However, I
> > > > > > don't
> > > > > > > > see
> > > > > > > > > > > much
> > > > > > > > > > > > > > actual
> > > > > > > > > > > > > > > > > > > > > > > benefit/convenience in this and
> > in
> > > > fact
> > > > > > > think
> > > > > > > > > it
> > > > > > > > > > > is a
> > > > > > > > > > > > > > > > > non-issue.
> > > > > > > > > > > > > > > > > > Let
> > > > > > > > > > > > > > > > > > > > > > > me know if I'm overlooking
> > > something
> > > > > > here.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > As an example, say we need to
> > > > initiate
> > > > > > > > > partition
> > > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > > > > > > issuing the new
> > > > > ReassignPartitionsRequest
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > > (assume
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > already have the desired manual
> > > > > partition
> > > > > > > > > > > > assignment).
> > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > are to
> > > > > > > > > > > > > > > > > > > > > > > augment topic metadata response
> > > then
> > > > > the
> > > > > > > flow
> > > > > > > > > be
> > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > :
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > - Issue topic metadata request
> to
> > > any
> > > > > > > broker
> > > > > > > > > (and
> > > > > > > > > > > > > > discover
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > > > > > > > > > - Connect to controller if
> > required
> > > > > > (i.e.,
> > > > > > > if
> > > > > > > > > the
> > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > above
> > > > > > > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > > > > > > - Issue the partition
> > reassignment
> > > > > > request
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > > > > controller.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > With an explicit cluster
> metadata
> > > > > request
> > > > > > > it
> > > > > > > > > > would
> > > > > > > > > > > > be:
> > > > > > > > > > > > > > > > > > > > > > > - Issue cluster metadata
> request
> > to
> > > > any
> > > > > > > > broker
> > > > > > > > > > > > > > > > > > > > > > > - Connect to controller if
> > required
> > > > > > (i.e.,
> > > > > > > if
> > > > > > > > > the
> > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > above
> > > > > > > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > > > > > > - Issue the partition
> > reassignment
> > > > > > request
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > So it seems to add little
> > practical
> > > > > value
> > > > > > > and
> > > > > > > > > > > bloats
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > response with an irrelevant
> > detail.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > The other angle to this is the
> > > > > following
> > > > > > -
> > > > > > > is
> > > > > > > > > it
> > > > > > > > > > a
> > > > > > > > > > > > > matter
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > naming?
> > > > > > > > > > > > > > > > > > > > > > > Should we just rename topic
> > > metadata
> > > > > > > > > > > request/response
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > > > > MetadataRequest/Response and
> add
> > > > > cluster
> > > > > > > > > metadata
> > > > > > > > > > > to
> > > > > > > > > > > > > it?
> > > > > > > > > > > > > > By
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > same
> > > > > > > > > > > > > > > > > > > > > > > token should we also allow
> > querying
> > > > for
> > > > > > the
> > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > coordinator
> > > > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > > > > in future transaction
> > coordinator)
> > > as
> > > > > > well?
> > > > > > > > > This
> > > > > > > > > > > > leads
> > > > > > > > > > > > > > to a
> > > > > > > > > > > > > > > > > > bloated
> > > > > > > > > > > > > > > > > > > > > > > request which isn't very
> > appealing
> > > > and
> > > > > > > > > altogether
> > > > > > > > > > > > > > > confusing.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at
> > 09:34:12AM
> > > > > -0700,
> > > > > > > Jun
> > > > > > > > > Rao
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 1. I just realized that in
> > order
> > > to
> > > > > > start
> > > > > > > > > > working
> > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > > need to merge the changes to
> > > > > > > > > > OffsetCommitRequest
> > > > > > > > > > > > > (from
> > > > > > > > > > > > > > > > 0.8.2)
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > > > > > > > > > This is planned to be done as
> > > part
> > > > of
> > > > > > > > > > KAFKA-1634.
> > > > > > > > > > > > So,
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > > > > > > and Joel's help to wrap this
> > up.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 2. Thinking about this a bit
> > > more,
> > > > if
> > > > > > the
> > > > > > > > > > > semantic
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > "write"
> > > > > > > > > > > > > > > > > > > > > > > > requests is async (i.e.,
> after
> > > the
> > > > > > client
> > > > > > > > > gets
> > > > > > > > > > a
> > > > > > > > > > > > > > > response,
> > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > > > > > > > > > that the operation is
> > initiated,
> > > > but
> > > > > > not
> > > > > > > > > > > > necessarily
> > > > > > > > > > > > > > > > > > completed), we
> > > > > > > > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > > > > > > really need to forward the
> > > requests
> > > > > to
> > > > > > > the
> > > > > > > > > > > > > controller.
> > > > > > > > > > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > > > > > > > > > receiving broker can just
> write
> > > the
> > > > > > > > operation
> > > > > > > > > > to
> > > > > > > > > > > ZK
> > > > > > > > > > > > > as
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > > > > > > > > > line tool previously does.
> This
> > > > will
> > > > > > > > simplify
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > > implementation.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 8. There is another
> > > implementation
> > > > > > detail
> > > > > > > > for
> > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > topic.
> > > > > > > > > > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > want to read the topic config
> > > from
> > > > > the
> > > > > > > > broker
> > > > > > > > > > > > cache,
> > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > > > > > > > > > Currently, every broker reads
> > the
> > > > > > > > topic-level
> > > > > > > > > > > > config
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > > > > > > However, it ignores those for
> > > > topics
> > > > > > not
> > > > > > > > > hosted
> > > > > > > > > > > on
> > > > > > > > > > > > > > > itself.
> > > > > > > > > > > > > > > > > So,
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > > to change TopicConfigManager
> a
> > > bit
> > > > so
> > > > > > > that
> > > > > > > > it
> > > > > > > > > > > > caches
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13
> > PM,
> > > > > Andrii
> > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > Thanks for a great
> > discussion!
> > > > > > > > > > > > > > > > > > > > > > > > > Here are the actions
> points:
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala
> > > > requests
> > > > > > > > > objects,
> > > > > > > > > > > use
> > > > > > > > > > > > > java
> > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > > > > > > > > > >     A: Gwen kindly took
> that
> > > > > > > > (KAFKA-1927).
> > > > > > > > > > It's
> > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > speed
> > > > > > > > > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > > > > > > > > > >          there since this
> > > ticket
> > > > > > blocks
> > > > > > > > > other
> > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > changes.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute
> > > facility
> > > > > vs
> > > > > > > > client
> > > > > > > > > > > > > > maintaining
> > > > > > > > > > > > > > > > > > cluster
> > > > > > > > > > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > > > > > > > > > >     A: Jay has added pseudo
> > > code
> > > > to
> > > > > > > > > > KAFKA-1912
> > > > > > > > > > > -
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > > > > > > > > > >         easy to implement
> as
> > a
> > > > > > > > server-side
> > > > > > > > > > > > feature
> > > > > > > > > > > > > > > > > (comments
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > 3. Q: Controller field in
> > wire
> > > > > > > protocol.
> > > > > > > > > > > > > > > > > > > > > > > > >     A: This might be useful
> > for
> > > > > > > clients,
> > > > > > > > > add
> > > > > > > > > > > this
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic
> > creation
> > > > > from
> > > > > > > TMR.
> > > > > > > > > > > > > > > > > > > > > > > > >     A: I will add proposed
> by
> > > Jun
> > > > > > > > solution
> > > > > > > > > > > (using
> > > > > > > > > > > > > > > > clientId
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > 5. Q: Bumping new versions
> of
> > > TMR
> > > > > vs
> > > > > > > > > grabbing
> > > > > > > > > > > all
> > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > > > > > >     A: It was decided to
> try
> > to
> > > > > > gather
> > > > > > > > all
> > > > > > > > > > > > changes
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > > > > > > > > > >         In case of TMR it
> > worth
> > > > > > > checking:
> > > > > > > > > > > > > KAFKA-2020
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to
> > > > > > deserialize
> > > > > > > > > > user's
> > > > > > > > > > > > > input
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > CLI
> > > > > > > > > > > > > > > > > > tool.
> > > > > > > > > > > > > > > > > > > > > > > > >     A: Use jackson for
> that,
> > > > /tools
> > > > > > > > project
> > > > > > > > > > is
> > > > > > > > > > > a
> > > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > jar so
> > > > > > > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > 7.  Q:
> > VerifyReassingPartitions
> > > > vs
> > > > > > > > generic
> > > > > > > > > > > status
> > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > command.
> > > > > > > > > > > > > > > > > > > > > > > > >      A: For long-running
> > > requests
> > > > > > like
> > > > > > > > > > reassign
> > > > > > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > > > > > > > > > >          it makes sense to
> > > > > introduce
> > > > > > > it.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >  Please add, correct me if
> I
> > > > missed
> > > > > > > > > > something.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at
> 6:20
> > > PM,
> > > > > > Andrii
> > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > > > >
> andrii.biletskyi@stealth.ly>
> > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > You are right, I removed
> > > > > > > > ClusterMetadata
> > > > > > > > > > > > because
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > > > > > > > > > what we need in
> > > TopicMetadata.
> > > > > > Also,
> > > > > > > as
> > > > > > > > > Jay
> > > > > > > > > > > > > pointed
> > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > earlier,
> > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > > would like to have
> > > "orthogonal"
> > > > > > API,
> > > > > > > > but
> > > > > > > > > at
> > > > > > > > > > > the
> > > > > > > > > > > > > > same
> > > > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > > > > to be backward
> compatible.
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > But I like your idea and
> > even
> > > > > have
> > > > > > > some
> > > > > > > > > > other
> > > > > > > > > > > > > > > arguments
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > > > > > > > > > There is also
> > > > > DescribeTopicRequest
> > > > > > > > which
> > > > > > > > > > was
> > > > > > > > > > > > > > proposed
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > > > > > > > > > it returns topic configs,
> > > > > > partitions,
> > > > > > > > > > > > replication
> > > > > > > > > > > > > > > > factor
> > > > > > > > > > > > > > > > > > plus
> > > > > > > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > > > > > > > > > leader replica. The later
> > > part
> > > > is
> > > > > > > > really
> > > > > > > > > > > > already
> > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > > > > > > > > > So again we'll have to
> add
> > > > stuff
> > > > > to
> > > > > > > > TMR,
> > > > > > > > > > not
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > > > > newly added requests.
> > > However,
> > > > > this
> > > > > > > way
> > > > > > > > > > we'll
> > > > > > > > > > > > end
> > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > > > > > > > > > request which returns
> > cluster
> > > > > > > metadata,
> > > > > > > > > > topic
> > > > > > > > > > > > > > > > replication
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > > > > > > > plus partition
> replication
> > > > data.
> > > > > > > Seems
> > > > > > > > > > > logical
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > split
> > > > > > > > > > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > > > > > > > > > - ClusterMetadata
> (brokers
> > +
> > > > > > > > controller,
> > > > > > > > > > > maybe
> > > > > > > > > > > > > smth
> > > > > > > > > > > > > > > > else)
> > > > > > > > > > > > > > > > > > > > > > > > > > - TopicMetadata (topic
> > info +
> > > > > > > partition
> > > > > > > > > > > > details)
> > > > > > > > > > > > > > > > > > > > > > > > > > But since current TMR is
> > > > involved
> > > > > > in
> > > > > > > > lots
> > > > > > > > > > of
> > > > > > > > > > > > > places
> > > > > > > > > > > > > > > > > > (including
> > > > > > > > > > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > > > > > > > > as I understand) this
> might
> > > be
> > > > > very
> > > > > > > > > serious
> > > > > > > > > > > > > change
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > > > > > > > > > sense to stick with
> current
> > > > > > approach.
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at
> > 5:29
> > > > PM,
> > > > > > Joel
> > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> I may be missing some
> > > context
> > > > > but
> > > > > > > > > > hopefully
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > also be
> > > > > > > > > > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > > > > > > > > > >> today: I thought the
> > earlier
> > > > > > > proposal
> > > > > > > > > > where
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > > > > > > > > > >> ClusterMetadata request
> > was
> > > > > > clearer
> > > > > > > > and
> > > > > > > > > > > > > explicit.
> > > > > > > > > > > > > > > > During
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > > >> this thread I think the
> > > > > conclusion
> > > > > > > was
> > > > > > > > > > that
> > > > > > > > > > > > the
> > > > > > > > > > > > > > main
> > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> controller information
> and
> > > > that
> > > > > > can
> > > > > > > be
> > > > > > > > > > > rolled
> > > > > > > > > > > > > into
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > > > >> response but that seems
> a
> > > bit
> > > > > > > > irrelevant
> > > > > > > > > > to
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > metadata.
> > > > > > > > > > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > > > > > > > > > >> think the full
> broker-list
> > > is
> > > > > also
> > > > > > > > > > > irrelevant
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > > > >> it is already there and
> in
> > > > use.
> > > > > I
> > > > > > > > think
> > > > > > > > > > > there
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > still
> > > > > > > > > > > > > > > > > > room
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > > > > > >> explicit ClusterMetadata
> > > > request
> > > > > > > since
> > > > > > > > > > there
> > > > > > > > > > > > may
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > > > > > > >> cluster-level
> information
> > > that
> > > > > we
> > > > > > > may
> > > > > > > > > want
> > > > > > > > > > > to
> > > > > > > > > > > > > add
> > > > > > > > > > > > > > > over
> > > > > > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> have nothing to do with
> > > topic
> > > > > > > > metadata).
> > > > > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at
> > > > > 02:45:30PM
> > > > > > > > > +0200,
> > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > 101. Okay, if you say
> > that
> > > > > such
> > > > > > > use
> > > > > > > > > case
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > > important. I
> > > > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > > > > > > >> > using clientId for
> these
> > > > > > purposes
> > > > > > > is
> > > > > > > > > > fine
> > > > > > > > > > > -
> > > > > > > > > > > > if
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > > > > > > > > > >> > as part of all Wire
> > > protocol
> > > > > > > > messages,
> > > > > > > > > > why
> > > > > > > > > > > > not
> > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > that.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > I will update KIP-4
> page
> > > if
> > > > > > nobody
> > > > > > > > has
> > > > > > > > > > > other
> > > > > > > > > > > > > > ideas
> > > > > > > > > > > > > > > > > > (which
> > > > > > > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > > > > > > > > > >> > during the call
> today).
> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll
> update
> > > the
> > > > > KIP
> > > > > > > > > > > > accordingly.
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > we can
> > > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > fine-grained error
> codes
> > > if
> > > > > some
> > > > > > > > error
> > > > > > > > > > > code
> > > > > > > > > > > > > > > received
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > > > > > >> > won't give enough
> > context
> > > to
> > > > > > > return
> > > > > > > > a
> > > > > > > > > > > > > > descriptive
> > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > message
> > > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > Look forward to
> > discussing
> > > > all
> > > > > > > > > > outstanding
> > > > > > > > > > > > > > issues
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > detail
> > > > > > > > > > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015
> at
> > > > 10:59
> > > > > > PM,
> > > > > > > > Jun
> > > > > > > > > > Rao
> > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 101. There may be a
> > use
> > > > case
> > > > > > > where
> > > > > > > > > you
> > > > > > > > > > > > only
> > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > manually by admins.
> > > > > Currently,
> > > > > > > you
> > > > > > > > > can
> > > > > > > > > > > do
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > > > disabling
> > > > > > > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > creation and issue
> > topic
> > > > > > > creation
> > > > > > > > > from
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > TopicCommand.
> > > > > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > topic creation
> > > completely
> > > > on
> > > > > > the
> > > > > > > > > > broker
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > have a
> > > > > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > between topic
> creation
> > > > > > requests
> > > > > > > > from
> > > > > > > > > > the
> > > > > > > > > > > > > > regular
> > > > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > can't support manual
> > > topic
> > > > > > > > creation
> > > > > > > > > > any
> > > > > > > > > > > > > more.
> > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > way of
> distinguishing
> > > the
> > > > > > > clients
> > > > > > > > > > making
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > creation
> > > > > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > using clientId. For
> > > > example,
> > > > > > the
> > > > > > > > > admin
> > > > > > > > > > > > tool
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > set
> > > > > > > > > > > > > > > > > > it to
> > > > > > > > > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin and the broker
> > can
> > > > > treat
> > > > > > > > that
> > > > > > > > > > > > clientId
> > > > > > > > > > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Also, there is a
> > related
> > > > > > > > discussion
> > > > > > > > > in
> > > > > > > > > > > > > > > KAFKA-2020.
> > > > > > > > > > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > following in
> > > > > > > > TopicMetadataResponse:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 1. If leader is not
> > > > > available,
> > > > > > > we
> > > > > > > > > set
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 2. If a non-leader
> > > replica
> > > > > is
> > > > > > > not
> > > > > > > > > > > > available,
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > take
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > the assigned replica
> > > list
> > > > > and
> > > > > > > isr
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > > response.
> > > > > > > > > > > > > > > > > As
> > > > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > doing that, we set
> the
> > > > > > partition
> > > > > > > > > level
> > > > > > > > > > > > error
> > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > This has a few
> > problems.
> > > > > > First,
> > > > > > > > > > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > an error, at least
> for
> > > the
> > > > > > > normal
> > > > > > > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to find out the
> > leader.
> > > > > > Second,
> > > > > > > it
> > > > > > > > > can
> > > > > > > > > > > > > happen
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > both
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > another replica are
> > not
> > > > > > > available
> > > > > > > > at
> > > > > > > > > > the
> > > > > > > > > > > > > same
> > > > > > > > > > > > > > > > time.
> > > > > > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to indicate both.
> > Third,
> > > > > even
> > > > > > > if a
> > > > > > > > > > > replica
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > available,
> > > > > > > > > > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > useful to return its
> > > > replica
> > > > > > id
> > > > > > > > > since
> > > > > > > > > > > some
> > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > (e.g.
> > > > > > > > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > still make use of
> it.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > One way to address
> > this
> > > > > issue
> > > > > > is
> > > > > > > > to
> > > > > > > > > > > always
> > > > > > > > > > > > > > > return
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > leader, assigned
> > > replicas,
> > > > > and
> > > > > > > isr
> > > > > > > > > > > > > regardless
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > broker is live or
> not.
> > > > Since
> > > > > > we
> > > > > > > > also
> > > > > > > > > > > > return
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > list
> > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > client can figure
> out
> > > > > whether
> > > > > > a
> > > > > > > > > leader
> > > > > > > > > > > or
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > accordingly. This
> way,
> > > we
> > > > > > don't
> > > > > > > > need
> > > > > > > > > > to
> > > > > > > > > > > > set
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > when the leader or a
> > > > replica
> > > > > > is
> > > > > > > > not
> > > > > > > > > > > > > available.
> > > > > > > > > > > > > > > > This
> > > > > > > > > > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > protocol, but does
> > > change
> > > > > the
> > > > > > > > > > semantics.
> > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > of
> > TopicMetadataRequest
> > > > > here,
> > > > > > we
> > > > > > > > can
> > > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > > piggyback
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > 102.1 For those
> types
> > of
> > > > > > errors
> > > > > > > > due
> > > > > > > > > to
> > > > > > > > > > > > > invalid
> > > > > > > > > > > > > > > > > input,
> > > > > > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > guard it at
> parameter
> > > > > > validation
> > > > > > > > > time
> > > > > > > > > > > and
> > > > > > > > > > > > > > throw
> > > > > > > > > > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > without even sending
> > the
> > > > > > request
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > > broker?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015
> > at
> > > > > 10:37
> > > > > > > AM,
> > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > andrii.biletskyi@stealth.ly
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Answering your
> > > > questions:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 101. If I
> understand
> > > you
> > > > > > > > > correctly,
> > > > > > > > > > > you
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > saying
> > > > > > > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will be ported to
> > > > TMR_V1)
> > > > > > > won't
> > > > > > > > be
> > > > > > > > > > > able
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > automatically
> > > > > > > > > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > unconditionally
> > remove
> > > > > topic
> > > > > > > > > > creation
> > > > > > > > > > > > from
> > > > > > > > > > > > > > > > there).
> > > > > > > > > > > > > > > > > > But
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Ok, about your
> > > proposal:
> > > > > I'm
> > > > > > > > not a
> > > > > > > > > > big
> > > > > > > > > > > > fan
> > > > > > > > > > > > > > > too,
> > > > > > > > > > > > > > > > > > when it
> > > > > > > > > > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > clients directly
> in
> > > > > protocol
> > > > > > > > > schema.
> > > > > > > > > > > And
> > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > auto.create.topics.enable
> > > > > > is a
> > > > > > > > > > server
> > > > > > > > > > > > side
> > > > > > > > > > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in future
> versions,
> > > add
> > > > > this
> > > > > > > > > setting
> > > > > > > > > > > to
> > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > based
> > > > > > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > UnknownTopic
> create
> > > > topic
> > > > > > > > > explicitly
> > > > > > > > > > > by
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes.
> It's
> > > > > because
> > > > > > > we
> > > > > > > > > want
> > > > > > > > > > > to
> > > > > > > > > > > > > > > support
> > > > > > > > > > > > > > > > > > batching
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > want to give
> > > descriptive
> > > > > > error
> > > > > > > > > > > messages
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > clients.
> > > > > > > > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to construct such
> > > > messages
> > > > > > > (e.g.
> > > > > > > > > > > > > AdminClient
> > > > > > > > > > > > > > > > layer
> > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> InvalidArgumentsCode
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > means two cases:
> > > either
> > > > > > > invalid
> > > > > > > > > > > number -
> > > > > > > > > > > > > > e.g.
> > > > > > > > > > > > > > > > -1;
> > > > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > partitions
> argument
> > > > > wasn't)
> > > > > > -
> > > > > > > I
> > > > > > > > > > > wrapped
> > > > > > > > > > > > > > > > responses
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > other ideas, this
> > was
> > > > just
> > > > > > > > initial
> > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I
> agree.
> > > > I'll
> > > > > > > change
> > > > > > > > > > that
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13,
> 2015
> > > at
> > > > > 7:16
> > > > > > > PM,
> > > > > > > > > Jun
> > > > > > > > > > > > Rao <
> > > > > > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 101. That's
> what I
> > > was
> > > > > > > > thinking
> > > > > > > > > > too,
> > > > > > > > > > > > but
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > not be
> > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we can let it
> not
> > > > > trigger
> > > > > > > auto
> > > > > > > > > > topic
> > > > > > > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > Then,
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > > > > > > > > > UnknownTopicException,
> > > > > > > > > > > > it
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> createTopicRequest
> > > for
> > > > > > auto
> > > > > > > > > topic
> > > > > > > > > > > > > > creation.
> > > > > > > > > > > > > > > On
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > never issue
> > > > > > > > createTopicRequest.
> > > > > > > > > > This
> > > > > > > > > > > > > works
> > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > enabled on the
> > > broker
> > > > > > side.
> > > > > > > > > > > However, I
> > > > > > > > > > > > > am
> > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > sure how
> > > > > > > > > > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > when auto topic
> > > > creation
> > > > > > is
> > > > > > > > > > disabled
> > > > > > > > > > > > on
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > > side.
> > > > > > > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > want to have a
> way
> > > to
> > > > > > > manually
> > > > > > > > > > > create
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > topic,
> > > > > > > > > > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > commands.
> However,
> > > > then
> > > > > we
> > > > > > > > need
> > > > > > > > > a
> > > > > > > > > > > way
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > issued from the
> > > > producer
> > > > > > > > clients
> > > > > > > > > > and
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > tools.
> > > > > > > > > > > > > > > > > > > > > May
> > > > > > > > > > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > new field in
> > > > > > > > createTopicRequest
> > > > > > > > > > and
> > > > > > > > > > > > set
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > differently
> > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > client and the
> > admin
> > > > > > client.
> > > > > > > > > > > However,
> > > > > > > > > > > > I
> > > > > > > > > > > > > am
> > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > sure if
> > > > > > > > > > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 2. Yes,
> > refactoring
> > > > > > existing
> > > > > > > > > > > requests
> > > > > > > > > > > > > is a
> > > > > > > > > > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > posted some
> > comments
> > > > in
> > > > > > > > > > KAFKA-1927.
> > > > > > > > > > > We
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > first, before
> > adding
> > > > the
> > > > > > new
> > > > > > > > > logic
> > > > > > > > > > > in
> > > > > > > > > > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102. About the
> > > > > > AdminClient:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a
> bit
> > > > weird
> > > > > > that
> > > > > > > > we
> > > > > > > > > > > return
> > > > > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we should either
> > > > return
> > > > > > > error
> > > > > > > > > code
> > > > > > > > > > > or
> > > > > > > > > > > > > > throw
> > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.2. We
> probably
> > > > > > shouldn't
> > > > > > > > > > > > explicitly
> > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Not every
> request
> > > > > > evolution
> > > > > > > > > > requires
> > > > > > > > > > > > an
> > > > > > > > > > > > > > api
> > > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13,
> > 2015
> > > > at
> > > > > > 4:08
> > > > > > > > AM,
> > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > andrii.biletskyi@stealth.ly
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks for you
> > > > > comments.
> > > > > > > > > Answers
> > > > > > > > > > > > > inline:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 100. There
> are a
> > > few
> > > > > > > fields
> > > > > > > > > such
> > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > > > > > PartitionsSerialized
> > > > > > > > > that
> > > > > > > > > > > are
> > > > > > > > > > > > > > > > > represented
> > > > > > > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
> > > > structures
> > > > > > in
> > > > > > > > > json.
> > > > > > > > > > > > Could
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
> > > > definition
> > > > > as
> > > > > > > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, now with
> > > Admin
> > > > > > Client
> > > > > > > > > this
> > > > > > > > > > > > looks
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > > weird.
> > > > > > > > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > ReassignPartitionCommand
> > > > > > > > > accepts
> > > > > > > > > > > > input
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > json,
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > interfaces
> > > > unchanged,
> > > > > > > where
> > > > > > > > > > > > possible.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > If we port it
> to
> > > > > > > > deserialized
> > > > > > > > > > > > format,
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > CLI
> > > > > > > > > > > > > > > > > > (/tools
> > > > > > > > > > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json library
> > since
> > > > > > /tools
> > > > > > > is
> > > > > > > > > > > written
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > java
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > provided by a
> > > user.
> > > > > Can
> > > > > > we
> > > > > > > > > > quickly
> > > > > > > > > > > > > agree
> > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > what
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > (Jackson,
> GSON,
> > > > > > whatever)?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> > > > > > > > TopicMetadataRequest
> > > > > > > > > > v1
> > > > > > > > > > > > > still
> > > > > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a
> bit
> > > > weird
> > > > > > now
> > > > > > > > that
> > > > > > > > > > we
> > > > > > > > > > > > > have a
> > > > > > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought
> > > about
> > > > > how
> > > > > > > the
> > > > > > > > > new
> > > > > > > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be
> > used
> > > in
> > > > > the
> > > > > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For
> > > > example,
> > > > > > > > ideally,
> > > > > > > > > > we
> > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to
> > > > trigger
> > > > > > auto
> > > > > > > > > topic
> > > > > > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > I agree, this
> > > > strange
> > > > > > > logic
> > > > > > > > > > should
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > fixed.
> > > > > > > > > > > > > > > > > > I'm not
> > > > > > > > > > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > correct me if
> > I'm
> > > > > wrong,
> > > > > > > but
> > > > > > > > > it
> > > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > look
> > > > > > > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > leverage
> > > AdminClient
> > > > > for
> > > > > > > > that
> > > > > > > > > in
> > > > > > > > > > > > > > Producer
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > creation from
> > the
> > > > > > > > > > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay
> > > meant
> > > > > > > getting
> > > > > > > > > rid
> > > > > > > > > > > of
> > > > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > > > > > > HeartbeatRequestAndHeader
> > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >>
> > HeartbeatResponseAndHeader.
> > > We
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a
> stop-gap
> > > > thing
> > > > > > when
> > > > > > > > > > adding
> > > > > > > > > > > > the
> > > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the
> > > long
> > > > > term
> > > > > > > > plan
> > > > > > > > > is
> > > > > > > > > > > to
> > > > > > > > > > > > > get
> > > > > > > > > > > > > > > rid
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> > > > > request/response
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > client.
> > > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant
> > > number
> > > > > of
> > > > > > > new
> > > > > > > > > > > > requests,
> > > > > > > > > > > > > > > > perhaps
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the
> > > > > existing
> > > > > > > > scala
> > > > > > > > > > > > requests
> > > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks
> like
> > I
> > > > > > > > > misunderstood
> > > > > > > > > > > the
> > > > > > > > > > > > > > point
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > rework that.
> The
> > > > only
> > > > > > > thing
> > > > > > > > is
> > > > > > > > > > > that
> > > > > > > > > > > > I
> > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > see
> > > > > > > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > least one
> > existing
> > > > > > > protocol
> > > > > > > > > > > message.
> > > > > > > > > > > > > > Thus,
> > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are going to
> do
> > > it.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Re porting all
> > > > > existing
> > > > > > > > RQ/RP
> > > > > > > > > in
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > patch.
> > > > > > > > > > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > an
> *obligatory*
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > requirement to
> > > have
> > > > > > Admin
> > > > > > > > KIP
> > > > > > > > > > > done,
> > > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > > afraid
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > blocker for
> us.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > There are 13
> > > > protocol
> > > > > > > > messages
> > > > > > > > > > and
> > > > > > > > > > > > all
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > tests but
> quite
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > intensive
> manual
> > > > > > testing,
> > > > > > > > no?
> > > > > > > > > > I'm
> > > > > > > > > > > > > afraid
> > > > > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > > > > not the
> > > > > > > > > > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > pretty much
> all
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka core
> > > internals
> > > > > :).
> > > > > > > Let
> > > > > > > > > me
> > > > > > > > > > > know
> > > > > > > > > > > > > > your
> > > > > > > > > > > > > > > > > > thoughts
> > > > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > follow-up this
> > > > issue (
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >
> https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Andrii
> Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar
> 13,
> > > 2015
> > > > > at
> > > > > > > 6:40
> > > > > > > > > AM,
> > > > > > > > > > > Jun
> > > > > > > > > > > > > > Rao <
> > > > > > > > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > A few more
> > > > comments.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 100. There
> > are a
> > > > few
> > > > > > > > fields
> > > > > > > > > > such
> > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > > > > > PartitionsSerialized
> > > > > > > > > that
> > > > > > > > > > > are
> > > > > > > > > > > > > > > > > represented
> > > > > > > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
> > > > structures
> > > > > > in
> > > > > > > > > json.
> > > > > > > > > > > > Could
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
> > > > definition
> > > > > as
> > > > > > > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> > > > > > > > > TopicMetadataRequest
> > > > > > > > > > > v1
> > > > > > > > > > > > > > still
> > > > > > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a
> bit
> > > > weird
> > > > > > now
> > > > > > > > that
> > > > > > > > > > we
> > > > > > > > > > > > > have a
> > > > > > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought
> > > about
> > > > > how
> > > > > > > the
> > > > > > > > > new
> > > > > > > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be
> > used
> > > in
> > > > > the
> > > > > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For
> > > > example,
> > > > > > > > ideally,
> > > > > > > > > > we
> > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to
> > > > trigger
> > > > > > auto
> > > > > > > > > topic
> > > > > > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 2. I think
> Jay
> > > > meant
> > > > > > > > getting
> > > > > > > > > > rid
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > > > > > > HeartbeatRequestAndHeader
> > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >>
> > HeartbeatResponseAndHeader.
> > > We
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a
> stop-gap
> > > > thing
> > > > > > when
> > > > > > > > > > adding
> > > > > > > > > > > > the
> > > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the
> > > long
> > > > > term
> > > > > > > > plan
> > > > > > > > > is
> > > > > > > > > > > to
> > > > > > > > > > > > > get
> > > > > > > > > > > > > > > rid
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> > > > > request/response
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > client.
> > > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant
> > > number
> > > > > of
> > > > > > > new
> > > > > > > > > > > > requests,
> > > > > > > > > > > > > > > > perhaps
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the
> > > > > existing
> > > > > > > > scala
> > > > > > > > > > > > requests
> > > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar
> > 12,
> > > > 2015
> > > > > > at
> > > > > > > > 3:37
> > > > > > > > > > PM,
> > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > andrii.biletskyi@stealth.ly
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > As said
> > above
> > > -
> > > > I
> > > > > > list
> > > > > > > > > again
> > > > > > > > > > > all
> > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can see
> > what's
> > > > > left
> > > > > > > and
> > > > > > > > > > > finalize
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > pending
> > > > > > > > > > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments
> > from
> > > > Jay:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is
> > > much
> > > > > > needed
> > > > > > > > > > > > > > functionality,
> > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > really
> think
> > > > these
> > > > > > > > > protocols
> > > > > > > > > > > > > > through.
> > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > of well
> > > > > thought-out,
> > > > > > > > > > > orthoganol
> > > > > > > > > > > > > > apis.
> > > > > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > important
> to
> > > > think
> > > > > > > > through
> > > > > > > > > > the
> > > > > > > > > > > > end
> > > > > > > > > > > > > > > state
> > > > > > > > > > > > > > > > > > even if
> > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > won't
> > > implement
> > > > in
> > > > > > the
> > > > > > > > > first
> > > > > > > > > > > > > phase.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> > Definitely
> > > > > behind
> > > > > > > > this.
> > > > > > > > > > > Would
> > > > > > > > > > > > > > > > > appreciate
> > > > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > how this
> can
> > > be
> > > > > > > > improved.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's
> > > please
> > > > > > please
> > > > > > > > > > please
> > > > > > > > > > > > wait
> > > > > > > > > > > > > > > until
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to the new
> > > java
> > > > > > > protocol
> > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > objects
> that
> > > is
> > > > > just
> > > > > > > > > > > generating
> > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > work
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > have to
> do.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed
> in
> > > the
> > > > > > latest
> > > > > > > > > > patch -
> > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This
> > > proposal
> > > > > > > > > introduces
> > > > > > > > > > a
> > > > > > > > > > > > new
> > > > > > > > > > > > > > type
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> inconsistent
> > > > with
> > > > > > > > > everything
> > > > > > > > > > > > else
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > other
> marker
> > > > > value.
> > > > > > > You
> > > > > > > > > > could
> > > > > > > > > > > > > argue
> > > > > > > > > > > > > > > > either
> > > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > for
> > > consistency.
> > > > > For
> > > > > > > > > clients
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > implemented
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than our
> > scala
> > > > > code
> > > > > > > > these
> > > > > > > > > > > basic
> > > > > > > > > > > > > > > > primitives
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed
> in
> > > the
> > > > > > latest
> > > > > > > > > > patch -
> > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > > MaybeOf
> > > > > > > > > > > > > > > > > > > > > type
> > > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> accordingly.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 4.
> > > > > ClusterMetadata:
> > > > > > > This
> > > > > > > > > > seems
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers,
> > > topics,
> > > > > and
> > > > > > > > > > > > partitions. I
> > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > ClusterMetadataRequest
> > > > > > > > (or
> > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> controller.
> > Or
> > > > are
> > > > > > > there
> > > > > > > > > > other
> > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I
> agree.
> > > > > Updated
> > > > > > > the
> > > > > > > > > KIP.
> > > > > > > > > > > > Let's
> > > > > > > > > > > > > > > > extends
> > > > > > > > > > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > include
> > > > > controller.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We
> have a
> > > > > > tendency
> > > > > > > to
> > > > > > > > > try
> > > > > > > > > > > to
> > > > > > > > > > > > > > make a
> > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > particular
> > > > nodes.
> > > > > > This
> > > > > > > > > adds
> > > > > > > > > > a
> > > > > > > > > > > > lot
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > burden
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > sounds
> easy
> > > but
> > > > > each
> > > > > > > > > > discovery
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > fail
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > many
> > > > > > > > > > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > full state
> > > > machine
> > > > > > to
> > > > > > > do
> > > > > > > > > > > > right). I
> > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > commands
> and
> > > > > ideally
> > > > > > > as
> > > > > > > > > many
> > > > > > > > > > > of
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers
> and
> > > just
> > > > > > > > redirect
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > there
> would
> > > be a
> > > > > > > general
> > > > > > > > > way
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > encapsulate
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a
> > very
> > > > > > > > interesting
> > > > > > > > > > > idea,
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > seems
> > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > feature
> > (like
> > > > > > > > performance
> > > > > > > > > > > > > > > > considerations,
> > > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I believe
> > this
> > > > > > > shouldn't
> > > > > > > > > be
> > > > > > > > > > a
> > > > > > > > > > > > > > blocker.
> > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point it
> > won't
> > > > > > affect
> > > > > > > > > Admin
> > > > > > > > > > > > > changes
> > > > > > > > > > > > > > -
> > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > least
> > > > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
> > > > required.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We
> should
> > > > > > probably
> > > > > > > > > > > normalize
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > key
> > > > > > > > > > > > > > > > > > value
> > > > > > > > > > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
> > > embedding a
> > > > > new
> > > > > > > > > > > formatting.
> > > > > > > > > > > > > So
> > > > > > > > > > > > > > > two
> > > > > > > > > > > > > > > > > > strings
> > > > > > > > > > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > internal
> > > equals
> > > > > > sign.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed
> in
> > > the
> > > > > > latest
> > > > > > > > > > patch -
> > > > > > > > > > > > > > > > normalized
> > > > > > > > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> accordingly.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the
> > > > > > > postcondition
> > > > > > > > of
> > > > > > > > > > > these
> > > > > > > > > > > > > > APIs
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> command
> > > has
> > > > > been
> > > > > > > > > > > completed?
> > > > > > > > > > > > It
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > been
> > completed
> > > > so
> > > > > > you
> > > > > > > > know
> > > > > > > > > > > that
> > > > > > > > > > > > if
> > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > > create a
> > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it you
> won't
> > > get
> > > > > an
> > > > > > > > > > exception
> > > > > > > > > > > > > about
> > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > being
> > > > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: For
> long
> > > > > running
> > > > > > > > > requests
> > > > > > > > > > > > (like
> > > > > > > > > > > > > > > > > reassign
> > > > > > > > > > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > condition
> is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
> has
> > > > begun
> > > > > -
> > > > > > so
> > > > > > > > we
> > > > > > > > > > > don't
> > > > > > > > > > > > > > block
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > client.
> > > > > > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
> > > commands,
> > > > > this
> > > > > > > > will
> > > > > > > > > be
> > > > > > > > > > > > > > > refactored
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> immediately,
> > > > since
> > > > > > the
> > > > > > > > > > > > Controller
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > serve
> > > > > > > > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up
> > > > ticket
> > > > > > > > > > KAFKA-1777).
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 8.
> Describe
> > > > topic
> > > > > > and
> > > > > > > > list
> > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request.
> Is
> > > > there
> > > > > a
> > > > > > > > reason
> > > > > > > > > > to
> > > > > > > > > > > > give
> > > > > > > > > > > > > > > back
> > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like if we
> > > just
> > > > > make
> > > > > > > the
> > > > > > > > > > > > > > > post-condition
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic is
> > > deleted
> > > > > > that
> > > > > > > > will
> > > > > > > > > > get
> > > > > > > > > > > > rid
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be much
> more
> > > > > > > intuitive.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed
> in
> > > the
> > > > > > latest
> > > > > > > > > > patch -
> > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > ListTopicsRequest.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should
> we
> > > > > > consider
> > > > > > > > > > batching
> > > > > > > > > > > > > these
> > > > > > > > > > > > > > > > > > requests?
> > > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > allow
> > multiple
> > > > > > > > operations
> > > > > > > > > to
> > > > > > > > > > > be
> > > > > > > > > > > > > > > batched.
> > > > > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we will
> get
> > a
> > > > lot
> > > > > of
> > > > > > > > code
> > > > > > > > > > that
> > > > > > > > > > > > > does
> > > > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > for(topic:
> > > > > > > > > > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this code
> > will
> > > > > work
> > > > > > > > great
> > > > > > > > > > when
> > > > > > > > > > > > you
> > > > > > > > > > > > > > > test
> > > > > > > > > > > > > > > > > on 5
> > > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > you have
> > 50k.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated
> > the
> > > > > KIP -
> > > > > > > > > please
> > > > > > > > > > > > check
> > > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I
> think
> > we
> > > > > > should
> > > > > > > > also
> > > > > > > > > > > > discuss
> > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > client api
> > for
> > > > > these
> > > > > > > > > > > operations.
> > > > > > > > > > > > > > > > Currently
> > > > > > > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > is totally
> > > > > sketchy.
> > > > > > I
> > > > > > > > > think
> > > > > > > > > > we
> > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > that
> exposes
> > > > > > > > > administrative
> > > > > > > > > > > > > > > > functionality.
> > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > properly
> > test
> > > > the
> > > > > > new
> > > > > > > > > apis,
> > > > > > > > > > I
> > > > > > > > > > > > > > suspect.
> > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated
> > the
> > > > > KIP -
> > > > > > > > > please
> > > > > > > > > > > > check
> > > > > > > > > > > > > > > "Admin
> > > > > > > > > > > > > > > > > > Client"
> > > > > > > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > API
> > proposal.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The
> > other
> > > > > > > > information
> > > > > > > > > > that
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> information
> > > > about
> > > > > > > > > > > > partitions--how
> > > > > > > > > > > > > > much
> > > > > > > > > > > > > > > > > data
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> segment
> > > > > offsets,
> > > > > > > > what
> > > > > > > > > is
> > > > > > > > > > > the
> > > > > > > > > > > > > > > log-end
> > > > > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > compaction
> > > > > > point,
> > > > > > > > > etc. I
> > > > > > > > > > > > think
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > done
> > > > > > > > > > > > > > > > > > > > right
> > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > successor
> to
> > > the
> > > > > > very
> > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > OffsetRequest
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I
> removed
> > > > > > > > > > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> should
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be
> resolved
> > > in a
> > > > > > > > separate
> > > > > > > > > > KIP
> > > > > > > > > > > /
> > > > > > > > > > > > > jira
> > > > > > > > > > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 12.
> > Generally
> > > we
> > > > > can
> > > > > > > do
> > > > > > > > > good
> > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > handling
> > > > > > > > > > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> server-side
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > messages.
> > I.e.
> > > > > > > generally
> > > > > > > > > the
> > > > > > > > > > > > > client
> > > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > context
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > an error
> > that
> > > > the
> > > > > > > topic
> > > > > > > > > > > doesn't
> > > > > > > > > > > > > > exist
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > say
> > > > > > > > > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
> "error
> > > code
> > > > > 14"
> > > > > > > (or
> > > > > > > > > > > > > whatever).
> > > > > > > > > > > > > > > > Maybe
> > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this is
> > hard?
> > > If
> > > > > we
> > > > > > > want
> > > > > > > > > to
> > > > > > > > > > > add
> > > > > > > > > > > > > > > > > server-side
> > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > need to do
> > > this
> > > > > in a
> > > > > > > > > > > consistent
> > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > across
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated
> > the
> > > > > KIP -
> > > > > > > > > please
> > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > "Protocol
> > > > > > > > > > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > comprehensive,
> > > > > > > > > fine-grained
> > > > > > > > > > > list
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments
> > from
> > > > > > > Guozhang:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 13.
> Describe
> > > > topic
> > > > > > > > > request:
> > > > > > > > > > it
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > great to
> > > > > > > > > > > > > > > > > > > > > go
> > > > > > > > > > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name
> > > regex
> > > > > for
> > > > > > > > this
> > > > > > > > > > > > request.
> > > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > > example,
> > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the topic
> > > > command
> > > > > is
> > > > > > > to
> > > > > > > > > list
> > > > > > > > > > > all
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name
> > > regex
> > > > > > then
> > > > > > > we
> > > > > > > > > > have
> > > > > > > > > > > to
> > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> description
> > > info
> > > > > and
> > > > > > > > then
> > > > > > > > > > > filter
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > burden on
> > ZK.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config
> > > K-Vs
> > > > in
> > > > > > > > create
> > > > > > > > > > > topic:
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > related
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > maybe we
> can
> > > add
> > > > > > > another
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > K-V
> > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > with
> config
> > > K-V
> > > > in
> > > > > > > > create
> > > > > > > > > > > topic
> > > > > > > > > > > > > like
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > field can
> be
> > > > quite
> > > > > > > > useful
> > > > > > > > > in
> > > > > > > > > > > > > storing
> > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > who issue
> > the
> > > > > create
> > > > > > > > > > command,
> > > > > > > > > > > > etc,
> > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> multi-tenant
> > > > > > setting.
> > > > > > > > Then
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > on regex
> of
> > > the
> > > > > > > metadata
> > > > > > > > > > > field.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: As
> > > discussed
> > > > it
> > > > > > is
> > > > > > > > very
> > > > > > > > > > > > > > interesting
> > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we have
> some
> > > > basic
> > > > > > > > > > > functionality
> > > > > > > > > > > > > > > there.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today
> > all
> > > > the
> > > > > > > admin
> > > > > > > > > > > > operations
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > async in
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > return
> once
> > it
> > > > is
> > > > > > > > written
> > > > > > > > > in
> > > > > > > > > > > ZK,
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > is why
> > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > > > > > > > > > testUtil.waitForTopicCreated()
> > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request,
> > etc.
> > > > With
> > > > > > > admin
> > > > > > > > > > > > requests
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> synchronous
> > > > > > requests;
> > > > > > > > when
> > > > > > > > > > it
> > > > > > > > > > > is
> > > > > > > > > > > > > > > turned
> > > > > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > until the
> > > > request
> > > > > > has
> > > > > > > > been
> > > > > > > > > > > > > > completed.
> > > > > > > > > > > > > > > > And
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > "token"
> > field
> > > in
> > > > > the
> > > > > > > > > > response,
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> verification
> > > > > > request"
> > > > > > > > with
> > > > > > > > > > the
> > > > > > > > > > > > > given
> > > > > > > > > > > > > > > > token
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > has been
> > > > > completed.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I see
> > your
> > > > > point.
> > > > > > > My
> > > > > > > > > idea
> > > > > > > > > > > was
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long
> running
> > > > > > request,
> > > > > > > > > where
> > > > > > > > > > > > > needed.
> > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > do it
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > concern is
> > > that
> > > > > > > > > introducing
> > > > > > > > > > a
> > > > > > > > > > > > > token
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > again
> > > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to do
> > similar
> > > > > thing
> > > > > > > > > > > introducing
> > > > > > > > > > > > > > single
> > > > > > > > > > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > but
> rejected
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this idea
> > > > because
> > > > > we
> > > > > > > > > wanted
> > > > > > > > > > to
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > choice
> > > between:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > a) have
> > fixed
> > > > > schema
> > > > > > > but
> > > > > > > > > > > > introduce
> > > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > > time new
> > > > > > > > > > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> long-running
> > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > b) use one
> > > > request
> > > > > > for
> > > > > > > > > > > > > verification
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > generalize
> > > > > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine
> > with
> > > > > > whatever
> > > > > > > > > > > decision
> > > > > > > > > > > > > > > > community
> > > > > > > > > > > > > > > > > > come
> > > > > > > > > > > > > > > > > > > > to.
> > > > > > > > > > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comment
> from
> > > > Gwen:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 16.
> > > Specifically
> > > > > for
> > > > > > > > > > > ownership,
> > > > > > > > > > > > I
> > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like you
> are
> > > > > > > describing
> > > > > > > > > ACL)
> > > > > > > > > > > via
> > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I remember
> > > > KIP-11
> > > > > > > > > described
> > > > > > > > > > > > this,
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay,
> no
> > > > > problem.
> > > > > > > Not
> > > > > > > > > > sure
> > > > > > > > > > > > > though
> > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
> > > > committed
> > > > > > > first
> > > > > > > > > and
> > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I
> > > added
> > > > > this
> > > > > > > > note
> > > > > > > > > to
> > > > > > > > > > > > "Open
> > > > > > > > > > > > > > > > > > Questions"
> > > > > > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii
> > > Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri,
> Mar
> > > 13,
> > > > > 2015
> > > > > > > at
> > > > > > > > > > 12:34
> > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > andrii.biletskyi@stealth.ly
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Today I
> > > > uploaded
> > > > > > the
> > > > > > > > > patch
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > covers
> > > > > > > > > > > > > > > > > > some of
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> removed
> > > > > MaybeOf
> > > > > > > > > optional
> > > > > > > > > > > > type
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> switched
> > > to
> > > > > java
> > > > > > > > > > protocol
> > > > > > > > > > > > > > > > definitions
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> > simplified
> > > > > > > messages
> > > > > > > > > > > > > (normalized
> > > > > > > > > > > > > > > > > configs,
> > > > > > > > > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> deletion)
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I also
> > > updated
> > > > > the
> > > > > > > > KIP-4
> > > > > > > > > > > with
> > > > > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > proposal
> > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > pending
> > > items:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch
> > > Admin
> > > > > > > > Operations
> > > > > > > > > > ->
> > > > > > > > > > > > > > updated
> > > > > > > > > > > > > > > > Wire
> > > > > > > > > > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove
> > > > > > > > ClusterMetadata
> > > > > > > > > > ->
> > > > > > > > > > > > > > changed
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > extend
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin
> > > Client
> > > > > ->
> > > > > > > > > updated
> > > > > > > > > > my
> > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > > proposal
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Error
> > > codes
> > > > ->
> > > > > > > > > proposed
> > > > > > > > > > > > > > > fine-grained
> > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > AdminRequestFailed
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I will
> > also
> > > > > send a
> > > > > > > > > > separate
> > > > > > > > > > > > > email
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > cover all
> > > > > > > > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii
> > > > Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu,
> > Mar
> > > > 12,
> > > > > > 2015
> > > > > > > > at
> > > > > > > > > > 9:26
> > > > > > > > > > > > PM,
> > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Found
> > > KIP-11
> > > > (
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> It
> > actually
> > > > > > > specifies
> > > > > > > > > > > changes
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> both
> KIPs
> > > are
> > > > > > > > > consistent
> > > > > > > > > > in
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > regard
> > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu,
> > Mar
> > > > 12,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > 12:21
> > > > > > > > > > > > > PM,
> > > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > Specifically
> > > > > > for
> > > > > > > > > > > > ownership, I
> > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > like
> > you
> > > > are
> > > > > > > > > describing
> > > > > > > > > > > > ACL)
> > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I
> > > remember
> > > > > > KIP-11
> > > > > > > > > > > described
> > > > > > > > > > > > > > this,
> > > > > > > > > > > > > > > > > but I
> > > > > > > > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > Regardless, I
> > > > > > > think
> > > > > > > > > > KIP-4
> > > > > > > > > > > > > > focuses
> > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> exists
> > > from
> > > > > > Kafka
> > > > > > > > > > > brokers,
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> should
> > > > exist
> > > > > > but
> > > > > > > > > > doesn't
> > > > > > > > > > > > yet?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On
> Thu,
> > > Mar
> > > > > 12,
> > > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > > 6:37
> > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> Folks,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just
> > > want
> > > > to
> > > > > > > > > > elaborate a
> > > > > > > > > > > > bit
> > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> batching
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > describe-topic
> > > > > > > > based
> > > > > > > > > > on
> > > > > > > > > > > > > > config /
> > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on
> > > > > KAFKA-1694.
> > > > > > > The
> > > > > > > > > > main
> > > > > > > > > > > > > > > motivation
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > management
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > mechanisms,
> > > > > > > which
> > > > > > > > I
> > > > > > > > > > > think
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > architecture:
> > > > > > > > today
> > > > > > > > > > > anyone
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> cluster,
> > > but
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> there
> > is
> > > > no
> > > > > > > > concept
> > > > > > > > > or
> > > > > > > > > > > > > > > "ownership"
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> different
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> users.
> > > For
> > > > > > > > example,
> > > > > > > > > at
> > > > > > > > > > > > > > LinkedIn
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some
> > > > casual
> > > > > > > topic
> > > > > > > > > name
> > > > > > > > > > > > > prefix,
> > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > is a
> > > > > > > > > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we
> > scale
> > > > our
> > > > > > > > > > customers.
> > > > > > > > > > > It
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > Describe
> > > > all
> > > > > > > > topics
> > > > > > > > > > that
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > created
> > > > > > > > > > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > Describe
> > > > all
> > > > > > > > topics
> > > > > > > > > > > whose
> > > > > > > > > > > > > > > > retention
> > > > > > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > Describe
> > > > all
> > > > > > > > topics
> > > > > > > > > > > whose
> > > > > > > > > > > > > > > writable
> > > > > > > > > > > > > > > > > > group
> > > > > > > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> related
> > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > authorization),
> > > > > > > > > etc..
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One
> > > > possible
> > > > > > way
> > > > > > > > to
> > > > > > > > > > > > achieve
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > create-topic
> > > > > > > > > request,
> > > > > > > > > > > > whose
> > > > > > > > > > > > > > > value
> > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> topic;
> > > > then
> > > > > > > > > > > > describe-topics
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > choose to
> > > > > > > > > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> regex,
> > > 2)
> > > > > > config
> > > > > > > > K-V
> > > > > > > > > > > > > matching,
> > > > > > > > > > > > > > > 3)
> > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > Thoughts?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > Guozhang
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On
> > Thu,
> > > > Mar
> > > > > 5,
> > > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > > 4:37
> > > > > > > > > > > > > > PM,
> > > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > Thanks
> > > > for
> > > > > > the
> > > > > > > > > > updated
> > > > > > > > > > > > > wiki.
> > > > > > > > > > > > > > A
> > > > > > > > > > > > > > > > few
> > > > > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1.
> > > Error
> > > > > > > > > description
> > > > > > > > > > in
> > > > > > > > > > > > > > > > response: I
> > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> indicate
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > several
> > > > > > > different
> > > > > > > > > > error
> > > > > > > > > > > > > cases
> > > > > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> multiple
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > codes.
> > > In
> > > > > > > general
> > > > > > > > > the
> > > > > > > > > > > > > > errorCode
> > > > > > > > > > > > > > > > > > itself
> > > > > > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > sufficient
> > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > describing
> > > > > > the
> > > > > > > > > server
> > > > > > > > > > > > side
> > > > > > > > > > > > > > > > errors.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2.
> > > > Describe
> > > > > > > topic
> > > > > > > > > > > > request:
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> batching
> > on
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> topic
> > > > name
> > > > > > > regex
> > > > > > > > > for
> > > > > > > > > > > this
> > > > > > > > > > > > > > > > request.
> > > > > > > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the
> > > topic
> > > > > > > command
> > > > > > > > > is
> > > > > > > > > > to
> > > > > > > > > > > > > list
> > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> topic
> > > > name
> > > > > > > regex
> > > > > > > > > then
> > > > > > > > > > > we
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > description
> > > > > > > info
> > > > > > > > > and
> > > > > > > > > > > then
> > > > > > > > > > > > > > > filter
> > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > burden
> > > on
> > > > > ZK.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3.
> > > Config
> > > > > > K-Vs
> > > > > > > in
> > > > > > > > > > > create
> > > > > > > > > > > > > > topic:
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> maybe
> > > we
> > > > > can
> > > > > > > add
> > > > > > > > > > > another
> > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > K-V or
> > > > > > > > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> with
> > > > config
> > > > > > K-V
> > > > > > > > in
> > > > > > > > > > > create
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> request.
> > > This
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> field
> > > can
> > > > > be
> > > > > > > > quite
> > > > > > > > > > > useful
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > storing
> > > > > > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who
> > > issue
> > > > > the
> > > > > > > > > create
> > > > > > > > > > > > > command,
> > > > > > > > > > > > > > > > etc,
> > > > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > multi-tenant
> > > > > > > > > setting.
> > > > > > > > > > > > Then
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on
> > > regex
> > > > of
> > > > > > the
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > field.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4.
> > > Today
> > > > > all
> > > > > > > the
> > > > > > > > > > admin
> > > > > > > > > > > > > > > operations
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > return
> > > > once
> > > > > > it
> > > > > > > is
> > > > > > > > > > > written
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > ZK,
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > verification
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> like
> > > > > > > > > > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > request,
> > > > > etc.
> > > > > > > > With
> > > > > > > > > > > admin
> > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > synchronous
> > > > > > > > > requests;
> > > > > > > > > > > > when
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > turned
> > > > > > > > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> until
> > > the
> > > > > > > request
> > > > > > > > > has
> > > > > > > > > > > > been
> > > > > > > > > > > > > > > > > > completed. And
> > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > "token"
> > > > > field
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > > > response,
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > verification
> > > > > > > > > request"
> > > > > > > > > > > > with
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > given
> > > > > > > > > > > > > > > > > > > > token
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has
> > > been
> > > > > > > > completed.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5.
> +1
> > > for
> > > > > > > > extending
> > > > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > coordinator
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > information,
> > > > > > > and
> > > > > > > > > then
> > > > > > > > > > > we
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > ClusterMetadata
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > requests.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > Guozhang
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On
> > Tue,
> > > > Mar
> > > > > > 3,
> > > > > > > > 2015
> > > > > > > > > > at
> > > > > > > > > > > > > 10:23
> > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > jjkoshy.w@gmail.com
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > Thanks
> > > > for
> > > > > > > > sending
> > > > > > > > > > > that
> > > > > > > > > > > > > out
> > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > -
> > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it
> > > > today,
> > > > > so
> > > > > > > if
> > > > > > > > > > notes
> > > > > > > > > > > > can
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On
> > > Mon,
> > > > > Mar
> > > > > > > 02,
> > > > > > > > > 2015
> > > > > > > > > > > at
> > > > > > > > > > > > > > > > 09:16:13AM
> > > > > > > > > > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > Thanks
> > > > > for
> > > > > > > > > sending
> > > > > > > > > > > > this
> > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > Joe.
> > > > > > > > > > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > everyone
> > > > > :)
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> On
> > > > Mon,
> > > > > > Mar
> > > > > > > 2,
> > > > > > > > > > 2015
> > > > > > > > > > > at
> > > > > > > > > > > > > > 6:46
> > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > > > Hey, I
> > > > > > > just
> > > > > > > > > sent
> > > > > > > > > > > > out a
> > > > > > > > > > > > > > > > google
> > > > > > > > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > committers
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > > > > > everyone I
> > > > > > > > > found
> > > > > > > > > > > > > working
> > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > please
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > > let
> > > > me
> > > > > > > know
> > > > > > > > > and
> > > > > > > > > > > can
> > > > > > > > > > > > > > update
> > > > > > > > > > > > > > > > it,
> > > > > > > > > > > > > > > > > > np.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > We
> > > > > > should
> > > > > > > do
> > > > > > > > > > this
> > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > Tuesday
> > > > > > > > > > > > > > > > > > @
> > > > > > > > > > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > INFRA
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > > help
> > > > > to
> > > > > > > > make a
> > > > > > > > > > > > google
> > > > > > > > > > > > > > > > account
> > > > > > > > > > > > > > > > > > so we
> > > > > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > To
> > > > > > discuss
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > in
> > > > > > > progress
> > > > > > > > > and
> > > > > > > > > > > > > related
> > > > > > > > > > > > > > > JIRA
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > ~
> > > > Joe
> > > > > > > Stein
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > On
> > > > > Tue,
> > > > > > > Feb
> > > > > > > > > 24,
> > > > > > > > > > > 2015
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > 2:59
> > > > > > > > > > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > jay.kreps@gmail.com
> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > Let's
> > > > > > > stay
> > > > > > > > on
> > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > hangouts
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > sessions
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > > > available
> > > > > > > > on
> > > > > > > > > > > > youtube.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > -Jay
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > On
> > > > > Tue,
> > > > > > > Feb
> > > > > > > > > 24,
> > > > > > > > > > > > 2015
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > 11:49
> > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > jholoman@cloudera.com
> > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > Jay /
> > > > > > > Joe
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > We're
> > > > > > > > happy
> > > > > > > > > > to
> > > > > > > > > > > > send
> > > > > > > > > > > > > > > out a
> > > > > > > > > > > > > > > > > > Webex
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> the
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > > > sessions
> > > > > > > > if
> > > > > > > > > > > there
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > interest and
> > > > > > > > > > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > > Thanks
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > Jeff
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > On
> > > > > > Tue,
> > > > > > > > Feb
> > > > > > > > > > 24,
> > > > > > > > > > > > > 2015
> > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > Let's
> > > > > > > > try
> > > > > > > > > > to
> > > > > > > > > > > > get
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > technical
> > > > > > > > > > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > really
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > think
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > there
> > > > > > > > is
> > > > > > > > > > some
> > > > > > > > > > > > > > benefit
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > hopeful
> > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > if
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > we
> > > > > > > post
> > > > > > > > > > > > > > instructions
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > give
> > > > > > > > > > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> get
> > it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > > > working.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > > Tuesday
> > > > > > > > > at
> > > > > > > > > > > that
> > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > work
> > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > -Jay
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > On
> > > > > > > Tue,
> > > > > > > > > Feb
> > > > > > > > > > > 24,
> > > > > > > > > > > > > > 2015
> > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > 8:18
> > > > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > > Weekly
> > > > > > > > > > > would
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > > maybe
> > > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > ????
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > I
> > > > > > > > don't
> > > > > > > > > > > mind
> > > > > > > > > > > > > > google
> > > > > > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> issue
> or
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > > > whatever
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > so
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > we
> > > > > > > > know
> > > > > > > > > > the
> > > > > > > > > > > > > > apache
> > > > > > > > > > > > > > > > irc
> > > > > > > > > > > > > > > > > > > > channel
> > > > > > > > > > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> see
> > > how
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > goes?
> > > > > > > > > We
> > > > > > > > > > > can
> > > > > > > > > > > > > pull
> > > > > > > > > > > > > > > > > > transcripts
> > > > > > > > > > > > > > > > > > > > > too
> > > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets
> > if
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> need
> > > be
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > makes
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > it
> > > > > > > > > > helpful
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > things.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > ~
> > > > > > > > > > Joestein
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > On
> > > > > > > > Tue,
> > > > > > > > > > Feb
> > > > > > > > > > > > 24,
> > > > > > > > > > > > > > > 2015
> > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > 11:10
> > > > > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > jay.kreps@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > We'd
> > > > > > > > > > > talked
> > > > > > > > > > > > > > about
> > > > > > > > > > > > > > > > > > doing a
> > > > > > > > > > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> What
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > about
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > > > > > generalizing
> > > > > > > > > > > > > > > that a
> > > > > > > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > would
> > > be
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > good
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > > > everyone
> > > > > > > > > > > > > > > spending a
> > > > > > > > > > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > stuff
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > maybe
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > sync
> > > > > > > > > up
> > > > > > > > > > > > once
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > week.
> > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > through
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > > design
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > > stuff,
> > > > > > > > > > > make
> > > > > > > > > > > > > > sure
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > are on
> > > > > > > > > > > > > > > > > > > > > top
> > > > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> any
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > > tricky
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > > issues,
> > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > We
> > > > > > > > > can
> > > > > > > > > > > make
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > publicly
> > > > > > > > > > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > along
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > who
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > > likes.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > Any
> > > > > > > > > > > > interest
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > doing
> > > > > > > > > > > > > > > > > > this?
> > > > > > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > starting
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > next
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > week.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > -Jay
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > On
> > > > > > > > > Tue,
> > > > > > > > > > > Feb
> > > > > > > > > > > > > 24,
> > > > > > > > > > > > > > > > 2015
> > > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > >
> > > > > > > > Hi
> > > > > > > > > > all,
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > >
> > > > > > > > > I've
> > > > > > > > > > > > > updated
> > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > page,
> > > > > > > > > > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > structure.
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > Also I
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >>
> > > > >
> > > > > > added
> > > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > >> >
> > > > > >
> > > > > > > >
> > > > > > > > > some
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > ...
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > [Message clipped]
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > --
> > > > > > > > > > > > > -- Guozhang
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > -- Guozhang
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > Thanks,
> > Neha
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Hi all,

A short summary of our discussion:

201. DescribeTopicRequest(DTR) vs TopicMetadataRequest(TMR)

*Issue*: Topic description should include topic-level configuration,
other information is available in TMR - leader, ISR, AR.
So now there are two options - either add full-fledged new DTR
that will include TMR + all needed information (which will be used then
instead TMR) or extend TMR to include topic-level configuration.

*Resolution*: it was decided to vote on it: either a) *extend TMR* or b) *add
DTR*.
I'd go with a) - we will have to roll out TMR_V1 but it's better than having
"dead" requests if we add DTR and leave TMR unused in future versions

202. CreateTopicRequest sync/async semantics

*Issue*: What should be a post-condition of Create/Alter/Delete
requests - the command has been completed or command has been
initiated?
Even though it looks reasonable that users would typically expect to
have topic created upon CreateTopicRequest the corner case is that
topic can be deleted right after creation, so received CreateTopicResponse
cannot be a 100% guarantee topic is created and available, even if
CreateTopicRequest is synchronous. Also, making request blocking on
server may be a hard thing to do right now (check Jun's email from Mar 21).

*Resolution*: for now it makes sense to start with async semantics for
Create/Alter/Delete.

203. VerifyReassignPartition vs leverage DescribeTopicRequest

*Issue*: DescribeTopicRequest (or TopicMetadaRequest) should be enough,
since controller doesn't remove partitions from admin/reassign_partitions
until
the assignment actually matches target assignment so partition reassignment
cannot fail for specific partition in the current implementation.
Everything else
("Completed" and "In Progress" state) can be checked with assigned
partitions
field from DescribeTopicRequest.

*Resolution*: remove VerifyReassignPartitionsRequest

204. Also it was discussed that after we finalize question 201 and port
java protocol definition (KAFKA-1927) I will split patch to separate pieces,
first will cover Wire Protocol changes, since we have a common vision on it.

Please let me know your thoughts on q. 201!

Thanks,
Andrii Biletskyi




On Tue, Mar 24, 2015 at 7:57 PM, Neha Narkhede <ne...@confluent.io> wrote:

> I'd like to resurface the discussion of sync vs async topic creation.
> Possibly, we can discuss that in today's KIP meeting. I am worried about
> pushing the burden of handling async topic creation on the clients. This is
> one of the most frequently asked questions on the mailing list and I hope
> we find a way to address it as we separate the metadata request and topic
> creation. Since topic creation doesn't take that long, there should be a
> really easy way for clients to express their preference for a sync topic
> creation request where the receipt of a response suggests completion of
> topic creation.
>
> The behavior that I'm expecting is that if I'm a client and I choose to
> create a topic synchronously, after I receive a response, my first send
> request for that topic should *not* fail with some error.
>
> On Tue, Mar 24, 2015 at 8:02 AM, Jun Rao <ju...@confluent.io> wrote:
>
> > Andrii,
> >
> >
> > 111.4 Yes, we can discuss this in the meeting today.
> >
> > 112. The question is when will a PartitionReassignment fail. Currently,
> it
> > can fail if the input is incorrect (e.g, invalid partition/replica).
> > However, this can be detected when the ReassignPartitionRequest is
> issued.
> > Once the process of partition reassignment is started, it will not fail.
> > The controller just waits until the process completes. We can improve the
> > status reporting of partition reassignment. However, I am not sure if
> that
> > needs to be tied to this KIP.
> >
> > 115.1 Yes, that makes sense. I misunderstood it. I thought
> > PreferredReplicaLeaderElectionInProgress and ReassignPartitionsInProgress
> > are used in the status check.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Tue, Mar 24, 2015 at 3:46 AM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Jun,
> > >
> > > Thanks for such an accurate review!
> > >
> > > Most of your remarks I didn't fix in the previous change because
> > > I thought we'd finalize them today, during the call. Anyway, I've
> > > fixed them now.
> > > My comments to ensure I covered everything:
> > >
> > > 110. Fixed. Removed global error code.
> > >
> > > 111.1-3. Removed redundant TopicName, Partitions and ReplicationFactor
> > > 111.4 Yes, now we are in the situation when TopicMetadataResponse
> > > contains everything we need except topic level configs. I'm
> > > okay with both solutions, probably makes sense to discuss and pick
> > > the preferable one.
> > >
> > > 112. Need to think about it a bit more. Currently we may distinguish
> > > 3 states of specific partition being reassigned:
> > > a) Completed - absent in /reassign_partitions zk path and
> DescribeTopic/
> > > TopicMetadata shows expected assignment
> > > 2) In progress - present in /reassign_partitions
> > > 3) Failed - absent in /reassign_partitions but
> > DescribeTopic/TopicMetadata
> > > replicas field doesn't correspond expected
> > > Now, if use only DescribeTopic to check reassignment status - how can
> we
> > > distinguish states "In Progress" and "Failed"?
> > > Also, there are some concerns that current reassignment status provided
> > > by ReaasignPartitionsCommand (and the same was intended in KIP-4) is
> > > very spurious, non-informative. Probably it makes sense to add
> additional
> > > data there (thus separate request may be useful), I'm considering
> > different
> > > options now - btw, comments here are highly appreciated!
> > >
> > > 113-114. Okay, made it compliant with topic commands. About "The error
> > code
> > > will
> > > then be per topic" - just a note: currently (ReassignPartitionsCommand)
> > > validation
> > > is done for the whole reassignment string, so in case of errors for
> > > specific
> > > partitions the reassignment is NOT started at all. Both options look
> fine
> > > for me,
> > > it's just the logic will be slightly changed.
> > >
> > > 115.1 Removed NotControllerReceivedAdminCommand. But why should we
> > > remove PreferredReplicaLeaderElectionInProgress and
> > > ReassignPartitionsInProgress?
> > > On Reassign/Preferred..Request we create admin zk path, we can of
> course
> > > update the path, but wouldn't it be safer if we simply refuse to start
> > new
> > > reassignment in case one is in progress?
> > >
> > > 115.2 Yes, paragraph Protocol Errors covers only errors that are
> proposed
> > > to be added. I didn't want to list all errors that are to be added
> since
> > > anyway
> > > the final version will be clear after the development is completed.
> > >
> > > 100. Fixed, normalized json string in Create/Alter.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Tue, Mar 24, 2015 at 12:58 AM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Andrii,
> > > >
> > > > I looked at the latest wiki for this KIP. I have a few more comments.
> > > >
> > > > 110. In CreateTopicResponse, AlterTopicResponse, DeleteTopicResponse
> > and
> > > > DescribeTopicResponse, we probably don't need the global error code.
> > Our
> > > > current convention is to just use the same global error code in each
> > > topic.
> > > > This simplifies the error checking on the client side.
> > > >
> > > > 111. DescribeTopicResponse:
> > > > 111.1 Our protocol definition doesn't support optional fields. So, we
> > > can't
> > > > make Leader an optional field.
> > > > 111.2 Do we really need the field Partitions and ReplicationFactor in
> > > > TopicConfigDetails?
> > > > The former is basically the size of the TopicPartitionDetails array
> and
> > > the
> > > > latter can be found from the size of the Replica array.
> > > > 111.3 Do we need TopicName in TopicDescription since it's already in
> > > > DescribeTopicResponse?
> > > > 111.4 Finally, DescribeTopicResponse is very similar to
> > > > TopicMetadataResponse. So, it probably makes sense just to keep one
> of
> > > them
> > > > in the future. Should we just use DescribeTopicRequest/Response to
> > > replace
> > > > TopicMetadataRequest/Response in the producer/consumer client in the
> > > future
> > > > (since DescribeTopicRequest doesn't trigger auto topic creation)? If
> > so,
> > > we
> > > > will need to add the broker list in DescribeTopicResponse.
> > > >
> > > > 112. Thinking about this a bit more, I don't think we need a separate
> > > > VerifyReassignPartitionRequest/Response.
> > > > We can just use DescribeTopic to get the assigned replicas and check
> if
> > > > they match the target replica assignment in the client. The
> controller
> > > > propagates the metadata change after the reassignment completes for
> > each
> > > > partition.
> > > >
> > > > 113. ReassignPartitionRequest: For consistency, we probably want to
> > nest
> > > > the partition data under topic. So instead of
> > > >   ReassignPartitionRequest => [Topic PartitionId [ReplicaId]]
> > > > we can have sth like
> > > >   ReassignPartitionRequest => [Topic [PartitionId [ReplicaId]]]
> > > > The error code will then be per topic.
> > > >
> > > > 114. PreferredReplicaLeaderElectionRequest: Same as the above.
> Instead
> > of
> > > >   PreferredReplicaLeaderElectionRequest => [Topic PartitionId]
> > > > we can have
> > > >   PreferredReplicaLeaderElectionRequest => [Topic [PartitionId]]
> > > > Again, the error code will be per topic.
> > > >
> > > > 115. ErrorCode:
> > > > 115.1 NotControllerReceivedAdminRequest,
> > > > PreferredReplicaLeaderElectionInProgress
> > > > and ReassignPartitionsInProgress are likely not needed any more.
> > > > 115.2 We probably need to add UnknownTopicOrPartitionCode since it's
> a
> > > > valid error code for some of the requests (e.g. describe, alter, etc)
> > > when
> > > > a non-existing topic is specified.
> > > >
> > > >
> > > > Previous comments.
> > > > 100. ReplicaAssignment in CreateTopicRequest and AlterTopicRequest
> are
> > > > still a json string. We need to flatten them.
> > > >
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Sat, Mar 21, 2015 at 2:03 AM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Guozhang,
> > > > >
> > > > > I'm not sure I understand how can we use those tools in CLI.
> > > > > First of all, those are *Test*Utils, e.g.
> > waitUntilMetadataIsPropagated
> > > > > (which might be very useful for us) requires
> servers:Seq[KafkaServer]
> > > > > as an argument:
> > > > >
> > > > > def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic:
> > > > String,
> > > > > partition: Int, timeout: Long = 5000L): Int
> > > > >
> > > > > how can we have access it at runtime in one of the brokers?
> > > > >
> > > > > Secondly, how can user directly call these tools if zookeeper might
> > be
> > > > > not accessible (which is used in the tools, right?) at all - e.g.
> > > behind
> > > > > the VPC
> > > > > in AWS. Furthermore, I think with KIP-4 we are trying to create an
> > > > > abstraction and
> > > > > a single point of zookeeper interactions, to eliminate direct calls
> > to
> > > ZK
> > > > > either to get
> > > > > some cluster information or to change something.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > > On Sat, Mar 21, 2015 at 3:37 AM, Guozhang Wang <wangguoz@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > Andrii,
> > > > > >
> > > > > > Actually the checking logic Jun mentioned is already implemented
> as
> > > > > > TestUtils.waitUntilXXX (LeaderIsElected, MetadataIsPropagated,
> > > etc...)
> > > > I
> > > > > > think we can extend these functions as CLI tools like
> TopicCommand
> > so
> > > > > that
> > > > > > users re-implementing such endpoint can directly call something
> > like
> > > > > > java.tools.WaitUntilXXX (of course this requires them to have
> javac
> > > > > > installed, which should be a reasonable requirement?)
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Jun,
> > > > > > >
> > > > > > > Not that I was saying we need to make requests blocking on
> > server,
> > > > > > > it was just to emphasize that with async requests a client
> > > > > > implementations
> > > > > > > may be a little bit more than just issue request - get the
> > > response.
> > > > > > > Thanks for the explanation, I understand now that we can go
> with
> > > > agreed
> > > > > > > solution though it may not be perfect.
> > > > > > > I believe this was one of the last controversial questions from
> > the
> > > > > list.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > > On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > >
> > > > > > > > Andrii,
> > > > > > > >
> > > > > > > > A few points.
> > > > > > > >
> > > > > > > > 1. Create/Alter can typically complete quickly. So, it's
> > possible
> > > > to
> > > > > > make
> > > > > > > > the request block until it's completed. However, currently,
> > doing
> > > > > this
> > > > > > at
> > > > > > > > the broker is a bit involved. To make Create block, we will
> > need
> > > to
> > > > > add
> > > > > > > > some callbacks in KafkaController. This is possible. However,
> > the
> > > > > > > > controller logic currently is pretty completed. It would
> > probably
> > > > be
> > > > > > > better
> > > > > > > > if we clean it up first before adding more complexity to it.
> > > Alter
> > > > is
> > > > > > > even
> > > > > > > > trickier. Adding partition is currently handled through
> > > > > > KafkaController.
> > > > > > > So
> > > > > > > > it can be dealt with in a similar way. However, Alter config
> is
> > > > done
> > > > > > > > completely differently. It doesn't go through the controller.
> > > > > Instead,
> > > > > > > each
> > > > > > > > broker listens to ZooKeeper directly. So, it's not clear if
> > there
> > > > is
> > > > > an
> > > > > > > > easy way on the broker to figure out whether a config is
> > applied
> > > on
> > > > > > every
> > > > > > > > broker.
> > > > > > > >
> > > > > > > > 2. Delete can potentially take long if a replica to be
> deleted
> > is
> > > > > > > offline.
> > > > > > > > PreferredLeader/PartitionReassign can also take long. So, we
> > > can't
> > > > > > really
> > > > > > > > make those requests block on the broker.
> > > > > > > >
> > > > > > > > As you can see, at this moment it's not easy to make all
> admin
> > > > > requests
> > > > > > > > block on the broker. So, if we want the blocking feature in
> the
> > > > admin
> > > > > > > > utility in the short term, doing the completion check at the
> > > admin
> > > > > > client
> > > > > > > > is probably an easier route, even though it may not be ideal.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >
> > > > > > > > > Jun,
> > > > > > > > >
> > > > > > > > > I see your point. But wouldn't that lead to a "fat" client
> > > > > > > > implementations?
> > > > > > > > > Suppose someone would like to implement client for Admin
> Wire
> > > > > > protocol.
> > > > > > > > > Not only people will have to code quite complicated logic
> > like
> > > > > "send
> > > > > > > > > describe
> > > > > > > > > request to each broker" (again state machin?) but it will
> > also
> > > > mean
> > > > > > > > people
> > > > > > > > > must understand internal kafka logic related to topic
> storage
> > > and
> > > > > how
> > > > > > > > > information is propageted from the controller to brokers.
> > > > > > > > > I see this like a dilemma between having a concise Wire
> > > Protocol
> > > > > and
> > > > > > > > > self-sufficient API to make client implementations simple.
> > > > > > > > > I don't have a win-win solution though.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Andrii Biletskyi
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <
> jun@confluent.io>
> > > > > wrote:
> > > > > > > > >
> > > > > > > > > > For 1), 2) and 3), blocking would probably mean that the
> > new
> > > > > > metadata
> > > > > > > > is
> > > > > > > > > > propagated to every broker. To achieve that, the client
> can
> > > > keep
> > > > > > > > issuing
> > > > > > > > > > the describe topic request to every broker until it sees
> > the
> > > > new
> > > > > > > > metadata
> > > > > > > > > > in the response.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > >
> > > > > > > > > > > Hm, actually the ticket you linked, Guozhang, brings as
> > > back
> > > > > > > > > > > to the problem what should be considered a
> post-condition
> > > for
> > > > > > > > > > > each of the admin commands.
> > > > > > > > > > > In my understanding:
> > > > > > > > > > >
> > > > > > > > > > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > > > > > > > > > (Not the controller picked up changes from zk and
> > > broadcasted
> > > > > > > > > > > LeaderAndIsr and UpdateMetadata)
> > > > > > > > > > >
> > > > > > > > > > > 2) AlterTopic - same as 1) - broker changed assignment
> > data
> > > > > > > > > > > in zookeeper or created admin path for topic config
> > change
> > > > > > > > > > >
> > > > > > > > > > > 3) DeleteTopic - admin path /admin/delete_topics is
> > created
> > > > > > > > > > >
> > > > > > > > > > > 4) ReassignPartitions and PreferredReplica -
> > corresponding
> > > > > admin
> > > > > > > > > > > path is created
> > > > > > > > > > >
> > > > > > > > > > > Now what can be considered a completed operation from
> the
> > > > > > client's
> > > > > > > > > > > perspective?
> > > > > > > > > > > 1) Topic is created once corresponding data is in zk
> > > > > > > > > > > (I remember there were some thoughts that it'd be good
> to
> > > > > > consider
> > > > > > > > > > > topic created once all replicas receive information
> about
> > > it
> > > > > and
> > > > > > > thus
> > > > > > > > > > > clients can produce/consume from it, but as was
> discussed
> > > > this
> > > > > > > seems
> > > > > > > > > > > to be a hard thing to do)
> > > > > > > > > > >
> > > > > > > > > > > 2) Probably same as 1), so right after AlterTopic is
> > issued
> > > > > > > > > > >
> > > > > > > > > > > 3) The topic has been removed from /brokers/topics
> > > > > > > > > > >
> > > > > > > > > > > 4) ReassignPartitions and PrefferedReplica were
> discussed
> > > > > > earlier -
> > > > > > > > > > > in short the former is completed once partition state
> > info
> > > in
> > > > > zk
> > > > > > > > > matches
> > > > > > > > > > > reassignment request and admin path is empty, the
> latter
> > -
> > > > once
> > > > > > > data
> > > > > > > > > > > in zk shows that head of assignned replicas of the
> > > partition
> > > > > and
> > > > > > > > leader
> > > > > > > > > > > is the same replica
> > > > > > > > > > >
> > > > > > > > > > > Thoughts?
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <
> > > > > > wangguoz@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > I think while loop is fine for supporting blocking,
> > just
> > > > that
> > > > > > we
> > > > > > > > need
> > > > > > > > > > to
> > > > > > > > > > > > add back off to avoid bombarding brokers with
> > > DescribeTopic
> > > > > > > > requests.
> > > > > > > > > > > >
> > > > > > > > > > > > Also I have linked KAFKA-1125
> > > > > > > > > > > > <https://issues.apache.org/jira/browse/KAFKA-1125>
> to
> > > your
> > > > > > > > proposal,
> > > > > > > > > > and
> > > > > > > > > > > > when KAFKA-1694 is done this ticket can also be
> closed.
> > > > > > > > > > > >
> > > > > > > > > > > > Guozhang
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Great.
> > > > > > > > > > > > > I want to elaborate this a bit more, to see we are
> on
> > > the
> > > > > > same
> > > > > > > > page
> > > > > > > > > > > > > concerning the client code.
> > > > > > > > > > > > >
> > > > > > > > > > > > > So with all topic commands being async a client
> > > > > (AdminClient
> > > > > > in
> > > > > > > > our
> > > > > > > > > > > > > case or any other other client people would like to
> > > > > > implement)
> > > > > > > to
> > > > > > > > > > > support
> > > > > > > > > > > > > a blocking operation (which seems to be a natural
> > > > use-case
> > > > > > e.g.
> > > > > > > > for
> > > > > > > > > > > topic
> > > > > > > > > > > > > creation): would have to do:
> > > > > > > > > > > > > 1. issue CreateTopicRequest
> > > > > > > > > > > > > 2. if successful, in a "while" loop send
> > > > > DescribeTopicRequest
> > > > > > > and
> > > > > > > > > > > > > break the loop once all topics are returned in
> > response
> > > > (or
> > > > > > > upon
> > > > > > > > > > > > timeout).
> > > > > > > > > > > > > 3. if unsuccessful throw exception
> > > > > > > > > > > > > Would it be okay?
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <
> > > > jun@confluent.io
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I think you are right. It seems that only
> > > > > > ReassignPartitions
> > > > > > > > > needs
> > > > > > > > > > a
> > > > > > > > > > > > > > separate verification request.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii
> Biletskyi <
> > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > I like this idea too. Let's stick with that.
> I'll
> > > > > update
> > > > > > > KIP
> > > > > > > > > > > > > accordingly.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I was also thinking we can avoid adding
> dedicated
> > > > > status
> > > > > > > > check
> > > > > > > > > > > > > > > requests for topic commands. - We have
> everything
> > > in
> > > > > > > > > > DescribeTopic
> > > > > > > > > > > > > > > for that! E.g.:
> > > > > > > > > > > > > > > User issued CreateTopic - to check the status
> > > client
> > > > > > sends
> > > > > > > > > > > > > DescribeTopic
> > > > > > > > > > > > > > > and checks whether is something returned for
> that
> > > > > topic.
> > > > > > > The
> > > > > > > > > same
> > > > > > > > > > > for
> > > > > > > > > > > > > > > alteration, deletion.
> > > > > > > > > > > > > > > Btw, PreferredReplica status can be also
> checked
> > > with
> > > > > > > > > > > > > > DescribeTopicRequest
> > > > > > > > > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > > > > > > > > For ReassignPartitions as discussed we'll need
> to
> > > > have
> > > > > a
> > > > > > > > > separate
> > > > > > > > > > > > > > Verify...
> > > > > > > > > > > > > > > request.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang
> <
> > > > > > > > > > wangguoz@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1 on broker writing to ZK for async
> handling.
> > I
> > > > was
> > > > > > > > thinking
> > > > > > > > > > > that
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > end state the admin requests would be
> > eventually
> > > > sent
> > > > > > to
> > > > > > > > > > > controller
> > > > > > > > > > > > > > > either
> > > > > > > > > > > > > > > > through re-routing or clients discovering
> them,
> > > > > instead
> > > > > > > of
> > > > > > > > > > > letting
> > > > > > > > > > > > > > > > controller listen on ZK admin path. But
> > thinking
> > > > > about
> > > > > > > it a
> > > > > > > > > > > second
> > > > > > > > > > > > > > time,
> > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > think it is actually simpler to let
> controller
> > > > manage
> > > > > > > > > > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > +1 as well. I think it helps to keep the
> > > > rerouting
> > > > > > > > approach
> > > > > > > > > > > > > > orthogonal
> > > > > > > > > > > > > > > > > to this KIP.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700,
> Jay
> > > > Kreps
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > I'm +1 on Jun's suggestion as long as it
> > can
> > > > work
> > > > > > for
> > > > > > > > all
> > > > > > > > > > the
> > > > > > > > > > > > > > > requests.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao
> <
> > > > > > > > > jun@confluent.io
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > (a) Admin requests can be sent to and
> > > handled
> > > > > by
> > > > > > > any
> > > > > > > > > > > broker.
> > > > > > > > > > > > > > > > > > > (b) Admin requests are processed
> > > > > asynchronously,
> > > > > > at
> > > > > > > > > least
> > > > > > > > > > > for
> > > > > > > > > > > > > > now.
> > > > > > > > > > > > > > > > > That is,
> > > > > > > > > > > > > > > > > > > when the client gets a response, it
> just
> > > > means
> > > > > > that
> > > > > > > > the
> > > > > > > > > > > > request
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > initiated, but not necessarily
> completed.
> > > > Then,
> > > > > > > it's
> > > > > > > > up
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > issue another request to check the
> status
> > > for
> > > > > > > > > completion.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > To support (a), we were thinking of
> doing
> > > > > request
> > > > > > > > > > > forwarding
> > > > > > > > > > > > to
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > controller (utilizing KAFKA-1912). I am
> > > > making
> > > > > an
> > > > > > > > > > > alternative
> > > > > > > > > > > > > > > > proposal.
> > > > > > > > > > > > > > > > > > > Basically, the broker can just write to
> > > > > ZooKeeper
> > > > > > > to
> > > > > > > > > > inform
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > > > about the request. For example, to
> handle
> > > > > > > > > > > > > partitionReassignment,
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > will just write the requested
> partitions
> > to
> > > > > > > > > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > > > > > > > > (like what AdminUtils currently does)
> and
> > > > then
> > > > > > > send a
> > > > > > > > > > > > response
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > client. This shouldn't take long and
> the
> > > > > > > > implementation
> > > > > > > > > > > will
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > > simpler
> > > > > > > > > > > > > > > > > > > than forwarding the requests to the
> > > > controller
> > > > > > > > through
> > > > > > > > > > RPC.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii
> > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Jun,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I might be wrong but didn't we agree
> we
> > > > will
> > > > > > let
> > > > > > > > any
> > > > > > > > > > > broker
> > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > cluster handle *long-running* admin
> > > > requests
> > > > > > (at
> > > > > > > > this
> > > > > > > > > > > time
> > > > > > > > > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > reassignPartitions), via zk admin
> path.
> > > > Thus
> > > > > > > > > > CreateTopics
> > > > > > > > > > > > etc
> > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun
> > > Rao <
> > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > I think we agreed that those admin
> > > > requests
> > > > > > can
> > > > > > > > be
> > > > > > > > > > > issued
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > > broker.
> > > > > > > > > > > > > > > > > > > > > Because of that, there doesn't seem
> > to
> > > > be a
> > > > > > > > strong
> > > > > > > > > > need
> > > > > > > > > > > > to
> > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > controller. So, perhaps we can
> > proceed
> > > by
> > > > > not
> > > > > > > > > making
> > > > > > > > > > > any
> > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > format of TMR right now. When we
> > start
> > > > > using
> > > > > > > > create
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > producer, we will need a new
> version
> > of
> > > > TMR
> > > > > > > that
> > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > creation. But that can be done
> later.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > As a first cut implementation, I
> > think
> > > > the
> > > > > > > broker
> > > > > > > > > can
> > > > > > > > > > > > just
> > > > > > > > > > > > > > > write
> > > > > > > > > > > > > > > > > to ZK
> > > > > > > > > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > >
> createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > > > > > > > > requests, instead of forwarding
> them
> > to
> > > > the
> > > > > > > > > > controller.
> > > > > > > > > > > > > This
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > > > > > > > > the implementation on the broker
> > side.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM,
> > Joel
> > > > > Koshy
> > > > > > <
> > > > > > > > > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > For (1) yes we will circle back
> on
> > > that
> > > > > > > shortly
> > > > > > > > > > after
> > > > > > > > > > > > > > syncing
> > > > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > person. I think it is close to
> > > getting
> > > > > > > > committed
> > > > > > > > > > > > although
> > > > > > > > > > > > > > > > > development
> > > > > > > > > > > > > > > > > > > > > > for KAFKA-1927 can probably begin
> > > > without
> > > > > > it.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > There is one more item we covered
> > at
> > > > the
> > > > > > > > hangout.
> > > > > > > > > > > i.e.,
> > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > want to add the coordinator to
> the
> > > > topic
> > > > > > > > metadata
> > > > > > > > > > > > > response
> > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > There are two reasons I think we
> > > should
> > > > > try
> > > > > > > and
> > > > > > > > > > avoid
> > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > > > > > > > > - It is irrelevant to topic
> > metadata
> > > > > > > > > > > > > > > > > > > > > > - If we finally do request
> > rerouting
> > > in
> > > > > > Kafka
> > > > > > > > > then
> > > > > > > > > > > the
> > > > > > > > > > > > > > field
> > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > >   little to no value. (It still
> > helps
> > > > to
> > > > > > > have a
> > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > > > >   ClusterMetadataRequest to query
> > for
> > > > > > > > > cluster-wide
> > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > > > >   'which broker is the
> controller?'
> > > as
> > > > > Joe
> > > > > > > > > > > mentioned.)
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I think it would be cleaner to
> have
> > > an
> > > > > > > explicit
> > > > > > > > > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > that you can send to any broker
> in
> > > > order
> > > > > to
> > > > > > > > > obtain
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > > > in the future possibly other
> > > > cluster-wide
> > > > > > > > > > > > information). I
> > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > main argument against doing this
> > and
> > > > > > instead
> > > > > > > > > adding
> > > > > > > > > > > it
> > > > > > > > > > > > to
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > metadata response was
> convenience -
> > > > i.e.,
> > > > > > you
> > > > > > > > > don't
> > > > > > > > > > > > have
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > discover
> > > > > > > > > > > > > > > > > > > > > > the controller in advance.
> > However, I
> > > > > don't
> > > > > > > see
> > > > > > > > > > much
> > > > > > > > > > > > > actual
> > > > > > > > > > > > > > > > > > > > > > benefit/convenience in this and
> in
> > > fact
> > > > > > think
> > > > > > > > it
> > > > > > > > > > is a
> > > > > > > > > > > > > > > > non-issue.
> > > > > > > > > > > > > > > > > Let
> > > > > > > > > > > > > > > > > > > > > > me know if I'm overlooking
> > something
> > > > > here.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > As an example, say we need to
> > > initiate
> > > > > > > > partition
> > > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > > > > > issuing the new
> > > > ReassignPartitionsRequest
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > (assume
> > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > already have the desired manual
> > > > partition
> > > > > > > > > > > assignment).
> > > > > > > > > > > > > If
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > are to
> > > > > > > > > > > > > > > > > > > > > > augment topic metadata response
> > then
> > > > the
> > > > > > flow
> > > > > > > > be
> > > > > > > > > > > > > something
> > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > :
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > - Issue topic metadata request to
> > any
> > > > > > broker
> > > > > > > > (and
> > > > > > > > > > > > > discover
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > > > > > > > > - Connect to controller if
> required
> > > > > (i.e.,
> > > > > > if
> > > > > > > > the
> > > > > > > > > > > > broker
> > > > > > > > > > > > > > > above
> > > > > > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > > > > > - Issue the partition
> reassignment
> > > > > request
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > > > > controller.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > With an explicit cluster metadata
> > > > request
> > > > > > it
> > > > > > > > > would
> > > > > > > > > > > be:
> > > > > > > > > > > > > > > > > > > > > > - Issue cluster metadata request
> to
> > > any
> > > > > > > broker
> > > > > > > > > > > > > > > > > > > > > > - Connect to controller if
> required
> > > > > (i.e.,
> > > > > > if
> > > > > > > > the
> > > > > > > > > > > > broker
> > > > > > > > > > > > > > > above
> > > > > > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > > > > > - Issue the partition
> reassignment
> > > > > request
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > So it seems to add little
> practical
> > > > value
> > > > > > and
> > > > > > > > > > bloats
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > response with an irrelevant
> detail.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > The other angle to this is the
> > > > following
> > > > > -
> > > > > > is
> > > > > > > > it
> > > > > > > > > a
> > > > > > > > > > > > matter
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > naming?
> > > > > > > > > > > > > > > > > > > > > > Should we just rename topic
> > metadata
> > > > > > > > > > request/response
> > > > > > > > > > > > to
> > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > > > MetadataRequest/Response and add
> > > > cluster
> > > > > > > > metadata
> > > > > > > > > > to
> > > > > > > > > > > > it?
> > > > > > > > > > > > > By
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > same
> > > > > > > > > > > > > > > > > > > > > > token should we also allow
> querying
> > > for
> > > > > the
> > > > > > > > > > consumer
> > > > > > > > > > > > > > > > coordinator
> > > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > > > in future transaction
> coordinator)
> > as
> > > > > well?
> > > > > > > > This
> > > > > > > > > > > leads
> > > > > > > > > > > > > to a
> > > > > > > > > > > > > > > > > bloated
> > > > > > > > > > > > > > > > > > > > > > request which isn't very
> appealing
> > > and
> > > > > > > > altogether
> > > > > > > > > > > > > > confusing.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at
> 09:34:12AM
> > > > -0700,
> > > > > > Jun
> > > > > > > > Rao
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 1. I just realized that in
> order
> > to
> > > > > start
> > > > > > > > > working
> > > > > > > > > > > on
> > > > > > > > > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > need to merge the changes to
> > > > > > > > > OffsetCommitRequest
> > > > > > > > > > > > (from
> > > > > > > > > > > > > > > 0.8.2)
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > > > > > > > > This is planned to be done as
> > part
> > > of
> > > > > > > > > KAFKA-1634.
> > > > > > > > > > > So,
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > > > > > and Joel's help to wrap this
> up.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 2. Thinking about this a bit
> > more,
> > > if
> > > > > the
> > > > > > > > > > semantic
> > > > > > > > > > > of
> > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > "write"
> > > > > > > > > > > > > > > > > > > > > > > requests is async (i.e., after
> > the
> > > > > client
> > > > > > > > gets
> > > > > > > > > a
> > > > > > > > > > > > > > response,
> > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > > > > > > > > that the operation is
> initiated,
> > > but
> > > > > not
> > > > > > > > > > > necessarily
> > > > > > > > > > > > > > > > > completed), we
> > > > > > > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > > > > > really need to forward the
> > requests
> > > > to
> > > > > > the
> > > > > > > > > > > > controller.
> > > > > > > > > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > > > > > > > > receiving broker can just write
> > the
> > > > > > > operation
> > > > > > > > > to
> > > > > > > > > > ZK
> > > > > > > > > > > > as
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > > > > > > > > line tool previously does. This
> > > will
> > > > > > > simplify
> > > > > > > > > the
> > > > > > > > > > > > > > > > > implementation.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 8. There is another
> > implementation
> > > > > detail
> > > > > > > for
> > > > > > > > > > > > describe
> > > > > > > > > > > > > > > topic.
> > > > > > > > > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > want to read the topic config
> > from
> > > > the
> > > > > > > broker
> > > > > > > > > > > cache,
> > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > > > > > > > > Currently, every broker reads
> the
> > > > > > > topic-level
> > > > > > > > > > > config
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > > > > > However, it ignores those for
> > > topics
> > > > > not
> > > > > > > > hosted
> > > > > > > > > > on
> > > > > > > > > > > > > > itself.
> > > > > > > > > > > > > > > > So,
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > to change TopicConfigManager a
> > bit
> > > so
> > > > > > that
> > > > > > > it
> > > > > > > > > > > caches
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13
> PM,
> > > > Andrii
> > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Thanks for a great
> discussion!
> > > > > > > > > > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala
> > > requests
> > > > > > > > objects,
> > > > > > > > > > use
> > > > > > > > > > > > java
> > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > > > > > > > > >     A: Gwen kindly took that
> > > > > > > (KAFKA-1927).
> > > > > > > > > It's
> > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > speed
> > > > > > > > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > > > > > > > > >          there since this
> > ticket
> > > > > blocks
> > > > > > > > other
> > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > changes.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute
> > facility
> > > > vs
> > > > > > > client
> > > > > > > > > > > > > maintaining
> > > > > > > > > > > > > > > > > cluster
> > > > > > > > > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > > > > > > > > >     A: Jay has added pseudo
> > code
> > > to
> > > > > > > > > KAFKA-1912
> > > > > > > > > > -
> > > > > > > > > > > > need
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > > > > > > > > >         easy to implement as
> a
> > > > > > > server-side
> > > > > > > > > > > feature
> > > > > > > > > > > > > > > > (comments
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 3. Q: Controller field in
> wire
> > > > > > protocol.
> > > > > > > > > > > > > > > > > > > > > > > >     A: This might be useful
> for
> > > > > > clients,
> > > > > > > > add
> > > > > > > > > > this
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic
> creation
> > > > from
> > > > > > TMR.
> > > > > > > > > > > > > > > > > > > > > > > >     A: I will add proposed by
> > Jun
> > > > > > > solution
> > > > > > > > > > (using
> > > > > > > > > > > > > > > clientId
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 5. Q: Bumping new versions of
> > TMR
> > > > vs
> > > > > > > > grabbing
> > > > > > > > > > all
> > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > > > > >     A: It was decided to try
> to
> > > > > gather
> > > > > > > all
> > > > > > > > > > > changes
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > > > > > > > > >         In case of TMR it
> worth
> > > > > > checking:
> > > > > > > > > > > > KAFKA-2020
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to
> > > > > deserialize
> > > > > > > > > user's
> > > > > > > > > > > > input
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > CLI
> > > > > > > > > > > > > > > > > tool.
> > > > > > > > > > > > > > > > > > > > > > > >     A: Use jackson for that,
> > > /tools
> > > > > > > project
> > > > > > > > > is
> > > > > > > > > > a
> > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > jar so
> > > > > > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > 7.  Q:
> VerifyReassingPartitions
> > > vs
> > > > > > > generic
> > > > > > > > > > status
> > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > command.
> > > > > > > > > > > > > > > > > > > > > > > >      A: For long-running
> > requests
> > > > > like
> > > > > > > > > reassign
> > > > > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > > > > > > > > >          it makes sense to
> > > > introduce
> > > > > > it.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >  Please add, correct me if I
> > > missed
> > > > > > > > > something.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20
> > PM,
> > > > > Andrii
> > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > You are right, I removed
> > > > > > > ClusterMetadata
> > > > > > > > > > > because
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > > > > > > > > what we need in
> > TopicMetadata.
> > > > > Also,
> > > > > > as
> > > > > > > > Jay
> > > > > > > > > > > > pointed
> > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > earlier,
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > > would like to have
> > "orthogonal"
> > > > > API,
> > > > > > > but
> > > > > > > > at
> > > > > > > > > > the
> > > > > > > > > > > > > same
> > > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > But I like your idea and
> even
> > > > have
> > > > > > some
> > > > > > > > > other
> > > > > > > > > > > > > > arguments
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > > > > > > > > There is also
> > > > DescribeTopicRequest
> > > > > > > which
> > > > > > > > > was
> > > > > > > > > > > > > proposed
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > > > > > > > > it returns topic configs,
> > > > > partitions,
> > > > > > > > > > > replication
> > > > > > > > > > > > > > > factor
> > > > > > > > > > > > > > > > > plus
> > > > > > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > > > > > > > > leader replica. The later
> > part
> > > is
> > > > > > > really
> > > > > > > > > > > already
> > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > > > > > > > > So again we'll have to add
> > > stuff
> > > > to
> > > > > > > TMR,
> > > > > > > > > not
> > > > > > > > > > to
> > > > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > > > newly added requests.
> > However,
> > > > this
> > > > > > way
> > > > > > > > > we'll
> > > > > > > > > > > end
> > > > > > > > > > > > > up
> > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > > > > > > > > request which returns
> cluster
> > > > > > metadata,
> > > > > > > > > topic
> > > > > > > > > > > > > > > replication
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > > > > > > plus partition replication
> > > data.
> > > > > > Seems
> > > > > > > > > > logical
> > > > > > > > > > > to
> > > > > > > > > > > > > > split
> > > > > > > > > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > > > > > > > > - ClusterMetadata (brokers
> +
> > > > > > > controller,
> > > > > > > > > > maybe
> > > > > > > > > > > > smth
> > > > > > > > > > > > > > > else)
> > > > > > > > > > > > > > > > > > > > > > > > > - TopicMetadata (topic
> info +
> > > > > > partition
> > > > > > > > > > > details)
> > > > > > > > > > > > > > > > > > > > > > > > > But since current TMR is
> > > involved
> > > > > in
> > > > > > > lots
> > > > > > > > > of
> > > > > > > > > > > > places
> > > > > > > > > > > > > > > > > (including
> > > > > > > > > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > > > > > > > as I understand) this might
> > be
> > > > very
> > > > > > > > serious
> > > > > > > > > > > > change
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > > > > > > > > sense to stick with current
> > > > > approach.
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at
> 5:29
> > > PM,
> > > > > Joel
> > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> I may be missing some
> > context
> > > > but
> > > > > > > > > hopefully
> > > > > > > > > > > this
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > also be
> > > > > > > > > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > > > > > > > > >> today: I thought the
> earlier
> > > > > > proposal
> > > > > > > > > where
> > > > > > > > > > > > there
> > > > > > > > > > > > > > was
> > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > > > > > > > > >> ClusterMetadata request
> was
> > > > > clearer
> > > > > > > and
> > > > > > > > > > > > explicit.
> > > > > > > > > > > > > > > During
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > >> this thread I think the
> > > > conclusion
> > > > > > was
> > > > > > > > > that
> > > > > > > > > > > the
> > > > > > > > > > > > > main
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> controller information and
> > > that
> > > > > can
> > > > > > be
> > > > > > > > > > rolled
> > > > > > > > > > > > into
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > > >> response but that seems a
> > bit
> > > > > > > irrelevant
> > > > > > > > > to
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > metadata.
> > > > > > > > > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > > > > > > > > >> think the full broker-list
> > is
> > > > also
> > > > > > > > > > irrelevant
> > > > > > > > > > > to
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > > >> it is already there and in
> > > use.
> > > > I
> > > > > > > think
> > > > > > > > > > there
> > > > > > > > > > > is
> > > > > > > > > > > > > > still
> > > > > > > > > > > > > > > > > room
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > > > > >> explicit ClusterMetadata
> > > request
> > > > > > since
> > > > > > > > > there
> > > > > > > > > > > may
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > > > > > >> cluster-level information
> > that
> > > > we
> > > > > > may
> > > > > > > > want
> > > > > > > > > > to
> > > > > > > > > > > > add
> > > > > > > > > > > > > > over
> > > > > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> have nothing to do with
> > topic
> > > > > > > metadata).
> > > > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at
> > > > 02:45:30PM
> > > > > > > > +0200,
> > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > 101. Okay, if you say
> that
> > > > such
> > > > > > use
> > > > > > > > case
> > > > > > > > > > is
> > > > > > > > > > > > > > > > important. I
> > > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > > > > > >> > using clientId for these
> > > > > purposes
> > > > > > is
> > > > > > > > > fine
> > > > > > > > > > -
> > > > > > > > > > > if
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > > > > > > > > >> > as part of all Wire
> > protocol
> > > > > > > messages,
> > > > > > > > > why
> > > > > > > > > > > not
> > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > that.
> > > > > > > > > > > > > > > > > > > > > > > > >> > I will update KIP-4 page
> > if
> > > > > nobody
> > > > > > > has
> > > > > > > > > > other
> > > > > > > > > > > > > ideas
> > > > > > > > > > > > > > > > > (which
> > > > > > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update
> > the
> > > > KIP
> > > > > > > > > > > accordingly.
> > > > > > > > > > > > I
> > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > we can
> > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > > > > > > > > >> > fine-grained error codes
> > if
> > > > some
> > > > > > > error
> > > > > > > > > > code
> > > > > > > > > > > > > > received
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > > > > >> > won't give enough
> context
> > to
> > > > > > return
> > > > > > > a
> > > > > > > > > > > > > descriptive
> > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > message
> > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > Look forward to
> discussing
> > > all
> > > > > > > > > outstanding
> > > > > > > > > > > > > issues
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > detail
> > > > > > > > > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at
> > > 10:59
> > > > > PM,
> > > > > > > Jun
> > > > > > > > > Rao
> > > > > > > > > > <
> > > > > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > 101. There may be a
> use
> > > case
> > > > > > where
> > > > > > > > you
> > > > > > > > > > > only
> > > > > > > > > > > > > want
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > > > > > > > > >> > > manually by admins.
> > > > Currently,
> > > > > > you
> > > > > > > > can
> > > > > > > > > > do
> > > > > > > > > > > > that
> > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > > disabling
> > > > > > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> > > creation and issue
> topic
> > > > > > creation
> > > > > > > > from
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > TopicCommand.
> > > > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > > > > > > > > >> > > topic creation
> > completely
> > > on
> > > > > the
> > > > > > > > > broker
> > > > > > > > > > > and
> > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > have a
> > > > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > > > > > >> > > between topic creation
> > > > > requests
> > > > > > > from
> > > > > > > > > the
> > > > > > > > > > > > > regular
> > > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > can't support manual
> > topic
> > > > > > > creation
> > > > > > > > > any
> > > > > > > > > > > > more.
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > > > > > > > > >> > > way of distinguishing
> > the
> > > > > > clients
> > > > > > > > > making
> > > > > > > > > > > the
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > creation
> > > > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > using clientId. For
> > > example,
> > > > > the
> > > > > > > > admin
> > > > > > > > > > > tool
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > set
> > > > > > > > > > > > > > > > > it to
> > > > > > > > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > > > > > > > > >> > > admin and the broker
> can
> > > > treat
> > > > > > > that
> > > > > > > > > > > clientId
> > > > > > > > > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > Also, there is a
> related
> > > > > > > discussion
> > > > > > > > in
> > > > > > > > > > > > > > KAFKA-2020.
> > > > > > > > > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > following in
> > > > > > > TopicMetadataResponse:
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > 1. If leader is not
> > > > available,
> > > > > > we
> > > > > > > > set
> > > > > > > > > > the
> > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > 2. If a non-leader
> > replica
> > > > is
> > > > > > not
> > > > > > > > > > > available,
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > take
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > > > > > >> > > the assigned replica
> > list
> > > > and
> > > > > > isr
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > > > response.
> > > > > > > > > > > > > > > > As
> > > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > > > > > > > > >> > > doing that, we set the
> > > > > partition
> > > > > > > > level
> > > > > > > > > > > error
> > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > This has a few
> problems.
> > > > > First,
> > > > > > > > > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > > > > > > > > >> > > an error, at least for
> > the
> > > > > > normal
> > > > > > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > > > > > > > > >> > > to find out the
> leader.
> > > > > Second,
> > > > > > it
> > > > > > > > can
> > > > > > > > > > > > happen
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > both
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > another replica are
> not
> > > > > > available
> > > > > > > at
> > > > > > > > > the
> > > > > > > > > > > > same
> > > > > > > > > > > > > > > time.
> > > > > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > > > > > >> > > to indicate both.
> Third,
> > > > even
> > > > > > if a
> > > > > > > > > > replica
> > > > > > > > > > > > is
> > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > available,
> > > > > > > > > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > > > > > > > > >> > > useful to return its
> > > replica
> > > > > id
> > > > > > > > since
> > > > > > > > > > some
> > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > (e.g.
> > > > > > > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > One way to address
> this
> > > > issue
> > > > > is
> > > > > > > to
> > > > > > > > > > always
> > > > > > > > > > > > > > return
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > > > > > > > > >> > > leader, assigned
> > replicas,
> > > > and
> > > > > > isr
> > > > > > > > > > > > regardless
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > > > > > > > > >> > > broker is live or not.
> > > Since
> > > > > we
> > > > > > > also
> > > > > > > > > > > return
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > list
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > client can figure out
> > > > whether
> > > > > a
> > > > > > > > leader
> > > > > > > > > > or
> > > > > > > > > > > a
> > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > > > > > > > > >> > > accordingly. This way,
> > we
> > > > > don't
> > > > > > > need
> > > > > > > > > to
> > > > > > > > > > > set
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > > > > > >> > > when the leader or a
> > > replica
> > > > > is
> > > > > > > not
> > > > > > > > > > > > available.
> > > > > > > > > > > > > > > This
> > > > > > > > > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > > > > > > > > >> > > protocol, but does
> > change
> > > > the
> > > > > > > > > semantics.
> > > > > > > > > > > > Since
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > > > > > > > > >> > > of
> TopicMetadataRequest
> > > > here,
> > > > > we
> > > > > > > can
> > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > piggyback
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > 102.1 For those types
> of
> > > > > errors
> > > > > > > due
> > > > > > > > to
> > > > > > > > > > > > invalid
> > > > > > > > > > > > > > > > input,
> > > > > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > > >> > > guard it at parameter
> > > > > validation
> > > > > > > > time
> > > > > > > > > > and
> > > > > > > > > > > > > throw
> > > > > > > > > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > > > > > > > > >> > > without even sending
> the
> > > > > request
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > > broker?
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015
> at
> > > > 10:37
> > > > > > AM,
> > > > > > > > > Andrii
> > > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > andrii.biletskyi@stealth.ly
> > > > >
> > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > Answering your
> > > questions:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > 101. If I understand
> > you
> > > > > > > > correctly,
> > > > > > > > > > you
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > saying
> > > > > > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > will be ported to
> > > TMR_V1)
> > > > > > won't
> > > > > > > be
> > > > > > > > > > able
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > automatically
> > > > > > > > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > unconditionally
> remove
> > > > topic
> > > > > > > > > creation
> > > > > > > > > > > from
> > > > > > > > > > > > > > > there).
> > > > > > > > > > > > > > > > > But
> > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > Ok, about your
> > proposal:
> > > > I'm
> > > > > > > not a
> > > > > > > > > big
> > > > > > > > > > > fan
> > > > > > > > > > > > > > too,
> > > > > > > > > > > > > > > > > when it
> > > > > > > > > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > clients directly in
> > > > protocol
> > > > > > > > schema.
> > > > > > > > > > And
> > > > > > > > > > > > > also
> > > > > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > auto.create.topics.enable
> > > > > is a
> > > > > > > > > server
> > > > > > > > > > > side
> > > > > > > > > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > in future versions,
> > add
> > > > this
> > > > > > > > setting
> > > > > > > > > > to
> > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > based
> > > > > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > UnknownTopic create
> > > topic
> > > > > > > > explicitly
> > > > > > > > > > by
> > > > > > > > > > > a
> > > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's
> > > > because
> > > > > > we
> > > > > > > > want
> > > > > > > > > > to
> > > > > > > > > > > > > > support
> > > > > > > > > > > > > > > > > batching
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > want to give
> > descriptive
> > > > > error
> > > > > > > > > > messages
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > clients.
> > > > > > > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > to construct such
> > > messages
> > > > > > (e.g.
> > > > > > > > > > > > AdminClient
> > > > > > > > > > > > > > > layer
> > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > means two cases:
> > either
> > > > > > invalid
> > > > > > > > > > number -
> > > > > > > > > > > > > e.g.
> > > > > > > > > > > > > > > -1;
> > > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > partitions argument
> > > > wasn't)
> > > > > -
> > > > > > I
> > > > > > > > > > wrapped
> > > > > > > > > > > > > > > responses
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > other ideas, this
> was
> > > just
> > > > > > > initial
> > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree.
> > > I'll
> > > > > > change
> > > > > > > > > that
> > > > > > > > > > to
> > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015
> > at
> > > > 7:16
> > > > > > PM,
> > > > > > > > Jun
> > > > > > > > > > > Rao <
> > > > > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 101. That's what I
> > was
> > > > > > > thinking
> > > > > > > > > too,
> > > > > > > > > > > but
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > not be
> > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we can let it not
> > > > trigger
> > > > > > auto
> > > > > > > > > topic
> > > > > > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > Then,
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > > > > > > > > UnknownTopicException,
> > > > > > > > > > > it
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > createTopicRequest
> > for
> > > > > auto
> > > > > > > > topic
> > > > > > > > > > > > > creation.
> > > > > > > > > > > > > > On
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > never issue
> > > > > > > createTopicRequest.
> > > > > > > > > This
> > > > > > > > > > > > works
> > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > enabled on the
> > broker
> > > > > side.
> > > > > > > > > > However, I
> > > > > > > > > > > > am
> > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > sure how
> > > > > > > > > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > when auto topic
> > > creation
> > > > > is
> > > > > > > > > disabled
> > > > > > > > > > > on
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > side.
> > > > > > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > want to have a way
> > to
> > > > > > manually
> > > > > > > > > > create
> > > > > > > > > > > a
> > > > > > > > > > > > > > topic,
> > > > > > > > > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > commands. However,
> > > then
> > > > we
> > > > > > > need
> > > > > > > > a
> > > > > > > > > > way
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > issued from the
> > > producer
> > > > > > > clients
> > > > > > > > > and
> > > > > > > > > > > the
> > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > tools.
> > > > > > > > > > > > > > > > > > > > May
> > > > > > > > > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > new field in
> > > > > > > createTopicRequest
> > > > > > > > > and
> > > > > > > > > > > set
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > differently
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > client and the
> admin
> > > > > client.
> > > > > > > > > > However,
> > > > > > > > > > > I
> > > > > > > > > > > > am
> > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > sure if
> > > > > > > > > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 2. Yes,
> refactoring
> > > > > existing
> > > > > > > > > > requests
> > > > > > > > > > > > is a
> > > > > > > > > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > posted some
> comments
> > > in
> > > > > > > > > KAFKA-1927.
> > > > > > > > > > We
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > first, before
> adding
> > > the
> > > > > new
> > > > > > > > logic
> > > > > > > > > > in
> > > > > > > > > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102. About the
> > > > > AdminClient:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit
> > > weird
> > > > > that
> > > > > > > we
> > > > > > > > > > return
> > > > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we should either
> > > return
> > > > > > error
> > > > > > > > code
> > > > > > > > > > or
> > > > > > > > > > > > > throw
> > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.2. We probably
> > > > > shouldn't
> > > > > > > > > > > explicitly
> > > > > > > > > > > > > use
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Not every request
> > > > > evolution
> > > > > > > > > requires
> > > > > > > > > > > an
> > > > > > > > > > > > > api
> > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13,
> 2015
> > > at
> > > > > 4:08
> > > > > > > AM,
> > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > andrii.biletskyi@stealth.ly
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks for you
> > > > comments.
> > > > > > > > Answers
> > > > > > > > > > > > inline:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 100. There are a
> > few
> > > > > > fields
> > > > > > > > such
> > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > > > > PartitionsSerialized
> > > > > > > > that
> > > > > > > > > > are
> > > > > > > > > > > > > > > > represented
> > > > > > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
> > > structures
> > > > > in
> > > > > > > > json.
> > > > > > > > > > > Could
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
> > > definition
> > > > as
> > > > > > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, now with
> > Admin
> > > > > Client
> > > > > > > > this
> > > > > > > > > > > looks
> > > > > > > > > > > > a
> > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > weird.
> > > > > > > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > ReassignPartitionCommand
> > > > > > > > accepts
> > > > > > > > > > > input
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > json,
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > interfaces
> > > unchanged,
> > > > > > where
> > > > > > > > > > > possible.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > If we port it to
> > > > > > > deserialized
> > > > > > > > > > > format,
> > > > > > > > > > > > in
> > > > > > > > > > > > > > CLI
> > > > > > > > > > > > > > > > > (/tools
> > > > > > > > > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json library
> since
> > > > > /tools
> > > > > > is
> > > > > > > > > > written
> > > > > > > > > > > > in
> > > > > > > > > > > > > > java
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > provided by a
> > user.
> > > > Can
> > > > > we
> > > > > > > > > quickly
> > > > > > > > > > > > agree
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > what
> > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON,
> > > > > whatever)?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> > > > > > > TopicMetadataRequest
> > > > > > > > > v1
> > > > > > > > > > > > still
> > > > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit
> > > weird
> > > > > now
> > > > > > > that
> > > > > > > > > we
> > > > > > > > > > > > have a
> > > > > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought
> > about
> > > > how
> > > > > > the
> > > > > > > > new
> > > > > > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be
> used
> > in
> > > > the
> > > > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For
> > > example,
> > > > > > > ideally,
> > > > > > > > > we
> > > > > > > > > > > > don't
> > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to
> > > trigger
> > > > > auto
> > > > > > > > topic
> > > > > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > I agree, this
> > > strange
> > > > > > logic
> > > > > > > > > should
> > > > > > > > > > > be
> > > > > > > > > > > > > > fixed.
> > > > > > > > > > > > > > > > > I'm not
> > > > > > > > > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > correct me if
> I'm
> > > > wrong,
> > > > > > but
> > > > > > > > it
> > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > look
> > > > > > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > leverage
> > AdminClient
> > > > for
> > > > > > > that
> > > > > > > > in
> > > > > > > > > > > > > Producer
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > creation from
> the
> > > > > > > > > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay
> > meant
> > > > > > getting
> > > > > > > > rid
> > > > > > > > > > of
> > > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > > > > > HeartbeatRequestAndHeader
> > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >>
> HeartbeatResponseAndHeader.
> > We
> > > > > > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap
> > > thing
> > > > > when
> > > > > > > > > adding
> > > > > > > > > > > the
> > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the
> > long
> > > > term
> > > > > > > plan
> > > > > > > > is
> > > > > > > > > > to
> > > > > > > > > > > > get
> > > > > > > > > > > > > > rid
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> > > > request/response
> > > > > in
> > > > > > > the
> > > > > > > > > > > client.
> > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant
> > number
> > > > of
> > > > > > new
> > > > > > > > > > > requests,
> > > > > > > > > > > > > > > perhaps
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the
> > > > existing
> > > > > > > scala
> > > > > > > > > > > requests
> > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks like
> I
> > > > > > > > misunderstood
> > > > > > > > > > the
> > > > > > > > > > > > > point
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > rework that. The
> > > only
> > > > > > thing
> > > > > > > is
> > > > > > > > > > that
> > > > > > > > > > > I
> > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > see
> > > > > > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > least one
> existing
> > > > > > protocol
> > > > > > > > > > message.
> > > > > > > > > > > > > Thus,
> > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are going to do
> > it.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Re porting all
> > > > existing
> > > > > > > RQ/RP
> > > > > > > > in
> > > > > > > > > > > this
> > > > > > > > > > > > > > patch.
> > > > > > > > > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > requirement to
> > have
> > > > > Admin
> > > > > > > KIP
> > > > > > > > > > done,
> > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > afraid
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > There are 13
> > > protocol
> > > > > > > messages
> > > > > > > > > and
> > > > > > > > > > > all
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > intensive manual
> > > > > testing,
> > > > > > > no?
> > > > > > > > > I'm
> > > > > > > > > > > > afraid
> > > > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > > > not the
> > > > > > > > > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka core
> > internals
> > > > :).
> > > > > > Let
> > > > > > > > me
> > > > > > > > > > know
> > > > > > > > > > > > > your
> > > > > > > > > > > > > > > > > thoughts
> > > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > follow-up this
> > > issue (
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13,
> > 2015
> > > > at
> > > > > > 6:40
> > > > > > > > AM,
> > > > > > > > > > Jun
> > > > > > > > > > > > > Rao <
> > > > > > > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > A few more
> > > comments.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 100. There
> are a
> > > few
> > > > > > > fields
> > > > > > > > > such
> > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > > > > PartitionsSerialized
> > > > > > > > that
> > > > > > > > > > are
> > > > > > > > > > > > > > > > represented
> > > > > > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
> > > structures
> > > > > in
> > > > > > > > json.
> > > > > > > > > > > Could
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
> > > definition
> > > > as
> > > > > > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> > > > > > > > TopicMetadataRequest
> > > > > > > > > > v1
> > > > > > > > > > > > > still
> > > > > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit
> > > weird
> > > > > now
> > > > > > > that
> > > > > > > > > we
> > > > > > > > > > > > have a
> > > > > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought
> > about
> > > > how
> > > > > > the
> > > > > > > > new
> > > > > > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be
> used
> > in
> > > > the
> > > > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For
> > > example,
> > > > > > > ideally,
> > > > > > > > > we
> > > > > > > > > > > > don't
> > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to
> > > trigger
> > > > > auto
> > > > > > > > topic
> > > > > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay
> > > meant
> > > > > > > getting
> > > > > > > > > rid
> > > > > > > > > > > of
> > > > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > > > > > HeartbeatRequestAndHeader
> > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >>
> HeartbeatResponseAndHeader.
> > We
> > > > > > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap
> > > thing
> > > > > when
> > > > > > > > > adding
> > > > > > > > > > > the
> > > > > > > > > > > > > new
> > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the
> > long
> > > > term
> > > > > > > plan
> > > > > > > > is
> > > > > > > > > > to
> > > > > > > > > > > > get
> > > > > > > > > > > > > > rid
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> > > > request/response
> > > > > in
> > > > > > > the
> > > > > > > > > > > client.
> > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant
> > number
> > > > of
> > > > > > new
> > > > > > > > > > > requests,
> > > > > > > > > > > > > > > perhaps
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the
> > > > existing
> > > > > > > scala
> > > > > > > > > > > requests
> > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar
> 12,
> > > 2015
> > > > > at
> > > > > > > 3:37
> > > > > > > > > PM,
> > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > andrii.biletskyi@stealth.ly
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > As said
> above
> > -
> > > I
> > > > > list
> > > > > > > > again
> > > > > > > > > > all
> > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can see
> what's
> > > > left
> > > > > > and
> > > > > > > > > > finalize
> > > > > > > > > > > > all
> > > > > > > > > > > > > > > > pending
> > > > > > > > > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments
> from
> > > Jay:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is
> > much
> > > > > needed
> > > > > > > > > > > > > functionality,
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > really think
> > > these
> > > > > > > > protocols
> > > > > > > > > > > > > through.
> > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > of well
> > > > thought-out,
> > > > > > > > > > orthoganol
> > > > > > > > > > > > > apis.
> > > > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > important to
> > > think
> > > > > > > through
> > > > > > > > > the
> > > > > > > > > > > end
> > > > > > > > > > > > > > state
> > > > > > > > > > > > > > > > > even if
> > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > won't
> > implement
> > > in
> > > > > the
> > > > > > > > first
> > > > > > > > > > > > phase.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A:
> Definitely
> > > > behind
> > > > > > > this.
> > > > > > > > > > Would
> > > > > > > > > > > > > > > > appreciate
> > > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > how this can
> > be
> > > > > > > improved.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's
> > please
> > > > > please
> > > > > > > > > please
> > > > > > > > > > > wait
> > > > > > > > > > > > > > until
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to the new
> > java
> > > > > > protocol
> > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > objects that
> > is
> > > > just
> > > > > > > > > > generating
> > > > > > > > > > > > more
> > > > > > > > > > > > > > > work
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in
> > the
> > > > > latest
> > > > > > > > > patch -
> > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This
> > proposal
> > > > > > > > introduces
> > > > > > > > > a
> > > > > > > > > > > new
> > > > > > > > > > > > > type
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > inconsistent
> > > with
> > > > > > > > everything
> > > > > > > > > > > else
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > other marker
> > > > value.
> > > > > > You
> > > > > > > > > could
> > > > > > > > > > > > argue
> > > > > > > > > > > > > > > either
> > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > for
> > consistency.
> > > > For
> > > > > > > > clients
> > > > > > > > > > > that
> > > > > > > > > > > > > > > > > implemented
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than our
> scala
> > > > code
> > > > > > > these
> > > > > > > > > > basic
> > > > > > > > > > > > > > > primitives
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in
> > the
> > > > > latest
> > > > > > > > > patch -
> > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > MaybeOf
> > > > > > > > > > > > > > > > > > > > type
> > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 4.
> > > > ClusterMetadata:
> > > > > > This
> > > > > > > > > seems
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers,
> > topics,
> > > > and
> > > > > > > > > > > partitions. I
> > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > ClusterMetadataRequest
> > > > > > > (or
> > > > > > > > > > just
> > > > > > > > > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > controller.
> Or
> > > are
> > > > > > there
> > > > > > > > > other
> > > > > > > > > > > > > things
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I agree.
> > > > Updated
> > > > > > the
> > > > > > > > KIP.
> > > > > > > > > > > Let's
> > > > > > > > > > > > > > > extends
> > > > > > > > > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > include
> > > > controller.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a
> > > > > tendency
> > > > > > to
> > > > > > > > try
> > > > > > > > > > to
> > > > > > > > > > > > > make a
> > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > particular
> > > nodes.
> > > > > This
> > > > > > > > adds
> > > > > > > > > a
> > > > > > > > > > > lot
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > burden
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > sounds easy
> > but
> > > > each
> > > > > > > > > discovery
> > > > > > > > > > > can
> > > > > > > > > > > > > > fail
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > many
> > > > > > > > > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > full state
> > > machine
> > > > > to
> > > > > > do
> > > > > > > > > > > right). I
> > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > commands and
> > > > ideally
> > > > > > as
> > > > > > > > many
> > > > > > > > > > of
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers and
> > just
> > > > > > > redirect
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > there would
> > be a
> > > > > > general
> > > > > > > > way
> > > > > > > > > > to
> > > > > > > > > > > > > > > > encapsulate
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a
> very
> > > > > > > interesting
> > > > > > > > > > idea,
> > > > > > > > > > > > but
> > > > > > > > > > > > > > > seems
> > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > feature
> (like
> > > > > > > performance
> > > > > > > > > > > > > > > considerations,
> > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I believe
> this
> > > > > > shouldn't
> > > > > > > > be
> > > > > > > > > a
> > > > > > > > > > > > > blocker.
> > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point it
> won't
> > > > > affect
> > > > > > > > Admin
> > > > > > > > > > > > changes
> > > > > > > > > > > > > -
> > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > least
> > > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
> > > required.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We should
> > > > > probably
> > > > > > > > > > normalize
> > > > > > > > > > > > the
> > > > > > > > > > > > > > key
> > > > > > > > > > > > > > > > > value
> > > > > > > > > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
> > embedding a
> > > > new
> > > > > > > > > > formatting.
> > > > > > > > > > > > So
> > > > > > > > > > > > > > two
> > > > > > > > > > > > > > > > > strings
> > > > > > > > > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > internal
> > equals
> > > > > sign.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in
> > the
> > > > > latest
> > > > > > > > > patch -
> > > > > > > > > > > > > > > normalized
> > > > > > > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the
> > > > > > postcondition
> > > > > > > of
> > > > > > > > > > these
> > > > > > > > > > > > > APIs
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the command
> > has
> > > > been
> > > > > > > > > > completed?
> > > > > > > > > > > It
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > been
> completed
> > > so
> > > > > you
> > > > > > > know
> > > > > > > > > > that
> > > > > > > > > > > if
> > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > create a
> > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it you won't
> > get
> > > > an
> > > > > > > > > exception
> > > > > > > > > > > > about
> > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > being
> > > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: For long
> > > > running
> > > > > > > > requests
> > > > > > > > > > > (like
> > > > > > > > > > > > > > > > reassign
> > > > > > > > > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command has
> > > begun
> > > > -
> > > > > so
> > > > > > > we
> > > > > > > > > > don't
> > > > > > > > > > > > > block
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > client.
> > > > > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
> > commands,
> > > > this
> > > > > > > will
> > > > > > > > be
> > > > > > > > > > > > > > refactored
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > immediately,
> > > since
> > > > > the
> > > > > > > > > > > Controller
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > serve
> > > > > > > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up
> > > ticket
> > > > > > > > > KAFKA-1777).
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe
> > > topic
> > > > > and
> > > > > > > list
> > > > > > > > > > > topics
> > > > > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request. Is
> > > there
> > > > a
> > > > > > > reason
> > > > > > > > > to
> > > > > > > > > > > give
> > > > > > > > > > > > > > back
> > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like if we
> > just
> > > > make
> > > > > > the
> > > > > > > > > > > > > > post-condition
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic is
> > deleted
> > > > > that
> > > > > > > will
> > > > > > > > > get
> > > > > > > > > > > rid
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be much more
> > > > > > intuitive.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in
> > the
> > > > > latest
> > > > > > > > > patch -
> > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > ListTopicsRequest.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we
> > > > > consider
> > > > > > > > > batching
> > > > > > > > > > > > these
> > > > > > > > > > > > > > > > > requests?
> > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > allow
> multiple
> > > > > > > operations
> > > > > > > > to
> > > > > > > > > > be
> > > > > > > > > > > > > > batched.
> > > > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we will get
> a
> > > lot
> > > > of
> > > > > > > code
> > > > > > > > > that
> > > > > > > > > > > > does
> > > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> for(topic:
> > > > > > > > > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this code
> will
> > > > work
> > > > > > > great
> > > > > > > > > when
> > > > > > > > > > > you
> > > > > > > > > > > > > > test
> > > > > > > > > > > > > > > > on 5
> > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > you have
> 50k.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated
> the
> > > > KIP -
> > > > > > > > please
> > > > > > > > > > > check
> > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I think
> we
> > > > > should
> > > > > > > also
> > > > > > > > > > > discuss
> > > > > > > > > > > > > how
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > client api
> for
> > > > these
> > > > > > > > > > operations.
> > > > > > > > > > > > > > > Currently
> > > > > > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > is totally
> > > > sketchy.
> > > > > I
> > > > > > > > think
> > > > > > > > > we
> > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > that exposes
> > > > > > > > administrative
> > > > > > > > > > > > > > > functionality.
> > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > properly
> test
> > > the
> > > > > new
> > > > > > > > apis,
> > > > > > > > > I
> > > > > > > > > > > > > suspect.
> > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated
> the
> > > > KIP -
> > > > > > > > please
> > > > > > > > > > > check
> > > > > > > > > > > > > > "Admin
> > > > > > > > > > > > > > > > > Client"
> > > > > > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > API
> proposal.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The
> other
> > > > > > > information
> > > > > > > > > that
> > > > > > > > > > > > would
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > information
> > > about
> > > > > > > > > > > partitions--how
> > > > > > > > > > > > > much
> > > > > > > > > > > > > > > > data
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the segment
> > > > offsets,
> > > > > > > what
> > > > > > > > is
> > > > > > > > > > the
> > > > > > > > > > > > > > log-end
> > > > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> compaction
> > > > > point,
> > > > > > > > etc. I
> > > > > > > > > > > think
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > done
> > > > > > > > > > > > > > > > > > > right
> > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > successor to
> > the
> > > > > very
> > > > > > > > > awkward
> > > > > > > > > > > > > > > > OffsetRequest
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > > > > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be resolved
> > in a
> > > > > > > separate
> > > > > > > > > KIP
> > > > > > > > > > /
> > > > > > > > > > > > jira
> > > > > > > > > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 12.
> Generally
> > we
> > > > can
> > > > > > do
> > > > > > > > good
> > > > > > > > > > > error
> > > > > > > > > > > > > > > > handling
> > > > > > > > > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > messages.
> I.e.
> > > > > > generally
> > > > > > > > the
> > > > > > > > > > > > client
> > > > > > > > > > > > > > has
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > context
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > an error
> that
> > > the
> > > > > > topic
> > > > > > > > > > doesn't
> > > > > > > > > > > > > exist
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > say
> > > > > > > > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than "error
> > code
> > > > 14"
> > > > > > (or
> > > > > > > > > > > > whatever).
> > > > > > > > > > > > > > > Maybe
> > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this is
> hard?
> > If
> > > > we
> > > > > > want
> > > > > > > > to
> > > > > > > > > > add
> > > > > > > > > > > > > > > > server-side
> > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > need to do
> > this
> > > > in a
> > > > > > > > > > consistent
> > > > > > > > > > > > way
> > > > > > > > > > > > > > > across
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated
> the
> > > > KIP -
> > > > > > > > please
> > > > > > > > > > > check
> > > > > > > > > > > > > > > > "Protocol
> > > > > > > > > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> comprehensive,
> > > > > > > > fine-grained
> > > > > > > > > > list
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments
> from
> > > > > > Guozhang:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe
> > > topic
> > > > > > > > request:
> > > > > > > > > it
> > > > > > > > > > > > would
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > great to
> > > > > > > > > > > > > > > > > > > > go
> > > > > > > > > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name
> > regex
> > > > for
> > > > > > > this
> > > > > > > > > > > request.
> > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > example,
> > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the topic
> > > command
> > > > is
> > > > > > to
> > > > > > > > list
> > > > > > > > > > all
> > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name
> > regex
> > > > > then
> > > > > > we
> > > > > > > > > have
> > > > > > > > > > to
> > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > description
> > info
> > > > and
> > > > > > > then
> > > > > > > > > > filter
> > > > > > > > > > > > at
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > burden on
> ZK.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config
> > K-Vs
> > > in
> > > > > > > create
> > > > > > > > > > topic:
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > related
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > maybe we can
> > add
> > > > > > another
> > > > > > > > > > > metadata
> > > > > > > > > > > > > K-V
> > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > with config
> > K-V
> > > in
> > > > > > > create
> > > > > > > > > > topic
> > > > > > > > > > > > like
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > field can be
> > > quite
> > > > > > > useful
> > > > > > > > in
> > > > > > > > > > > > storing
> > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > who issue
> the
> > > > create
> > > > > > > > > command,
> > > > > > > > > > > etc,
> > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant
> > > > > setting.
> > > > > > > Then
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > on regex of
> > the
> > > > > > metadata
> > > > > > > > > > field.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: As
> > discussed
> > > it
> > > > > is
> > > > > > > very
> > > > > > > > > > > > > interesting
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we have some
> > > basic
> > > > > > > > > > functionality
> > > > > > > > > > > > > > there.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today
> all
> > > the
> > > > > > admin
> > > > > > > > > > > operations
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > async in
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > return once
> it
> > > is
> > > > > > > written
> > > > > > > > in
> > > > > > > > > > ZK,
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > is why
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > > > > > > > > testUtil.waitForTopicCreated()
> > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request,
> etc.
> > > With
> > > > > > admin
> > > > > > > > > > > requests
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > synchronous
> > > > > requests;
> > > > > > > when
> > > > > > > > > it
> > > > > > > > > > is
> > > > > > > > > > > > > > turned
> > > > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > until the
> > > request
> > > > > has
> > > > > > > been
> > > > > > > > > > > > > completed.
> > > > > > > > > > > > > > > And
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > "token"
> field
> > in
> > > > the
> > > > > > > > > response,
> > > > > > > > > > > and
> > > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > verification
> > > > > request"
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > > > given
> > > > > > > > > > > > > > > token
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > has been
> > > > completed.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I see
> your
> > > > point.
> > > > > > My
> > > > > > > > idea
> > > > > > > > > > was
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long running
> > > > > request,
> > > > > > > > where
> > > > > > > > > > > > needed.
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > do it
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > concern is
> > that
> > > > > > > > introducing
> > > > > > > > > a
> > > > > > > > > > > > token
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > again
> > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to do
> similar
> > > > thing
> > > > > > > > > > introducing
> > > > > > > > > > > > > single
> > > > > > > > > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this idea
> > > because
> > > > we
> > > > > > > > wanted
> > > > > > > > > to
> > > > > > > > > > > > have
> > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > choice
> > between:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > a) have
> fixed
> > > > schema
> > > > > > but
> > > > > > > > > > > introduce
> > > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > > time new
> > > > > > > > > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long-running
> > > > > requests
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > b) use one
> > > request
> > > > > for
> > > > > > > > > > > > verification
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > generalize
> > > > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine
> with
> > > > > whatever
> > > > > > > > > > decision
> > > > > > > > > > > > > > > community
> > > > > > > > > > > > > > > > > come
> > > > > > > > > > > > > > > > > > > to.
> > > > > > > > > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comment from
> > > Gwen:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 16.
> > Specifically
> > > > for
> > > > > > > > > > ownership,
> > > > > > > > > > > I
> > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like you are
> > > > > > describing
> > > > > > > > ACL)
> > > > > > > > > > via
> > > > > > > > > > > > an
> > > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I remember
> > > KIP-11
> > > > > > > > described
> > > > > > > > > > > this,
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no
> > > > problem.
> > > > > > Not
> > > > > > > > > sure
> > > > > > > > > > > > though
> > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
> > > committed
> > > > > > first
> > > > > > > > and
> > > > > > > > > > > > include
> > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I
> > added
> > > > this
> > > > > > > note
> > > > > > > > to
> > > > > > > > > > > "Open
> > > > > > > > > > > > > > > > > Questions"
> > > > > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii
> > Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar
> > 13,
> > > > 2015
> > > > > > at
> > > > > > > > > 12:34
> > > > > > > > > > > AM,
> > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > andrii.biletskyi@stealth.ly
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Today I
> > > uploaded
> > > > > the
> > > > > > > > patch
> > > > > > > > > > > that
> > > > > > > > > > > > > > covers
> > > > > > > > > > > > > > > > > some of
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - removed
> > > > MaybeOf
> > > > > > > > optional
> > > > > > > > > > > type
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - switched
> > to
> > > > java
> > > > > > > > > protocol
> > > > > > > > > > > > > > > definitions
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > -
> simplified
> > > > > > messages
> > > > > > > > > > > > (normalized
> > > > > > > > > > > > > > > > configs,
> > > > > > > > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I also
> > updated
> > > > the
> > > > > > > KIP-4
> > > > > > > > > > with
> > > > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > proposal
> for
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > pending
> > items:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch
> > Admin
> > > > > > > Operations
> > > > > > > > > ->
> > > > > > > > > > > > > updated
> > > > > > > > > > > > > > > Wire
> > > > > > > > > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove
> > > > > > > ClusterMetadata
> > > > > > > > > ->
> > > > > > > > > > > > > changed
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > extend
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin
> > Client
> > > > ->
> > > > > > > > updated
> > > > > > > > > my
> > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > proposal
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Error
> > codes
> > > ->
> > > > > > > > proposed
> > > > > > > > > > > > > > fine-grained
> > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > AdminRequestFailed
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I will
> also
> > > > send a
> > > > > > > > > separate
> > > > > > > > > > > > email
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > cover all
> > > > > > > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii
> > > Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu,
> Mar
> > > 12,
> > > > > 2015
> > > > > > > at
> > > > > > > > > 9:26
> > > > > > > > > > > PM,
> > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Found
> > KIP-11
> > > (
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> It
> actually
> > > > > > specifies
> > > > > > > > > > changes
> > > > > > > > > > > > to
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs
> > are
> > > > > > > > consistent
> > > > > > > > > in
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > regard
> > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu,
> Mar
> > > 12,
> > > > > > 2015
> > > > > > > at
> > > > > > > > > > 12:21
> > > > > > > > > > > > PM,
> > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > Specifically
> > > > > for
> > > > > > > > > > > ownership, I
> > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > like
> you
> > > are
> > > > > > > > describing
> > > > > > > > > > > ACL)
> > > > > > > > > > > > > via
> > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I
> > remember
> > > > > KIP-11
> > > > > > > > > > described
> > > > > > > > > > > > > this,
> > > > > > > > > > > > > > > > but I
> > > > > > > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > Regardless, I
> > > > > > think
> > > > > > > > > KIP-4
> > > > > > > > > > > > > focuses
> > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > exists
> > from
> > > > > Kafka
> > > > > > > > > > brokers,
> > > > > > > > > > > > not
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > should
> > > exist
> > > > > but
> > > > > > > > > doesn't
> > > > > > > > > > > yet?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu,
> > Mar
> > > > 12,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > 6:37
> > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just
> > want
> > > to
> > > > > > > > > elaborate a
> > > > > > > > > > > bit
> > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > describe-topic
> > > > > > > based
> > > > > > > > > on
> > > > > > > > > > > > > config /
> > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on
> > > > KAFKA-1694.
> > > > > > The
> > > > > > > > > main
> > > > > > > > > > > > > > motivation
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> management
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > mechanisms,
> > > > > > which
> > > > > > > I
> > > > > > > > > > think
> > > > > > > > > > > is
> > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > architecture:
> > > > > > > today
> > > > > > > > > > anyone
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> cluster,
> > but
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> there
> is
> > > no
> > > > > > > concept
> > > > > > > > or
> > > > > > > > > > > > > > "ownership"
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> users.
> > For
> > > > > > > example,
> > > > > > > > at
> > > > > > > > > > > > > LinkedIn
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some
> > > casual
> > > > > > topic
> > > > > > > > name
> > > > > > > > > > > > prefix,
> > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > is a
> > > > > > > > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we
> scale
> > > our
> > > > > > > > > customers.
> > > > > > > > > > It
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> Describe
> > > all
> > > > > > > topics
> > > > > > > > > that
> > > > > > > > > > > is
> > > > > > > > > > > > > > > created
> > > > > > > > > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> Describe
> > > all
> > > > > > > topics
> > > > > > > > > > whose
> > > > > > > > > > > > > > > retention
> > > > > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> Describe
> > > all
> > > > > > > topics
> > > > > > > > > > whose
> > > > > > > > > > > > > > writable
> > > > > > > > > > > > > > > > > group
> > > > > > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> related
> to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > authorization),
> > > > > > > > etc..
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One
> > > possible
> > > > > way
> > > > > > > to
> > > > > > > > > > > achieve
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > create-topic
> > > > > > > > request,
> > > > > > > > > > > whose
> > > > > > > > > > > > > > value
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic;
> > > then
> > > > > > > > > > > describe-topics
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > choose to
> > > > > > > > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex,
> > 2)
> > > > > config
> > > > > > > K-V
> > > > > > > > > > > > matching,
> > > > > > > > > > > > > > 3)
> > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > Thoughts?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> Guozhang
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On
> Thu,
> > > Mar
> > > > 5,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > 4:37
> > > > > > > > > > > > > PM,
> > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> Thanks
> > > for
> > > > > the
> > > > > > > > > updated
> > > > > > > > > > > > wiki.
> > > > > > > > > > > > > A
> > > > > > > > > > > > > > > few
> > > > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1.
> > Error
> > > > > > > > description
> > > > > > > > > in
> > > > > > > > > > > > > > > response: I
> > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> several
> > > > > > different
> > > > > > > > > error
> > > > > > > > > > > > cases
> > > > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> codes.
> > In
> > > > > > general
> > > > > > > > the
> > > > > > > > > > > > > errorCode
> > > > > > > > > > > > > > > > > itself
> > > > > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> sufficient
> > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > describing
> > > > > the
> > > > > > > > server
> > > > > > > > > > > side
> > > > > > > > > > > > > > > errors.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2.
> > > Describe
> > > > > > topic
> > > > > > > > > > > request:
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> on
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic
> > > name
> > > > > > regex
> > > > > > > > for
> > > > > > > > > > this
> > > > > > > > > > > > > > > request.
> > > > > > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the
> > topic
> > > > > > command
> > > > > > > > is
> > > > > > > > > to
> > > > > > > > > > > > list
> > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic
> > > name
> > > > > > regex
> > > > > > > > then
> > > > > > > > > > we
> > > > > > > > > > > > have
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > description
> > > > > > info
> > > > > > > > and
> > > > > > > > > > then
> > > > > > > > > > > > > > filter
> > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> burden
> > on
> > > > ZK.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3.
> > Config
> > > > > K-Vs
> > > > > > in
> > > > > > > > > > create
> > > > > > > > > > > > > topic:
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe
> > we
> > > > can
> > > > > > add
> > > > > > > > > > another
> > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > K-V or
> > > > > > > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with
> > > config
> > > > > K-V
> > > > > > > in
> > > > > > > > > > create
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request.
> > This
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field
> > can
> > > > be
> > > > > > > quite
> > > > > > > > > > useful
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > > storing
> > > > > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who
> > issue
> > > > the
> > > > > > > > create
> > > > > > > > > > > > command,
> > > > > > > > > > > > > > > etc,
> > > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > multi-tenant
> > > > > > > > setting.
> > > > > > > > > > > Then
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on
> > regex
> > > of
> > > > > the
> > > > > > > > > > metadata
> > > > > > > > > > > > > field.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4.
> > Today
> > > > all
> > > > > > the
> > > > > > > > > admin
> > > > > > > > > > > > > > operations
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> return
> > > once
> > > > > it
> > > > > > is
> > > > > > > > > > written
> > > > > > > > > > > > in
> > > > > > > > > > > > > > ZK,
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > verification
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > request,
> > > > etc.
> > > > > > > With
> > > > > > > > > > admin
> > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > synchronous
> > > > > > > > requests;
> > > > > > > > > > > when
> > > > > > > > > > > > it
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > turned
> > > > > > > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until
> > the
> > > > > > request
> > > > > > > > has
> > > > > > > > > > > been
> > > > > > > > > > > > > > > > > completed. And
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> "token"
> > > > field
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > > response,
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > verification
> > > > > > > > request"
> > > > > > > > > > > with
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > given
> > > > > > > > > > > > > > > > > > > token
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has
> > been
> > > > > > > completed.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1
> > for
> > > > > > > extending
> > > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> coordinator
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > information,
> > > > > > and
> > > > > > > > then
> > > > > > > > > > we
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > ClusterMetadata
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > requests.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > Guozhang
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On
> Tue,
> > > Mar
> > > > > 3,
> > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > > 10:23
> > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > jjkoshy.w@gmail.com
> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> Thanks
> > > for
> > > > > > > sending
> > > > > > > > > > that
> > > > > > > > > > > > out
> > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > -
> > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it
> > > today,
> > > > so
> > > > > > if
> > > > > > > > > notes
> > > > > > > > > > > can
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On
> > Mon,
> > > > Mar
> > > > > > 02,
> > > > > > > > 2015
> > > > > > > > > > at
> > > > > > > > > > > > > > > 09:16:13AM
> > > > > > > > > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > Thanks
> > > > for
> > > > > > > > sending
> > > > > > > > > > > this
> > > > > > > > > > > > > out
> > > > > > > > > > > > > > > Joe.
> > > > > > > > > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > everyone
> > > > :)
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On
> > > Mon,
> > > > > Mar
> > > > > > 2,
> > > > > > > > > 2015
> > > > > > > > > > at
> > > > > > > > > > > > > 6:46
> > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > Hey, I
> > > > > > just
> > > > > > > > sent
> > > > > > > > > > > out a
> > > > > > > > > > > > > > > google
> > > > > > > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> committers
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > everyone I
> > > > > > > > found
> > > > > > > > > > > > working
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> please
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > let
> > > me
> > > > > > know
> > > > > > > > and
> > > > > > > > > > can
> > > > > > > > > > > > > update
> > > > > > > > > > > > > > > it,
> > > > > > > > > > > > > > > > > np.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> We
> > > > > should
> > > > > > do
> > > > > > > > > this
> > > > > > > > > > > > every
> > > > > > > > > > > > > > > > Tuesday
> > > > > > > > > > > > > > > > > @
> > > > > > > > > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> INFRA
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > help
> > > > to
> > > > > > > make a
> > > > > > > > > > > google
> > > > > > > > > > > > > > > account
> > > > > > > > > > > > > > > > > so we
> > > > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> To
> > > > > discuss
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> in
> > > > > > progress
> > > > > > > > and
> > > > > > > > > > > > related
> > > > > > > > > > > > > > JIRA
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> ~
> > > Joe
> > > > > > Stein
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> On
> > > > Tue,
> > > > > > Feb
> > > > > > > > 24,
> > > > > > > > > > 2015
> > > > > > > > > > > > at
> > > > > > > > > > > > > > 2:59
> > > > > > > > > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > jay.kreps@gmail.com
> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > Let's
> > > > > > stay
> > > > > > > on
> > > > > > > > > > > Google
> > > > > > > > > > > > > > > hangouts
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > sessions
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > available
> > > > > > > on
> > > > > > > > > > > youtube.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > -Jay
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > On
> > > > Tue,
> > > > > > Feb
> > > > > > > > 24,
> > > > > > > > > > > 2015
> > > > > > > > > > > > at
> > > > > > > > > > > > > > > 11:49
> > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > jholoman@cloudera.com
> > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > Jay /
> > > > > > Joe
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > We're
> > > > > > > happy
> > > > > > > > > to
> > > > > > > > > > > send
> > > > > > > > > > > > > > out a
> > > > > > > > > > > > > > > > > Webex
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > > > sessions
> > > > > > > if
> > > > > > > > > > there
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > interest and
> > > > > > > > > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > > Thanks
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > Jeff
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > On
> > > > > Tue,
> > > > > > > Feb
> > > > > > > > > 24,
> > > > > > > > > > > > 2015
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > Let's
> > > > > > > try
> > > > > > > > > to
> > > > > > > > > > > get
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > technical
> > > > > > > > > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> really
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > think
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > there
> > > > > > > is
> > > > > > > > > some
> > > > > > > > > > > > > benefit
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > hopeful
> > > > that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > if
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > we
> > > > > > post
> > > > > > > > > > > > > instructions
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > give
> > > > > > > > > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get
> it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > > > working.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > > Tuesday
> > > > > > > > at
> > > > > > > > > > that
> > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > work
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > -Jay
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > On
> > > > > > Tue,
> > > > > > > > Feb
> > > > > > > > > > 24,
> > > > > > > > > > > > > 2015
> > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > 8:18
> > > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > > Weekly
> > > > > > > > > > would
> > > > > > > > > > > be
> > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > maybe
> > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > ????
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > I
> > > > > > > don't
> > > > > > > > > > mind
> > > > > > > > > > > > > google
> > > > > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > whatever
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > so
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > we
> > > > > > > know
> > > > > > > > > the
> > > > > > > > > > > > > apache
> > > > > > > > > > > > > > > irc
> > > > > > > > > > > > > > > > > > > channel
> > > > > > > > > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see
> > how
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > goes?
> > > > > > > > We
> > > > > > > > > > can
> > > > > > > > > > > > pull
> > > > > > > > > > > > > > > > > transcripts
> > > > > > > > > > > > > > > > > > > > too
> > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets
> if
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need
> > be
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > makes
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > it
> > > > > > > > > helpful
> > > > > > > > > > > for
> > > > > > > > > > > > > > > things.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > ~
> > > > > > > > > Joestein
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > On
> > > > > > > Tue,
> > > > > > > > > Feb
> > > > > > > > > > > 24,
> > > > > > > > > > > > > > 2015
> > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > 11:10
> > > > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > jay.kreps@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > We'd
> > > > > > > > > > talked
> > > > > > > > > > > > > about
> > > > > > > > > > > > > > > > > doing a
> > > > > > > > > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > about
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > > > > > generalizing
> > > > > > > > > > > > > > that a
> > > > > > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> would
> > be
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > good
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > for
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > > > everyone
> > > > > > > > > > > > > > spending a
> > > > > > > > > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> stuff
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > maybe
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > sync
> > > > > > > > up
> > > > > > > > > > > once
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > week.
> > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > through
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > design
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > > stuff,
> > > > > > > > > > make
> > > > > > > > > > > > > sure
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > are on
> > > > > > > > > > > > > > > > > > > > top
> > > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > > tricky
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > > issues,
> > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > We
> > > > > > > > can
> > > > > > > > > > make
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > > publicly
> > > > > > > > > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> along
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > who
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > > likes.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > Any
> > > > > > > > > > > interest
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > doing
> > > > > > > > > > > > > > > > > this?
> > > > > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > starting
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > next
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > week.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > -Jay
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > On
> > > > > > > > Tue,
> > > > > > > > > > Feb
> > > > > > > > > > > > 24,
> > > > > > > > > > > > > > > 2015
> > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > >
> > > > > > > Hi
> > > > > > > > > all,
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > >
> > > > > > > > I've
> > > > > > > > > > > > updated
> > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > page,
> > > > > > > > > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> structure.
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> Also I
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > >
> > > > > added
> > > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> >> >
> > > > >
> > > > > > >
> > > > > > > > some
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > ...
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > [Message clipped]
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > --
> > > > > > > > > > > > -- Guozhang
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
>
> --
> Thanks,
> Neha
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Neha Narkhede <ne...@confluent.io>.
I'd like to resurface the discussion of sync vs async topic creation.
Possibly, we can discuss that in today's KIP meeting. I am worried about
pushing the burden of handling async topic creation on the clients. This is
one of the most frequently asked questions on the mailing list and I hope
we find a way to address it as we separate the metadata request and topic
creation. Since topic creation doesn't take that long, there should be a
really easy way for clients to express their preference for a sync topic
creation request where the receipt of a response suggests completion of
topic creation.

The behavior that I'm expecting is that if I'm a client and I choose to
create a topic synchronously, after I receive a response, my first send
request for that topic should *not* fail with some error.

On Tue, Mar 24, 2015 at 8:02 AM, Jun Rao <ju...@confluent.io> wrote:

> Andrii,
>
>
> 111.4 Yes, we can discuss this in the meeting today.
>
> 112. The question is when will a PartitionReassignment fail. Currently, it
> can fail if the input is incorrect (e.g, invalid partition/replica).
> However, this can be detected when the ReassignPartitionRequest is issued.
> Once the process of partition reassignment is started, it will not fail.
> The controller just waits until the process completes. We can improve the
> status reporting of partition reassignment. However, I am not sure if that
> needs to be tied to this KIP.
>
> 115.1 Yes, that makes sense. I misunderstood it. I thought
> PreferredReplicaLeaderElectionInProgress and ReassignPartitionsInProgress
> are used in the status check.
>
> Thanks,
>
> Jun
>
>
> On Tue, Mar 24, 2015 at 3:46 AM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Jun,
> >
> > Thanks for such an accurate review!
> >
> > Most of your remarks I didn't fix in the previous change because
> > I thought we'd finalize them today, during the call. Anyway, I've
> > fixed them now.
> > My comments to ensure I covered everything:
> >
> > 110. Fixed. Removed global error code.
> >
> > 111.1-3. Removed redundant TopicName, Partitions and ReplicationFactor
> > 111.4 Yes, now we are in the situation when TopicMetadataResponse
> > contains everything we need except topic level configs. I'm
> > okay with both solutions, probably makes sense to discuss and pick
> > the preferable one.
> >
> > 112. Need to think about it a bit more. Currently we may distinguish
> > 3 states of specific partition being reassigned:
> > a) Completed - absent in /reassign_partitions zk path and DescribeTopic/
> > TopicMetadata shows expected assignment
> > 2) In progress - present in /reassign_partitions
> > 3) Failed - absent in /reassign_partitions but
> DescribeTopic/TopicMetadata
> > replicas field doesn't correspond expected
> > Now, if use only DescribeTopic to check reassignment status - how can we
> > distinguish states "In Progress" and "Failed"?
> > Also, there are some concerns that current reassignment status provided
> > by ReaasignPartitionsCommand (and the same was intended in KIP-4) is
> > very spurious, non-informative. Probably it makes sense to add additional
> > data there (thus separate request may be useful), I'm considering
> different
> > options now - btw, comments here are highly appreciated!
> >
> > 113-114. Okay, made it compliant with topic commands. About "The error
> code
> > will
> > then be per topic" - just a note: currently (ReassignPartitionsCommand)
> > validation
> > is done for the whole reassignment string, so in case of errors for
> > specific
> > partitions the reassignment is NOT started at all. Both options look fine
> > for me,
> > it's just the logic will be slightly changed.
> >
> > 115.1 Removed NotControllerReceivedAdminCommand. But why should we
> > remove PreferredReplicaLeaderElectionInProgress and
> > ReassignPartitionsInProgress?
> > On Reassign/Preferred..Request we create admin zk path, we can of course
> > update the path, but wouldn't it be safer if we simply refuse to start
> new
> > reassignment in case one is in progress?
> >
> > 115.2 Yes, paragraph Protocol Errors covers only errors that are proposed
> > to be added. I didn't want to list all errors that are to be added since
> > anyway
> > the final version will be clear after the development is completed.
> >
> > 100. Fixed, normalized json string in Create/Alter.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> > On Tue, Mar 24, 2015 at 12:58 AM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Andrii,
> > >
> > > I looked at the latest wiki for this KIP. I have a few more comments.
> > >
> > > 110. In CreateTopicResponse, AlterTopicResponse, DeleteTopicResponse
> and
> > > DescribeTopicResponse, we probably don't need the global error code.
> Our
> > > current convention is to just use the same global error code in each
> > topic.
> > > This simplifies the error checking on the client side.
> > >
> > > 111. DescribeTopicResponse:
> > > 111.1 Our protocol definition doesn't support optional fields. So, we
> > can't
> > > make Leader an optional field.
> > > 111.2 Do we really need the field Partitions and ReplicationFactor in
> > > TopicConfigDetails?
> > > The former is basically the size of the TopicPartitionDetails array and
> > the
> > > latter can be found from the size of the Replica array.
> > > 111.3 Do we need TopicName in TopicDescription since it's already in
> > > DescribeTopicResponse?
> > > 111.4 Finally, DescribeTopicResponse is very similar to
> > > TopicMetadataResponse. So, it probably makes sense just to keep one of
> > them
> > > in the future. Should we just use DescribeTopicRequest/Response to
> > replace
> > > TopicMetadataRequest/Response in the producer/consumer client in the
> > future
> > > (since DescribeTopicRequest doesn't trigger auto topic creation)? If
> so,
> > we
> > > will need to add the broker list in DescribeTopicResponse.
> > >
> > > 112. Thinking about this a bit more, I don't think we need a separate
> > > VerifyReassignPartitionRequest/Response.
> > > We can just use DescribeTopic to get the assigned replicas and check if
> > > they match the target replica assignment in the client. The controller
> > > propagates the metadata change after the reassignment completes for
> each
> > > partition.
> > >
> > > 113. ReassignPartitionRequest: For consistency, we probably want to
> nest
> > > the partition data under topic. So instead of
> > >   ReassignPartitionRequest => [Topic PartitionId [ReplicaId]]
> > > we can have sth like
> > >   ReassignPartitionRequest => [Topic [PartitionId [ReplicaId]]]
> > > The error code will then be per topic.
> > >
> > > 114. PreferredReplicaLeaderElectionRequest: Same as the above. Instead
> of
> > >   PreferredReplicaLeaderElectionRequest => [Topic PartitionId]
> > > we can have
> > >   PreferredReplicaLeaderElectionRequest => [Topic [PartitionId]]
> > > Again, the error code will be per topic.
> > >
> > > 115. ErrorCode:
> > > 115.1 NotControllerReceivedAdminRequest,
> > > PreferredReplicaLeaderElectionInProgress
> > > and ReassignPartitionsInProgress are likely not needed any more.
> > > 115.2 We probably need to add UnknownTopicOrPartitionCode since it's a
> > > valid error code for some of the requests (e.g. describe, alter, etc)
> > when
> > > a non-existing topic is specified.
> > >
> > >
> > > Previous comments.
> > > 100. ReplicaAssignment in CreateTopicRequest and AlterTopicRequest are
> > > still a json string. We need to flatten them.
> > >
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Sat, Mar 21, 2015 at 2:03 AM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Guozhang,
> > > >
> > > > I'm not sure I understand how can we use those tools in CLI.
> > > > First of all, those are *Test*Utils, e.g.
> waitUntilMetadataIsPropagated
> > > > (which might be very useful for us) requires servers:Seq[KafkaServer]
> > > > as an argument:
> > > >
> > > > def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic:
> > > String,
> > > > partition: Int, timeout: Long = 5000L): Int
> > > >
> > > > how can we have access it at runtime in one of the brokers?
> > > >
> > > > Secondly, how can user directly call these tools if zookeeper might
> be
> > > > not accessible (which is used in the tools, right?) at all - e.g.
> > behind
> > > > the VPC
> > > > in AWS. Furthermore, I think with KIP-4 we are trying to create an
> > > > abstraction and
> > > > a single point of zookeeper interactions, to eliminate direct calls
> to
> > ZK
> > > > either to get
> > > > some cluster information or to change something.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Sat, Mar 21, 2015 at 3:37 AM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > Andrii,
> > > > >
> > > > > Actually the checking logic Jun mentioned is already implemented as
> > > > > TestUtils.waitUntilXXX (LeaderIsElected, MetadataIsPropagated,
> > etc...)
> > > I
> > > > > think we can extend these functions as CLI tools like TopicCommand
> so
> > > > that
> > > > > users re-implementing such endpoint can directly call something
> like
> > > > > java.tools.WaitUntilXXX (of course this requires them to have javac
> > > > > installed, which should be a reasonable requirement?)
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Jun,
> > > > > >
> > > > > > Not that I was saying we need to make requests blocking on
> server,
> > > > > > it was just to emphasize that with async requests a client
> > > > > implementations
> > > > > > may be a little bit more than just issue request - get the
> > response.
> > > > > > Thanks for the explanation, I understand now that we can go with
> > > agreed
> > > > > > solution though it may not be perfect.
> > > > > > I believe this was one of the last controversial questions from
> the
> > > > list.
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > > On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > >
> > > > > > > Andrii,
> > > > > > >
> > > > > > > A few points.
> > > > > > >
> > > > > > > 1. Create/Alter can typically complete quickly. So, it's
> possible
> > > to
> > > > > make
> > > > > > > the request block until it's completed. However, currently,
> doing
> > > > this
> > > > > at
> > > > > > > the broker is a bit involved. To make Create block, we will
> need
> > to
> > > > add
> > > > > > > some callbacks in KafkaController. This is possible. However,
> the
> > > > > > > controller logic currently is pretty completed. It would
> probably
> > > be
> > > > > > better
> > > > > > > if we clean it up first before adding more complexity to it.
> > Alter
> > > is
> > > > > > even
> > > > > > > trickier. Adding partition is currently handled through
> > > > > KafkaController.
> > > > > > So
> > > > > > > it can be dealt with in a similar way. However, Alter config is
> > > done
> > > > > > > completely differently. It doesn't go through the controller.
> > > > Instead,
> > > > > > each
> > > > > > > broker listens to ZooKeeper directly. So, it's not clear if
> there
> > > is
> > > > an
> > > > > > > easy way on the broker to figure out whether a config is
> applied
> > on
> > > > > every
> > > > > > > broker.
> > > > > > >
> > > > > > > 2. Delete can potentially take long if a replica to be deleted
> is
> > > > > > offline.
> > > > > > > PreferredLeader/PartitionReassign can also take long. So, we
> > can't
> > > > > really
> > > > > > > make those requests block on the broker.
> > > > > > >
> > > > > > > As you can see, at this moment it's not easy to make all admin
> > > > requests
> > > > > > > block on the broker. So, if we want the blocking feature in the
> > > admin
> > > > > > > utility in the short term, doing the completion check at the
> > admin
> > > > > client
> > > > > > > is probably an easier route, even though it may not be ideal.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >
> > > > > > > > Jun,
> > > > > > > >
> > > > > > > > I see your point. But wouldn't that lead to a "fat" client
> > > > > > > implementations?
> > > > > > > > Suppose someone would like to implement client for Admin Wire
> > > > > protocol.
> > > > > > > > Not only people will have to code quite complicated logic
> like
> > > > "send
> > > > > > > > describe
> > > > > > > > request to each broker" (again state machin?) but it will
> also
> > > mean
> > > > > > > people
> > > > > > > > must understand internal kafka logic related to topic storage
> > and
> > > > how
> > > > > > > > information is propageted from the controller to brokers.
> > > > > > > > I see this like a dilemma between having a concise Wire
> > Protocol
> > > > and
> > > > > > > > self-sufficient API to make client implementations simple.
> > > > > > > > I don't have a win-win solution though.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Andrii Biletskyi
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > >
> > > > > > > > > For 1), 2) and 3), blocking would probably mean that the
> new
> > > > > metadata
> > > > > > > is
> > > > > > > > > propagated to every broker. To achieve that, the client can
> > > keep
> > > > > > > issuing
> > > > > > > > > the describe topic request to every broker until it sees
> the
> > > new
> > > > > > > metadata
> > > > > > > > > in the response.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >
> > > > > > > > > > Hm, actually the ticket you linked, Guozhang, brings as
> > back
> > > > > > > > > > to the problem what should be considered a post-condition
> > for
> > > > > > > > > > each of the admin commands.
> > > > > > > > > > In my understanding:
> > > > > > > > > >
> > > > > > > > > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > > > > > > > > (Not the controller picked up changes from zk and
> > broadcasted
> > > > > > > > > > LeaderAndIsr and UpdateMetadata)
> > > > > > > > > >
> > > > > > > > > > 2) AlterTopic - same as 1) - broker changed assignment
> data
> > > > > > > > > > in zookeeper or created admin path for topic config
> change
> > > > > > > > > >
> > > > > > > > > > 3) DeleteTopic - admin path /admin/delete_topics is
> created
> > > > > > > > > >
> > > > > > > > > > 4) ReassignPartitions and PreferredReplica -
> corresponding
> > > > admin
> > > > > > > > > > path is created
> > > > > > > > > >
> > > > > > > > > > Now what can be considered a completed operation from the
> > > > > client's
> > > > > > > > > > perspective?
> > > > > > > > > > 1) Topic is created once corresponding data is in zk
> > > > > > > > > > (I remember there were some thoughts that it'd be good to
> > > > > consider
> > > > > > > > > > topic created once all replicas receive information about
> > it
> > > > and
> > > > > > thus
> > > > > > > > > > clients can produce/consume from it, but as was discussed
> > > this
> > > > > > seems
> > > > > > > > > > to be a hard thing to do)
> > > > > > > > > >
> > > > > > > > > > 2) Probably same as 1), so right after AlterTopic is
> issued
> > > > > > > > > >
> > > > > > > > > > 3) The topic has been removed from /brokers/topics
> > > > > > > > > >
> > > > > > > > > > 4) ReassignPartitions and PrefferedReplica were discussed
> > > > > earlier -
> > > > > > > > > > in short the former is completed once partition state
> info
> > in
> > > > zk
> > > > > > > > matches
> > > > > > > > > > reassignment request and admin path is empty, the latter
> -
> > > once
> > > > > > data
> > > > > > > > > > in zk shows that head of assignned replicas of the
> > partition
> > > > and
> > > > > > > leader
> > > > > > > > > > is the same replica
> > > > > > > > > >
> > > > > > > > > > Thoughts?
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Andrii Biletskyi
> > > > > > > > > >
> > > > > > > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <
> > > > > wangguoz@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > I think while loop is fine for supporting blocking,
> just
> > > that
> > > > > we
> > > > > > > need
> > > > > > > > > to
> > > > > > > > > > > add back off to avoid bombarding brokers with
> > DescribeTopic
> > > > > > > requests.
> > > > > > > > > > >
> > > > > > > > > > > Also I have linked KAFKA-1125
> > > > > > > > > > > <https://issues.apache.org/jira/browse/KAFKA-1125> to
> > your
> > > > > > > proposal,
> > > > > > > > > and
> > > > > > > > > > > when KAFKA-1694 is done this ticket can also be closed.
> > > > > > > > > > >
> > > > > > > > > > > Guozhang
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Great.
> > > > > > > > > > > > I want to elaborate this a bit more, to see we are on
> > the
> > > > > same
> > > > > > > page
> > > > > > > > > > > > concerning the client code.
> > > > > > > > > > > >
> > > > > > > > > > > > So with all topic commands being async a client
> > > > (AdminClient
> > > > > in
> > > > > > > our
> > > > > > > > > > > > case or any other other client people would like to
> > > > > implement)
> > > > > > to
> > > > > > > > > > support
> > > > > > > > > > > > a blocking operation (which seems to be a natural
> > > use-case
> > > > > e.g.
> > > > > > > for
> > > > > > > > > > topic
> > > > > > > > > > > > creation): would have to do:
> > > > > > > > > > > > 1. issue CreateTopicRequest
> > > > > > > > > > > > 2. if successful, in a "while" loop send
> > > > DescribeTopicRequest
> > > > > > and
> > > > > > > > > > > > break the loop once all topics are returned in
> response
> > > (or
> > > > > > upon
> > > > > > > > > > > timeout).
> > > > > > > > > > > > 3. if unsuccessful throw exception
> > > > > > > > > > > > Would it be okay?
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <
> > > jun@confluent.io
> > > > >
> > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > >
> > > > > > > > > > > > > I think you are right. It seems that only
> > > > > ReassignPartitions
> > > > > > > > needs
> > > > > > > > > a
> > > > > > > > > > > > > separate verification request.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > I like this idea too. Let's stick with that. I'll
> > > > update
> > > > > > KIP
> > > > > > > > > > > > accordingly.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I was also thinking we can avoid adding dedicated
> > > > status
> > > > > > > check
> > > > > > > > > > > > > > requests for topic commands. - We have everything
> > in
> > > > > > > > > DescribeTopic
> > > > > > > > > > > > > > for that! E.g.:
> > > > > > > > > > > > > > User issued CreateTopic - to check the status
> > client
> > > > > sends
> > > > > > > > > > > > DescribeTopic
> > > > > > > > > > > > > > and checks whether is something returned for that
> > > > topic.
> > > > > > The
> > > > > > > > same
> > > > > > > > > > for
> > > > > > > > > > > > > > alteration, deletion.
> > > > > > > > > > > > > > Btw, PreferredReplica status can be also checked
> > with
> > > > > > > > > > > > > DescribeTopicRequest
> > > > > > > > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > > > > > > > For ReassignPartitions as discussed we'll need to
> > > have
> > > > a
> > > > > > > > separate
> > > > > > > > > > > > > Verify...
> > > > > > > > > > > > > > request.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> > > > > > > > > wangguoz@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1 on broker writing to ZK for async handling.
> I
> > > was
> > > > > > > thinking
> > > > > > > > > > that
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > end state the admin requests would be
> eventually
> > > sent
> > > > > to
> > > > > > > > > > controller
> > > > > > > > > > > > > > either
> > > > > > > > > > > > > > > through re-routing or clients discovering them,
> > > > instead
> > > > > > of
> > > > > > > > > > letting
> > > > > > > > > > > > > > > controller listen on ZK admin path. But
> thinking
> > > > about
> > > > > > it a
> > > > > > > > > > second
> > > > > > > > > > > > > time,
> > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > think it is actually simpler to let controller
> > > manage
> > > > > > > > > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > +1 as well. I think it helps to keep the
> > > rerouting
> > > > > > > approach
> > > > > > > > > > > > > orthogonal
> > > > > > > > > > > > > > > > to this KIP.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay
> > > Kreps
> > > > > > > wrote:
> > > > > > > > > > > > > > > > > I'm +1 on Jun's suggestion as long as it
> can
> > > work
> > > > > for
> > > > > > > all
> > > > > > > > > the
> > > > > > > > > > > > > > requests.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <
> > > > > > > > jun@confluent.io
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > (a) Admin requests can be sent to and
> > handled
> > > > by
> > > > > > any
> > > > > > > > > > broker.
> > > > > > > > > > > > > > > > > > (b) Admin requests are processed
> > > > asynchronously,
> > > > > at
> > > > > > > > least
> > > > > > > > > > for
> > > > > > > > > > > > > now.
> > > > > > > > > > > > > > > > That is,
> > > > > > > > > > > > > > > > > > when the client gets a response, it just
> > > means
> > > > > that
> > > > > > > the
> > > > > > > > > > > request
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > initiated, but not necessarily completed.
> > > Then,
> > > > > > it's
> > > > > > > up
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > issue another request to check the status
> > for
> > > > > > > > completion.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > To support (a), we were thinking of doing
> > > > request
> > > > > > > > > > forwarding
> > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > controller (utilizing KAFKA-1912). I am
> > > making
> > > > an
> > > > > > > > > > alternative
> > > > > > > > > > > > > > > proposal.
> > > > > > > > > > > > > > > > > > Basically, the broker can just write to
> > > > ZooKeeper
> > > > > > to
> > > > > > > > > inform
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > > about the request. For example, to handle
> > > > > > > > > > > > partitionReassignment,
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > will just write the requested partitions
> to
> > > > > > > > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > > > > > > > (like what AdminUtils currently does) and
> > > then
> > > > > > send a
> > > > > > > > > > > response
> > > > > > > > > > > > to
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > client. This shouldn't take long and the
> > > > > > > implementation
> > > > > > > > > > will
> > > > > > > > > > > be
> > > > > > > > > > > > > > > simpler
> > > > > > > > > > > > > > > > > > than forwarding the requests to the
> > > controller
> > > > > > > through
> > > > > > > > > RPC.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii
> > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Jun,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I might be wrong but didn't we agree we
> > > will
> > > > > let
> > > > > > > any
> > > > > > > > > > broker
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > cluster handle *long-running* admin
> > > requests
> > > > > (at
> > > > > > > this
> > > > > > > > > > time
> > > > > > > > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > reassignPartitions), via zk admin path.
> > > Thus
> > > > > > > > > CreateTopics
> > > > > > > > > > > etc
> > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun
> > Rao <
> > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I think we agreed that those admin
> > > requests
> > > > > can
> > > > > > > be
> > > > > > > > > > issued
> > > > > > > > > > > > to
> > > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > broker.
> > > > > > > > > > > > > > > > > > > > Because of that, there doesn't seem
> to
> > > be a
> > > > > > > strong
> > > > > > > > > need
> > > > > > > > > > > to
> > > > > > > > > > > > > know
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > controller. So, perhaps we can
> proceed
> > by
> > > > not
> > > > > > > > making
> > > > > > > > > > any
> > > > > > > > > > > > > change
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > format of TMR right now. When we
> start
> > > > using
> > > > > > > create
> > > > > > > > > > topic
> > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > producer, we will need a new version
> of
> > > TMR
> > > > > > that
> > > > > > > > > > doesn't
> > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > As a first cut implementation, I
> think
> > > the
> > > > > > broker
> > > > > > > > can
> > > > > > > > > > > just
> > > > > > > > > > > > > > write
> > > > > > > > > > > > > > > > to ZK
> > > > > > > > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > > > > > > > requests, instead of forwarding them
> to
> > > the
> > > > > > > > > controller.
> > > > > > > > > > > > This
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > > > > > > > the implementation on the broker
> side.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM,
> Joel
> > > > Koshy
> > > > > <
> > > > > > > > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > For (1) yes we will circle back on
> > that
> > > > > > shortly
> > > > > > > > > after
> > > > > > > > > > > > > syncing
> > > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > person. I think it is close to
> > getting
> > > > > > > committed
> > > > > > > > > > > although
> > > > > > > > > > > > > > > > development
> > > > > > > > > > > > > > > > > > > > > for KAFKA-1927 can probably begin
> > > without
> > > > > it.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > There is one more item we covered
> at
> > > the
> > > > > > > hangout.
> > > > > > > > > > i.e.,
> > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > want to add the coordinator to the
> > > topic
> > > > > > > metadata
> > > > > > > > > > > > response
> > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > There are two reasons I think we
> > should
> > > > try
> > > > > > and
> > > > > > > > > avoid
> > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > > > > > > > - It is irrelevant to topic
> metadata
> > > > > > > > > > > > > > > > > > > > > - If we finally do request
> rerouting
> > in
> > > > > Kafka
> > > > > > > > then
> > > > > > > > > > the
> > > > > > > > > > > > > field
> > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > >   little to no value. (It still
> helps
> > > to
> > > > > > have a
> > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > > >   ClusterMetadataRequest to query
> for
> > > > > > > > cluster-wide
> > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > > >   'which broker is the controller?'
> > as
> > > > Joe
> > > > > > > > > > mentioned.)
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > I think it would be cleaner to have
> > an
> > > > > > explicit
> > > > > > > > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > > > > > > > that you can send to any broker in
> > > order
> > > > to
> > > > > > > > obtain
> > > > > > > > > > the
> > > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > > in the future possibly other
> > > cluster-wide
> > > > > > > > > > > information). I
> > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > main argument against doing this
> and
> > > > > instead
> > > > > > > > adding
> > > > > > > > > > it
> > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > metadata response was convenience -
> > > i.e.,
> > > > > you
> > > > > > > > don't
> > > > > > > > > > > have
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > discover
> > > > > > > > > > > > > > > > > > > > > the controller in advance.
> However, I
> > > > don't
> > > > > > see
> > > > > > > > > much
> > > > > > > > > > > > actual
> > > > > > > > > > > > > > > > > > > > > benefit/convenience in this and in
> > fact
> > > > > think
> > > > > > > it
> > > > > > > > > is a
> > > > > > > > > > > > > > > non-issue.
> > > > > > > > > > > > > > > > Let
> > > > > > > > > > > > > > > > > > > > > me know if I'm overlooking
> something
> > > > here.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > As an example, say we need to
> > initiate
> > > > > > > partition
> > > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > > > > issuing the new
> > > ReassignPartitionsRequest
> > > > > to
> > > > > > > the
> > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > (assume
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > already have the desired manual
> > > partition
> > > > > > > > > > assignment).
> > > > > > > > > > > > If
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > are to
> > > > > > > > > > > > > > > > > > > > > augment topic metadata response
> then
> > > the
> > > > > flow
> > > > > > > be
> > > > > > > > > > > > something
> > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > :
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > - Issue topic metadata request to
> any
> > > > > broker
> > > > > > > (and
> > > > > > > > > > > > discover
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > > > > > > > - Connect to controller if required
> > > > (i.e.,
> > > > > if
> > > > > > > the
> > > > > > > > > > > broker
> > > > > > > > > > > > > > above
> > > > > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > > > > - Issue the partition reassignment
> > > > request
> > > > > to
> > > > > > > the
> > > > > > > > > > > > > controller.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > With an explicit cluster metadata
> > > request
> > > > > it
> > > > > > > > would
> > > > > > > > > > be:
> > > > > > > > > > > > > > > > > > > > > - Issue cluster metadata request to
> > any
> > > > > > broker
> > > > > > > > > > > > > > > > > > > > > - Connect to controller if required
> > > > (i.e.,
> > > > > if
> > > > > > > the
> > > > > > > > > > > broker
> > > > > > > > > > > > > > above
> > > > > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > > > > - Issue the partition reassignment
> > > > request
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > So it seems to add little practical
> > > value
> > > > > and
> > > > > > > > > bloats
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > The other angle to this is the
> > > following
> > > > -
> > > > > is
> > > > > > > it
> > > > > > > > a
> > > > > > > > > > > matter
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > naming?
> > > > > > > > > > > > > > > > > > > > > Should we just rename topic
> metadata
> > > > > > > > > request/response
> > > > > > > > > > > to
> > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > > MetadataRequest/Response and add
> > > cluster
> > > > > > > metadata
> > > > > > > > > to
> > > > > > > > > > > it?
> > > > > > > > > > > > By
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > same
> > > > > > > > > > > > > > > > > > > > > token should we also allow querying
> > for
> > > > the
> > > > > > > > > consumer
> > > > > > > > > > > > > > > coordinator
> > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > > in future transaction coordinator)
> as
> > > > well?
> > > > > > > This
> > > > > > > > > > leads
> > > > > > > > > > > > to a
> > > > > > > > > > > > > > > > bloated
> > > > > > > > > > > > > > > > > > > > > request which isn't very appealing
> > and
> > > > > > > altogether
> > > > > > > > > > > > > confusing.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM
> > > -0700,
> > > > > Jun
> > > > > > > Rao
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 1. I just realized that in order
> to
> > > > start
> > > > > > > > working
> > > > > > > > > > on
> > > > > > > > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > need to merge the changes to
> > > > > > > > OffsetCommitRequest
> > > > > > > > > > > (from
> > > > > > > > > > > > > > 0.8.2)
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > > > > > > > This is planned to be done as
> part
> > of
> > > > > > > > KAFKA-1634.
> > > > > > > > > > So,
> > > > > > > > > > > > we
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 2. Thinking about this a bit
> more,
> > if
> > > > the
> > > > > > > > > semantic
> > > > > > > > > > of
> > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > "write"
> > > > > > > > > > > > > > > > > > > > > > requests is async (i.e., after
> the
> > > > client
> > > > > > > gets
> > > > > > > > a
> > > > > > > > > > > > > response,
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > > > > > > > that the operation is initiated,
> > but
> > > > not
> > > > > > > > > > necessarily
> > > > > > > > > > > > > > > > completed), we
> > > > > > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > > > > really need to forward the
> requests
> > > to
> > > > > the
> > > > > > > > > > > controller.
> > > > > > > > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > > > > > > > receiving broker can just write
> the
> > > > > > operation
> > > > > > > > to
> > > > > > > > > ZK
> > > > > > > > > > > as
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > > > > > > > line tool previously does. This
> > will
> > > > > > simplify
> > > > > > > > the
> > > > > > > > > > > > > > > > implementation.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 8. There is another
> implementation
> > > > detail
> > > > > > for
> > > > > > > > > > > describe
> > > > > > > > > > > > > > topic.
> > > > > > > > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > want to read the topic config
> from
> > > the
> > > > > > broker
> > > > > > > > > > cache,
> > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > > > > > > > Currently, every broker reads the
> > > > > > topic-level
> > > > > > > > > > config
> > > > > > > > > > > > for
> > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > > > > However, it ignores those for
> > topics
> > > > not
> > > > > > > hosted
> > > > > > > > > on
> > > > > > > > > > > > > itself.
> > > > > > > > > > > > > > > So,
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > to change TopicConfigManager a
> bit
> > so
> > > > > that
> > > > > > it
> > > > > > > > > > caches
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM,
> > > Andrii
> > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > wrote:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala
> > requests
> > > > > > > objects,
> > > > > > > > > use
> > > > > > > > > > > java
> > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > > > > > > > >     A: Gwen kindly took that
> > > > > > (KAFKA-1927).
> > > > > > > > It's
> > > > > > > > > > > > > important
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > speed
> > > > > > > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > > > > > > > >          there since this
> ticket
> > > > blocks
> > > > > > > other
> > > > > > > > > > > > important
> > > > > > > > > > > > > > > > changes.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute
> facility
> > > vs
> > > > > > client
> > > > > > > > > > > > maintaining
> > > > > > > > > > > > > > > > cluster
> > > > > > > > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > > > > > > > >     A: Jay has added pseudo
> code
> > to
> > > > > > > > KAFKA-1912
> > > > > > > > > -
> > > > > > > > > > > need
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > > > > > > > >         easy to implement as a
> > > > > > server-side
> > > > > > > > > > feature
> > > > > > > > > > > > > > > (comments
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 3. Q: Controller field in wire
> > > > > protocol.
> > > > > > > > > > > > > > > > > > > > > > >     A: This might be useful for
> > > > > clients,
> > > > > > > add
> > > > > > > > > this
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic creation
> > > from
> > > > > TMR.
> > > > > > > > > > > > > > > > > > > > > > >     A: I will add proposed by
> Jun
> > > > > > solution
> > > > > > > > > (using
> > > > > > > > > > > > > > clientId
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 5. Q: Bumping new versions of
> TMR
> > > vs
> > > > > > > grabbing
> > > > > > > > > all
> > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > > > >     A: It was decided to try to
> > > > gather
> > > > > > all
> > > > > > > > > > changes
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > > > > > > > >         In case of TMR it worth
> > > > > checking:
> > > > > > > > > > > KAFKA-2020
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to
> > > > deserialize
> > > > > > > > user's
> > > > > > > > > > > input
> > > > > > > > > > > > in
> > > > > > > > > > > > > > CLI
> > > > > > > > > > > > > > > > tool.
> > > > > > > > > > > > > > > > > > > > > > >     A: Use jackson for that,
> > /tools
> > > > > > project
> > > > > > > > is
> > > > > > > > > a
> > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > jar so
> > > > > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions
> > vs
> > > > > > generic
> > > > > > > > > status
> > > > > > > > > > > > check
> > > > > > > > > > > > > > > > command.
> > > > > > > > > > > > > > > > > > > > > > >      A: For long-running
> requests
> > > > like
> > > > > > > > reassign
> > > > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > > > > > > > >          it makes sense to
> > > introduce
> > > > > it.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >  Please add, correct me if I
> > missed
> > > > > > > > something.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20
> PM,
> > > > Andrii
> > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > You are right, I removed
> > > > > > ClusterMetadata
> > > > > > > > > > because
> > > > > > > > > > > we
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > > > > > > > what we need in
> TopicMetadata.
> > > > Also,
> > > > > as
> > > > > > > Jay
> > > > > > > > > > > pointed
> > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > earlier,
> > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > > would like to have
> "orthogonal"
> > > > API,
> > > > > > but
> > > > > > > at
> > > > > > > > > the
> > > > > > > > > > > > same
> > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > But I like your idea and even
> > > have
> > > > > some
> > > > > > > > other
> > > > > > > > > > > > > arguments
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > > > > > > > There is also
> > > DescribeTopicRequest
> > > > > > which
> > > > > > > > was
> > > > > > > > > > > > proposed
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > > > > > > > it returns topic configs,
> > > > partitions,
> > > > > > > > > > replication
> > > > > > > > > > > > > > factor
> > > > > > > > > > > > > > > > plus
> > > > > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > > > > > > > leader replica. The later
> part
> > is
> > > > > > really
> > > > > > > > > > already
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > > > > > > > So again we'll have to add
> > stuff
> > > to
> > > > > > TMR,
> > > > > > > > not
> > > > > > > > > to
> > > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > > newly added requests.
> However,
> > > this
> > > > > way
> > > > > > > > we'll
> > > > > > > > > > end
> > > > > > > > > > > > up
> > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > > > > > > > request which returns cluster
> > > > > metadata,
> > > > > > > > topic
> > > > > > > > > > > > > > replication
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > > > > > plus partition replication
> > data.
> > > > > Seems
> > > > > > > > > logical
> > > > > > > > > > to
> > > > > > > > > > > > > split
> > > > > > > > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > > > > > > > - ClusterMetadata (brokers +
> > > > > > controller,
> > > > > > > > > maybe
> > > > > > > > > > > smth
> > > > > > > > > > > > > > else)
> > > > > > > > > > > > > > > > > > > > > > > > - TopicMetadata (topic info +
> > > > > partition
> > > > > > > > > > details)
> > > > > > > > > > > > > > > > > > > > > > > > But since current TMR is
> > involved
> > > > in
> > > > > > lots
> > > > > > > > of
> > > > > > > > > > > places
> > > > > > > > > > > > > > > > (including
> > > > > > > > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > > > > > > as I understand) this might
> be
> > > very
> > > > > > > serious
> > > > > > > > > > > change
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > > > > > > > sense to stick with current
> > > > approach.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29
> > PM,
> > > > Joel
> > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> I may be missing some
> context
> > > but
> > > > > > > > hopefully
> > > > > > > > > > this
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > also be
> > > > > > > > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > > > > > > > >> today: I thought the earlier
> > > > > proposal
> > > > > > > > where
> > > > > > > > > > > there
> > > > > > > > > > > > > was
> > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > > > > > > > >> ClusterMetadata request was
> > > > clearer
> > > > > > and
> > > > > > > > > > > explicit.
> > > > > > > > > > > > > > During
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > >> this thread I think the
> > > conclusion
> > > > > was
> > > > > > > > that
> > > > > > > > > > the
> > > > > > > > > > > > main
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > >> controller information and
> > that
> > > > can
> > > > > be
> > > > > > > > > rolled
> > > > > > > > > > > into
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > >> response but that seems a
> bit
> > > > > > irrelevant
> > > > > > > > to
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > > metadata.
> > > > > > > > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > > > > > > > >> think the full broker-list
> is
> > > also
> > > > > > > > > irrelevant
> > > > > > > > > > to
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > >> it is already there and in
> > use.
> > > I
> > > > > > think
> > > > > > > > > there
> > > > > > > > > > is
> > > > > > > > > > > > > still
> > > > > > > > > > > > > > > > room
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > > > >> explicit ClusterMetadata
> > request
> > > > > since
> > > > > > > > there
> > > > > > > > > > may
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > > > > >> cluster-level information
> that
> > > we
> > > > > may
> > > > > > > want
> > > > > > > > > to
> > > > > > > > > > > add
> > > > > > > > > > > > > over
> > > > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> have nothing to do with
> topic
> > > > > > metadata).
> > > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at
> > > 02:45:30PM
> > > > > > > +0200,
> > > > > > > > > > Andrii
> > > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > 101. Okay, if you say that
> > > such
> > > > > use
> > > > > > > case
> > > > > > > > > is
> > > > > > > > > > > > > > > important. I
> > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > > > > >> > using clientId for these
> > > > purposes
> > > > > is
> > > > > > > > fine
> > > > > > > > > -
> > > > > > > > > > if
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > > > > > > > >> > as part of all Wire
> protocol
> > > > > > messages,
> > > > > > > > why
> > > > > > > > > > not
> > > > > > > > > > > > use
> > > > > > > > > > > > > > > that.
> > > > > > > > > > > > > > > > > > > > > > > >> > I will update KIP-4 page
> if
> > > > nobody
> > > > > > has
> > > > > > > > > other
> > > > > > > > > > > > ideas
> > > > > > > > > > > > > > > > (which
> > > > > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update
> the
> > > KIP
> > > > > > > > > > accordingly.
> > > > > > > > > > > I
> > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > we can
> > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > > > > > > > >> > fine-grained error codes
> if
> > > some
> > > > > > error
> > > > > > > > > code
> > > > > > > > > > > > > received
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > > > >> > won't give enough context
> to
> > > > > return
> > > > > > a
> > > > > > > > > > > > descriptive
> > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > message
> > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > Look forward to discussing
> > all
> > > > > > > > outstanding
> > > > > > > > > > > > issues
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > detail
> > > > > > > > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at
> > 10:59
> > > > PM,
> > > > > > Jun
> > > > > > > > Rao
> > > > > > > > > <
> > > > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > 101. There may be a use
> > case
> > > > > where
> > > > > > > you
> > > > > > > > > > only
> > > > > > > > > > > > want
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > > > > > > > >> > > manually by admins.
> > > Currently,
> > > > > you
> > > > > > > can
> > > > > > > > > do
> > > > > > > > > > > that
> > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > disabling
> > > > > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> > > creation and issue topic
> > > > > creation
> > > > > > > from
> > > > > > > > > the
> > > > > > > > > > > > > > > > TopicCommand.
> > > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > > > > > > > >> > > topic creation
> completely
> > on
> > > > the
> > > > > > > > broker
> > > > > > > > > > and
> > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > have a
> > > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > > > > >> > > between topic creation
> > > > requests
> > > > > > from
> > > > > > > > the
> > > > > > > > > > > > regular
> > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > > > > > > > >> > > can't support manual
> topic
> > > > > > creation
> > > > > > > > any
> > > > > > > > > > > more.
> > > > > > > > > > > > I
> > > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > > > > > > > >> > > way of distinguishing
> the
> > > > > clients
> > > > > > > > making
> > > > > > > > > > the
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > creation
> > > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > > > > >> > > using clientId. For
> > example,
> > > > the
> > > > > > > admin
> > > > > > > > > > tool
> > > > > > > > > > > > can
> > > > > > > > > > > > > > set
> > > > > > > > > > > > > > > > it to
> > > > > > > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > > > > > > > >> > > admin and the broker can
> > > treat
> > > > > > that
> > > > > > > > > > clientId
> > > > > > > > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > Also, there is a related
> > > > > > discussion
> > > > > > > in
> > > > > > > > > > > > > KAFKA-2020.
> > > > > > > > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > >> > > following in
> > > > > > TopicMetadataResponse:
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > 1. If leader is not
> > > available,
> > > > > we
> > > > > > > set
> > > > > > > > > the
> > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > > > > > > > >> > > 2. If a non-leader
> replica
> > > is
> > > > > not
> > > > > > > > > > available,
> > > > > > > > > > > > we
> > > > > > > > > > > > > > take
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > > > > >> > > the assigned replica
> list
> > > and
> > > > > isr
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > > response.
> > > > > > > > > > > > > > > As
> > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > > > > > > > >> > > doing that, we set the
> > > > partition
> > > > > > > level
> > > > > > > > > > error
> > > > > > > > > > > > > code
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > This has a few problems.
> > > > First,
> > > > > > > > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > > > > > > > >> > > an error, at least for
> the
> > > > > normal
> > > > > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > > > > > > > >> > > to find out the leader.
> > > > Second,
> > > > > it
> > > > > > > can
> > > > > > > > > > > happen
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > both
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> > > another replica are not
> > > > > available
> > > > > > at
> > > > > > > > the
> > > > > > > > > > > same
> > > > > > > > > > > > > > time.
> > > > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > > > > >> > > to indicate both. Third,
> > > even
> > > > > if a
> > > > > > > > > replica
> > > > > > > > > > > is
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > available,
> > > > > > > > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > > > > > > > >> > > useful to return its
> > replica
> > > > id
> > > > > > > since
> > > > > > > > > some
> > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > (e.g.
> > > > > > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > One way to address this
> > > issue
> > > > is
> > > > > > to
> > > > > > > > > always
> > > > > > > > > > > > > return
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > > > > > > > >> > > leader, assigned
> replicas,
> > > and
> > > > > isr
> > > > > > > > > > > regardless
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > > > > > > > >> > > broker is live or not.
> > Since
> > > > we
> > > > > > also
> > > > > > > > > > return
> > > > > > > > > > > > the
> > > > > > > > > > > > > > list
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > >> > > client can figure out
> > > whether
> > > > a
> > > > > > > leader
> > > > > > > > > or
> > > > > > > > > > a
> > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > > > > > > > >> > > accordingly. This way,
> we
> > > > don't
> > > > > > need
> > > > > > > > to
> > > > > > > > > > set
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > > > > >> > > when the leader or a
> > replica
> > > > is
> > > > > > not
> > > > > > > > > > > available.
> > > > > > > > > > > > > > This
> > > > > > > > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > > > > > > > >> > > protocol, but does
> change
> > > the
> > > > > > > > semantics.
> > > > > > > > > > > Since
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > > > > > > > >> > > of TopicMetadataRequest
> > > here,
> > > > we
> > > > > > can
> > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > piggyback
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > 102.1 For those types of
> > > > errors
> > > > > > due
> > > > > > > to
> > > > > > > > > > > invalid
> > > > > > > > > > > > > > > input,
> > > > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > >> > > guard it at parameter
> > > > validation
> > > > > > > time
> > > > > > > > > and
> > > > > > > > > > > > throw
> > > > > > > > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > > > > > > > >> > > without even sending the
> > > > request
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > > broker?
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at
> > > 10:37
> > > > > AM,
> > > > > > > > Andrii
> > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > andrii.biletskyi@stealth.ly
> > > >
> > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > Answering your
> > questions:
> > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > 101. If I understand
> you
> > > > > > > correctly,
> > > > > > > > > you
> > > > > > > > > > > are
> > > > > > > > > > > > > > saying
> > > > > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > > > > > > > >> > > > will be ported to
> > TMR_V1)
> > > > > won't
> > > > > > be
> > > > > > > > > able
> > > > > > > > > > to
> > > > > > > > > > > > > > > > automatically
> > > > > > > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > unconditionally remove
> > > topic
> > > > > > > > creation
> > > > > > > > > > from
> > > > > > > > > > > > > > there).
> > > > > > > > > > > > > > > > But
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > Ok, about your
> proposal:
> > > I'm
> > > > > > not a
> > > > > > > > big
> > > > > > > > > > fan
> > > > > > > > > > > > > too,
> > > > > > > > > > > > > > > > when it
> > > > > > > > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > > > > > > > >> > > > clients directly in
> > > protocol
> > > > > > > schema.
> > > > > > > > > And
> > > > > > > > > > > > also
> > > > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > auto.create.topics.enable
> > > > is a
> > > > > > > > server
> > > > > > > > > > side
> > > > > > > > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > > > > > > > >> > > > in future versions,
> add
> > > this
> > > > > > > setting
> > > > > > > > > to
> > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > based
> > > > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > > > > > > > >> > > > UnknownTopic create
> > topic
> > > > > > > explicitly
> > > > > > > > > by
> > > > > > > > > > a
> > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's
> > > because
> > > > > we
> > > > > > > want
> > > > > > > > > to
> > > > > > > > > > > > > support
> > > > > > > > > > > > > > > > batching
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > want to give
> descriptive
> > > > error
> > > > > > > > > messages
> > > > > > > > > > > for
> > > > > > > > > > > > > > > clients.
> > > > > > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > > > > > > > >> > > > to construct such
> > messages
> > > > > (e.g.
> > > > > > > > > > > AdminClient
> > > > > > > > > > > > > > layer
> > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > > > > > > > > >> > > > means two cases:
> either
> > > > > invalid
> > > > > > > > > number -
> > > > > > > > > > > > e.g.
> > > > > > > > > > > > > > -1;
> > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > > > > > > > >> > > > partitions argument
> > > wasn't)
> > > > -
> > > > > I
> > > > > > > > > wrapped
> > > > > > > > > > > > > > responses
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > > > > > > > >> > > > other ideas, this was
> > just
> > > > > > initial
> > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree.
> > I'll
> > > > > change
> > > > > > > > that
> > > > > > > > > to
> > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015
> at
> > > 7:16
> > > > > PM,
> > > > > > > Jun
> > > > > > > > > > Rao <
> > > > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > 101. That's what I
> was
> > > > > > thinking
> > > > > > > > too,
> > > > > > > > > > but
> > > > > > > > > > > > it
> > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > not be
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > we can let it not
> > > trigger
> > > > > auto
> > > > > > > > topic
> > > > > > > > > > > > > creation.
> > > > > > > > > > > > > > > > Then,
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > > > > > > > UnknownTopicException,
> > > > > > > > > > it
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > createTopicRequest
> for
> > > > auto
> > > > > > > topic
> > > > > > > > > > > > creation.
> > > > > > > > > > > > > On
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > never issue
> > > > > > createTopicRequest.
> > > > > > > > This
> > > > > > > > > > > works
> > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > enabled on the
> broker
> > > > side.
> > > > > > > > > However, I
> > > > > > > > > > > am
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > sure how
> > > > > > > > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > when auto topic
> > creation
> > > > is
> > > > > > > > disabled
> > > > > > > > > > on
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > side.
> > > > > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > want to have a way
> to
> > > > > manually
> > > > > > > > > create
> > > > > > > > > > a
> > > > > > > > > > > > > topic,
> > > > > > > > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > commands. However,
> > then
> > > we
> > > > > > need
> > > > > > > a
> > > > > > > > > way
> > > > > > > > > > to
> > > > > > > > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > issued from the
> > producer
> > > > > > clients
> > > > > > > > and
> > > > > > > > > > the
> > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > tools.
> > > > > > > > > > > > > > > > > > > May
> > > > > > > > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > new field in
> > > > > > createTopicRequest
> > > > > > > > and
> > > > > > > > > > set
> > > > > > > > > > > it
> > > > > > > > > > > > > > > > differently
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > client and the admin
> > > > client.
> > > > > > > > > However,
> > > > > > > > > > I
> > > > > > > > > > > am
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > sure if
> > > > > > > > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring
> > > > existing
> > > > > > > > > requests
> > > > > > > > > > > is a
> > > > > > > > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > posted some comments
> > in
> > > > > > > > KAFKA-1927.
> > > > > > > > > We
> > > > > > > > > > > > will
> > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > first, before adding
> > the
> > > > new
> > > > > > > logic
> > > > > > > > > in
> > > > > > > > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102. About the
> > > > AdminClient:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit
> > weird
> > > > that
> > > > > > we
> > > > > > > > > return
> > > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > we should either
> > return
> > > > > error
> > > > > > > code
> > > > > > > > > or
> > > > > > > > > > > > throw
> > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.2. We probably
> > > > shouldn't
> > > > > > > > > > explicitly
> > > > > > > > > > > > use
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > Not every request
> > > > evolution
> > > > > > > > requires
> > > > > > > > > > an
> > > > > > > > > > > > api
> > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015
> > at
> > > > 4:08
> > > > > > AM,
> > > > > > > > > > Andrii
> > > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > andrii.biletskyi@stealth.ly
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks for you
> > > comments.
> > > > > > > Answers
> > > > > > > > > > > inline:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 100. There are a
> few
> > > > > fields
> > > > > > > such
> > > > > > > > > as
> > > > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > > > PartitionsSerialized
> > > > > > > that
> > > > > > > > > are
> > > > > > > > > > > > > > > represented
> > > > > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
> > structures
> > > > in
> > > > > > > json.
> > > > > > > > > > Could
> > > > > > > > > > > we
> > > > > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
> > definition
> > > as
> > > > > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, now with
> Admin
> > > > Client
> > > > > > > this
> > > > > > > > > > looks
> > > > > > > > > > > a
> > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > weird.
> > > > > > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > ReassignPartitionCommand
> > > > > > > accepts
> > > > > > > > > > input
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > json,
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > interfaces
> > unchanged,
> > > > > where
> > > > > > > > > > possible.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > If we port it to
> > > > > > deserialized
> > > > > > > > > > format,
> > > > > > > > > > > in
> > > > > > > > > > > > > CLI
> > > > > > > > > > > > > > > > (/tools
> > > > > > > > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json library since
> > > > /tools
> > > > > is
> > > > > > > > > written
> > > > > > > > > > > in
> > > > > > > > > > > > > java
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > provided by a
> user.
> > > Can
> > > > we
> > > > > > > > quickly
> > > > > > > > > > > agree
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > what
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON,
> > > > whatever)?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> > > > > > TopicMetadataRequest
> > > > > > > > v1
> > > > > > > > > > > still
> > > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit
> > weird
> > > > now
> > > > > > that
> > > > > > > > we
> > > > > > > > > > > have a
> > > > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought
> about
> > > how
> > > > > the
> > > > > > > new
> > > > > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used
> in
> > > the
> > > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For
> > example,
> > > > > > ideally,
> > > > > > > > we
> > > > > > > > > > > don't
> > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to
> > trigger
> > > > auto
> > > > > > > topic
> > > > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > I agree, this
> > strange
> > > > > logic
> > > > > > > > should
> > > > > > > > > > be
> > > > > > > > > > > > > fixed.
> > > > > > > > > > > > > > > > I'm not
> > > > > > > > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > correct me if I'm
> > > wrong,
> > > > > but
> > > > > > > it
> > > > > > > > > > > doesn't
> > > > > > > > > > > > > look
> > > > > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > leverage
> AdminClient
> > > for
> > > > > > that
> > > > > > > in
> > > > > > > > > > > > Producer
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > > > > > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay
> meant
> > > > > getting
> > > > > > > rid
> > > > > > > > > of
> > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > > > > HeartbeatRequestAndHeader
> > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader.
> We
> > > > > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap
> > thing
> > > > when
> > > > > > > > adding
> > > > > > > > > > the
> > > > > > > > > > > > new
> > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the
> long
> > > term
> > > > > > plan
> > > > > > > is
> > > > > > > > > to
> > > > > > > > > > > get
> > > > > > > > > > > > > rid
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> > > request/response
> > > > in
> > > > > > the
> > > > > > > > > > client.
> > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant
> number
> > > of
> > > > > new
> > > > > > > > > > requests,
> > > > > > > > > > > > > > perhaps
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the
> > > existing
> > > > > > scala
> > > > > > > > > > requests
> > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks like I
> > > > > > > misunderstood
> > > > > > > > > the
> > > > > > > > > > > > point
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > rework that. The
> > only
> > > > > thing
> > > > > > is
> > > > > > > > > that
> > > > > > > > > > I
> > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > see
> > > > > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > least one existing
> > > > > protocol
> > > > > > > > > message.
> > > > > > > > > > > > Thus,
> > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are going to do
> it.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Re porting all
> > > existing
> > > > > > RQ/RP
> > > > > > > in
> > > > > > > > > > this
> > > > > > > > > > > > > patch.
> > > > > > > > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > requirement to
> have
> > > > Admin
> > > > > > KIP
> > > > > > > > > done,
> > > > > > > > > > > I'm
> > > > > > > > > > > > > > afraid
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > There are 13
> > protocol
> > > > > > messages
> > > > > > > > and
> > > > > > > > > > all
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > intensive manual
> > > > testing,
> > > > > > no?
> > > > > > > > I'm
> > > > > > > > > > > afraid
> > > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > > not the
> > > > > > > > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka core
> internals
> > > :).
> > > > > Let
> > > > > > > me
> > > > > > > > > know
> > > > > > > > > > > > your
> > > > > > > > > > > > > > > > thoughts
> > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > follow-up this
> > issue (
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13,
> 2015
> > > at
> > > > > 6:40
> > > > > > > AM,
> > > > > > > > > Jun
> > > > > > > > > > > > Rao <
> > > > > > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > A few more
> > comments.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 100. There are a
> > few
> > > > > > fields
> > > > > > > > such
> > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > > > PartitionsSerialized
> > > > > > > that
> > > > > > > > > are
> > > > > > > > > > > > > > > represented
> > > > > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
> > structures
> > > > in
> > > > > > > json.
> > > > > > > > > > Could
> > > > > > > > > > > we
> > > > > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
> > definition
> > > as
> > > > > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> > > > > > > TopicMetadataRequest
> > > > > > > > > v1
> > > > > > > > > > > > still
> > > > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit
> > weird
> > > > now
> > > > > > that
> > > > > > > > we
> > > > > > > > > > > have a
> > > > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought
> about
> > > how
> > > > > the
> > > > > > > new
> > > > > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used
> in
> > > the
> > > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For
> > example,
> > > > > > ideally,
> > > > > > > > we
> > > > > > > > > > > don't
> > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to
> > trigger
> > > > auto
> > > > > > > topic
> > > > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay
> > meant
> > > > > > getting
> > > > > > > > rid
> > > > > > > > > > of
> > > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > > > > HeartbeatRequestAndHeader
> > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader.
> We
> > > > > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap
> > thing
> > > > when
> > > > > > > > adding
> > > > > > > > > > the
> > > > > > > > > > > > new
> > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the
> long
> > > term
> > > > > > plan
> > > > > > > is
> > > > > > > > > to
> > > > > > > > > > > get
> > > > > > > > > > > > > rid
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> > > request/response
> > > > in
> > > > > > the
> > > > > > > > > > client.
> > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant
> number
> > > of
> > > > > new
> > > > > > > > > > requests,
> > > > > > > > > > > > > > perhaps
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the
> > > existing
> > > > > > scala
> > > > > > > > > > requests
> > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12,
> > 2015
> > > > at
> > > > > > 3:37
> > > > > > > > PM,
> > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > andrii.biletskyi@stealth.ly
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > As said above
> -
> > I
> > > > list
> > > > > > > again
> > > > > > > > > all
> > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can see what's
> > > left
> > > > > and
> > > > > > > > > finalize
> > > > > > > > > > > all
> > > > > > > > > > > > > > > pending
> > > > > > > > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from
> > Jay:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is
> much
> > > > needed
> > > > > > > > > > > > functionality,
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > really think
> > these
> > > > > > > protocols
> > > > > > > > > > > > through.
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > of well
> > > thought-out,
> > > > > > > > > orthoganol
> > > > > > > > > > > > apis.
> > > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > important to
> > think
> > > > > > through
> > > > > > > > the
> > > > > > > > > > end
> > > > > > > > > > > > > state
> > > > > > > > > > > > > > > > even if
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > won't
> implement
> > in
> > > > the
> > > > > > > first
> > > > > > > > > > > phase.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Definitely
> > > behind
> > > > > > this.
> > > > > > > > > Would
> > > > > > > > > > > > > > > appreciate
> > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > how this can
> be
> > > > > > improved.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's
> please
> > > > please
> > > > > > > > please
> > > > > > > > > > wait
> > > > > > > > > > > > > until
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to the new
> java
> > > > > protocol
> > > > > > > > > > > > definitions.
> > > > > > > > > > > > > If
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > objects that
> is
> > > just
> > > > > > > > > generating
> > > > > > > > > > > more
> > > > > > > > > > > > > > work
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in
> the
> > > > latest
> > > > > > > > patch -
> > > > > > > > > > > > removed
> > > > > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This
> proposal
> > > > > > > introduces
> > > > > > > > a
> > > > > > > > > > new
> > > > > > > > > > > > type
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > inconsistent
> > with
> > > > > > > everything
> > > > > > > > > > else
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > other marker
> > > value.
> > > > > You
> > > > > > > > could
> > > > > > > > > > > argue
> > > > > > > > > > > > > > either
> > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > for
> consistency.
> > > For
> > > > > > > clients
> > > > > > > > > > that
> > > > > > > > > > > > > > > > implemented
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than our scala
> > > code
> > > > > > these
> > > > > > > > > basic
> > > > > > > > > > > > > > primitives
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in
> the
> > > > latest
> > > > > > > > patch -
> > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > MaybeOf
> > > > > > > > > > > > > > > > > > > type
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 4.
> > > ClusterMetadata:
> > > > > This
> > > > > > > > seems
> > > > > > > > > > to
> > > > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers,
> topics,
> > > and
> > > > > > > > > > partitions. I
> > > > > > > > > > > > > think
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > ClusterMetadataRequest
> > > > > > (or
> > > > > > > > > just
> > > > > > > > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > controller. Or
> > are
> > > > > there
> > > > > > > > other
> > > > > > > > > > > > things
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I agree.
> > > Updated
> > > > > the
> > > > > > > KIP.
> > > > > > > > > > Let's
> > > > > > > > > > > > > > extends
> > > > > > > > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > include
> > > controller.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a
> > > > tendency
> > > > > to
> > > > > > > try
> > > > > > > > > to
> > > > > > > > > > > > make a
> > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > particular
> > nodes.
> > > > This
> > > > > > > adds
> > > > > > > > a
> > > > > > > > > > lot
> > > > > > > > > > > of
> > > > > > > > > > > > > > > burden
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > sounds easy
> but
> > > each
> > > > > > > > discovery
> > > > > > > > > > can
> > > > > > > > > > > > > fail
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > many
> > > > > > > > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > full state
> > machine
> > > > to
> > > > > do
> > > > > > > > > > right). I
> > > > > > > > > > > > > think
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > commands and
> > > ideally
> > > > > as
> > > > > > > many
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers and
> just
> > > > > > redirect
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > there would
> be a
> > > > > general
> > > > > > > way
> > > > > > > > > to
> > > > > > > > > > > > > > > encapsulate
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a very
> > > > > > interesting
> > > > > > > > > idea,
> > > > > > > > > > > but
> > > > > > > > > > > > > > seems
> > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > feature (like
> > > > > > performance
> > > > > > > > > > > > > > considerations,
> > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I believe this
> > > > > shouldn't
> > > > > > > be
> > > > > > > > a
> > > > > > > > > > > > blocker.
> > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point it won't
> > > > affect
> > > > > > > Admin
> > > > > > > > > > > changes
> > > > > > > > > > > > -
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > least
> > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
> > required.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We should
> > > > probably
> > > > > > > > > normalize
> > > > > > > > > > > the
> > > > > > > > > > > > > key
> > > > > > > > > > > > > > > > value
> > > > > > > > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than
> embedding a
> > > new
> > > > > > > > > formatting.
> > > > > > > > > > > So
> > > > > > > > > > > > > two
> > > > > > > > > > > > > > > > strings
> > > > > > > > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > internal
> equals
> > > > sign.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in
> the
> > > > latest
> > > > > > > > patch -
> > > > > > > > > > > > > > normalized
> > > > > > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the
> > > > > postcondition
> > > > > > of
> > > > > > > > > these
> > > > > > > > > > > > APIs
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the command
> has
> > > been
> > > > > > > > > completed?
> > > > > > > > > > It
> > > > > > > > > > > > is
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > been completed
> > so
> > > > you
> > > > > > know
> > > > > > > > > that
> > > > > > > > > > if
> > > > > > > > > > > > you
> > > > > > > > > > > > > > > > create a
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it you won't
> get
> > > an
> > > > > > > > exception
> > > > > > > > > > > about
> > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > being
> > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: For long
> > > running
> > > > > > > requests
> > > > > > > > > > (like
> > > > > > > > > > > > > > > reassign
> > > > > > > > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command has
> > begun
> > > -
> > > > so
> > > > > > we
> > > > > > > > > don't
> > > > > > > > > > > > block
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > client.
> > > > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic
> commands,
> > > this
> > > > > > will
> > > > > > > be
> > > > > > > > > > > > > refactored
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > immediately,
> > since
> > > > the
> > > > > > > > > > Controller
> > > > > > > > > > > > will
> > > > > > > > > > > > > > > serve
> > > > > > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up
> > ticket
> > > > > > > > KAFKA-1777).
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe
> > topic
> > > > and
> > > > > > list
> > > > > > > > > > topics
> > > > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request. Is
> > there
> > > a
> > > > > > reason
> > > > > > > > to
> > > > > > > > > > give
> > > > > > > > > > > > > back
> > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like if we
> just
> > > make
> > > > > the
> > > > > > > > > > > > > post-condition
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic is
> deleted
> > > > that
> > > > > > will
> > > > > > > > get
> > > > > > > > > > rid
> > > > > > > > > > > > of
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be much more
> > > > > intuitive.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in
> the
> > > > latest
> > > > > > > > patch -
> > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > ListTopicsRequest.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we
> > > > consider
> > > > > > > > batching
> > > > > > > > > > > these
> > > > > > > > > > > > > > > > requests?
> > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > allow multiple
> > > > > > operations
> > > > > > > to
> > > > > > > > > be
> > > > > > > > > > > > > batched.
> > > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we will get a
> > lot
> > > of
> > > > > > code
> > > > > > > > that
> > > > > > > > > > > does
> > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > > > > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this code will
> > > work
> > > > > > great
> > > > > > > > when
> > > > > > > > > > you
> > > > > > > > > > > > > test
> > > > > > > > > > > > > > > on 5
> > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the
> > > KIP -
> > > > > > > please
> > > > > > > > > > check
> > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I think we
> > > > should
> > > > > > also
> > > > > > > > > > discuss
> > > > > > > > > > > > how
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > client api for
> > > these
> > > > > > > > > operations.
> > > > > > > > > > > > > > Currently
> > > > > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > is totally
> > > sketchy.
> > > > I
> > > > > > > think
> > > > > > > > we
> > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > that exposes
> > > > > > > administrative
> > > > > > > > > > > > > > functionality.
> > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > properly test
> > the
> > > > new
> > > > > > > apis,
> > > > > > > > I
> > > > > > > > > > > > suspect.
> > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the
> > > KIP -
> > > > > > > please
> > > > > > > > > > check
> > > > > > > > > > > > > "Admin
> > > > > > > > > > > > > > > > Client"
> > > > > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The other
> > > > > > information
> > > > > > > > that
> > > > > > > > > > > would
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > information
> > about
> > > > > > > > > > partitions--how
> > > > > > > > > > > > much
> > > > > > > > > > > > > > > data
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the segment
> > > offsets,
> > > > > > what
> > > > > > > is
> > > > > > > > > the
> > > > > > > > > > > > > log-end
> > > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the compaction
> > > > point,
> > > > > > > etc. I
> > > > > > > > > > think
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > done
> > > > > > > > > > > > > > > > > > right
> > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > successor to
> the
> > > > very
> > > > > > > > awkward
> > > > > > > > > > > > > > > OffsetRequest
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > > > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be resolved
> in a
> > > > > > separate
> > > > > > > > KIP
> > > > > > > > > /
> > > > > > > > > > > jira
> > > > > > > > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 12. Generally
> we
> > > can
> > > > > do
> > > > > > > good
> > > > > > > > > > error
> > > > > > > > > > > > > > > handling
> > > > > > > > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > messages. I.e.
> > > > > generally
> > > > > > > the
> > > > > > > > > > > client
> > > > > > > > > > > > > has
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > context
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > an error that
> > the
> > > > > topic
> > > > > > > > > doesn't
> > > > > > > > > > > > exist
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > say
> > > > > > > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than "error
> code
> > > 14"
> > > > > (or
> > > > > > > > > > > whatever).
> > > > > > > > > > > > > > Maybe
> > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this is hard?
> If
> > > we
> > > > > want
> > > > > > > to
> > > > > > > > > add
> > > > > > > > > > > > > > > server-side
> > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > need to do
> this
> > > in a
> > > > > > > > > consistent
> > > > > > > > > > > way
> > > > > > > > > > > > > > across
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the
> > > KIP -
> > > > > > > please
> > > > > > > > > > check
> > > > > > > > > > > > > > > "Protocol
> > > > > > > > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > comprehensive,
> > > > > > > fine-grained
> > > > > > > > > list
> > > > > > > > > > > of
> > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from
> > > > > Guozhang:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe
> > topic
> > > > > > > request:
> > > > > > > > it
> > > > > > > > > > > would
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > great to
> > > > > > > > > > > > > > > > > > > go
> > > > > > > > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name
> regex
> > > for
> > > > > > this
> > > > > > > > > > request.
> > > > > > > > > > > > For
> > > > > > > > > > > > > > > > example,
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the topic
> > command
> > > is
> > > > > to
> > > > > > > list
> > > > > > > > > all
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name
> regex
> > > > then
> > > > > we
> > > > > > > > have
> > > > > > > > > to
> > > > > > > > > > > > first
> > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > description
> info
> > > and
> > > > > > then
> > > > > > > > > filter
> > > > > > > > > > > at
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config
> K-Vs
> > in
> > > > > > create
> > > > > > > > > topic:
> > > > > > > > > > > > this
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > related
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > maybe we can
> add
> > > > > another
> > > > > > > > > > metadata
> > > > > > > > > > > > K-V
> > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > with config
> K-V
> > in
> > > > > > create
> > > > > > > > > topic
> > > > > > > > > > > like
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > field can be
> > quite
> > > > > > useful
> > > > > > > in
> > > > > > > > > > > storing
> > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > who issue the
> > > create
> > > > > > > > command,
> > > > > > > > > > etc,
> > > > > > > > > > > > > which
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant
> > > > setting.
> > > > > > Then
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > on regex of
> the
> > > > > metadata
> > > > > > > > > field.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: As
> discussed
> > it
> > > > is
> > > > > > very
> > > > > > > > > > > > interesting
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we have some
> > basic
> > > > > > > > > functionality
> > > > > > > > > > > > > there.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today all
> > the
> > > > > admin
> > > > > > > > > > operations
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > async in
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > return once it
> > is
> > > > > > written
> > > > > > > in
> > > > > > > > > ZK,
> > > > > > > > > > > and
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > is why
> > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > > > > > > > testUtil.waitForTopicCreated()
> > > > > > > > > > > > /
> > > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request, etc.
> > With
> > > > > admin
> > > > > > > > > > requests
> > > > > > > > > > > we
> > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > synchronous
> > > > requests;
> > > > > > when
> > > > > > > > it
> > > > > > > > > is
> > > > > > > > > > > > > turned
> > > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > until the
> > request
> > > > has
> > > > > > been
> > > > > > > > > > > > completed.
> > > > > > > > > > > > > > And
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > "token" field
> in
> > > the
> > > > > > > > response,
> > > > > > > > > > and
> > > > > > > > > > > > > then
> > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > verification
> > > > request"
> > > > > > with
> > > > > > > > the
> > > > > > > > > > > given
> > > > > > > > > > > > > > token
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > has been
> > > completed.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I see your
> > > point.
> > > > > My
> > > > > > > idea
> > > > > > > > > was
> > > > > > > > > > > to
> > > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long running
> > > > request,
> > > > > > > where
> > > > > > > > > > > needed.
> > > > > > > > > > > > We
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > do it
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > concern is
> that
> > > > > > > introducing
> > > > > > > > a
> > > > > > > > > > > token
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > again
> > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to do similar
> > > thing
> > > > > > > > > introducing
> > > > > > > > > > > > single
> > > > > > > > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this idea
> > because
> > > we
> > > > > > > wanted
> > > > > > > > to
> > > > > > > > > > > have
> > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > choice
> between:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > a) have fixed
> > > schema
> > > > > but
> > > > > > > > > > introduce
> > > > > > > > > > > > > each
> > > > > > > > > > > > > > > > time new
> > > > > > > > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long-running
> > > > requests
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > b) use one
> > request
> > > > for
> > > > > > > > > > > verification
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > generalize
> > > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine with
> > > > whatever
> > > > > > > > > decision
> > > > > > > > > > > > > > community
> > > > > > > > > > > > > > > > come
> > > > > > > > > > > > > > > > > > to.
> > > > > > > > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comment from
> > Gwen:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 16.
> Specifically
> > > for
> > > > > > > > > ownership,
> > > > > > > > > > I
> > > > > > > > > > > > > think
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like you are
> > > > > describing
> > > > > > > ACL)
> > > > > > > > > via
> > > > > > > > > > > an
> > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I remember
> > KIP-11
> > > > > > > described
> > > > > > > > > > this,
> > > > > > > > > > > > but
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no
> > > problem.
> > > > > Not
> > > > > > > > sure
> > > > > > > > > > > though
> > > > > > > > > > > > > how
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
> > committed
> > > > > first
> > > > > > > and
> > > > > > > > > > > include
> > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I
> added
> > > this
> > > > > > note
> > > > > > > to
> > > > > > > > > > "Open
> > > > > > > > > > > > > > > > Questions"
> > > > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii
> Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar
> 13,
> > > 2015
> > > > > at
> > > > > > > > 12:34
> > > > > > > > > > AM,
> > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > andrii.biletskyi@stealth.ly
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Today I
> > uploaded
> > > > the
> > > > > > > patch
> > > > > > > > > > that
> > > > > > > > > > > > > covers
> > > > > > > > > > > > > > > > some of
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - removed
> > > MaybeOf
> > > > > > > optional
> > > > > > > > > > type
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - switched
> to
> > > java
> > > > > > > > protocol
> > > > > > > > > > > > > > definitions
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - simplified
> > > > > messages
> > > > > > > > > > > (normalized
> > > > > > > > > > > > > > > configs,
> > > > > > > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I also
> updated
> > > the
> > > > > > KIP-4
> > > > > > > > > with
> > > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > pending
> items:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch
> Admin
> > > > > > Operations
> > > > > > > > ->
> > > > > > > > > > > > updated
> > > > > > > > > > > > > > Wire
> > > > > > > > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove
> > > > > > ClusterMetadata
> > > > > > > > ->
> > > > > > > > > > > > changed
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > extend
> > > > > > > > > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin
> Client
> > > ->
> > > > > > > updated
> > > > > > > > my
> > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > proposal
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Error
> codes
> > ->
> > > > > > > proposed
> > > > > > > > > > > > > fine-grained
> > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > AdminRequestFailed
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I will also
> > > send a
> > > > > > > > separate
> > > > > > > > > > > email
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > cover all
> > > > > > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii
> > Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar
> > 12,
> > > > 2015
> > > > > > at
> > > > > > > > 9:26
> > > > > > > > > > PM,
> > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Found
> KIP-11
> > (
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> It actually
> > > > > specifies
> > > > > > > > > changes
> > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs
> are
> > > > > > > consistent
> > > > > > > > in
> > > > > > > > > > > this
> > > > > > > > > > > > > > regard
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar
> > 12,
> > > > > 2015
> > > > > > at
> > > > > > > > > 12:21
> > > > > > > > > > > PM,
> > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > Specifically
> > > > for
> > > > > > > > > > ownership, I
> > > > > > > > > > > > > think
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > like you
> > are
> > > > > > > describing
> > > > > > > > > > ACL)
> > > > > > > > > > > > via
> > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I
> remember
> > > > KIP-11
> > > > > > > > > described
> > > > > > > > > > > > this,
> > > > > > > > > > > > > > > but I
> > > > > > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > Regardless, I
> > > > > think
> > > > > > > > KIP-4
> > > > > > > > > > > > focuses
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > exists
> from
> > > > Kafka
> > > > > > > > > brokers,
> > > > > > > > > > > not
> > > > > > > > > > > > on
> > > > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > should
> > exist
> > > > but
> > > > > > > > doesn't
> > > > > > > > > > yet?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu,
> Mar
> > > 12,
> > > > > > 2015
> > > > > > > at
> > > > > > > > > > 6:37
> > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just
> want
> > to
> > > > > > > > elaborate a
> > > > > > > > > > bit
> > > > > > > > > > > > > more
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > describe-topic
> > > > > > based
> > > > > > > > on
> > > > > > > > > > > > config /
> > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on
> > > KAFKA-1694.
> > > > > The
> > > > > > > > main
> > > > > > > > > > > > > motivation
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > mechanisms,
> > > > > which
> > > > > > I
> > > > > > > > > think
> > > > > > > > > > is
> > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > architecture:
> > > > > > today
> > > > > > > > > anyone
> > > > > > > > > > > can
> > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> cluster,
> but
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> there is
> > no
> > > > > > concept
> > > > > > > or
> > > > > > > > > > > > > "ownership"
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> users.
> For
> > > > > > example,
> > > > > > > at
> > > > > > > > > > > > LinkedIn
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some
> > casual
> > > > > topic
> > > > > > > name
> > > > > > > > > > > prefix,
> > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > is a
> > > > > > > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale
> > our
> > > > > > > > customers.
> > > > > > > > > It
> > > > > > > > > > > > would
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe
> > all
> > > > > > topics
> > > > > > > > that
> > > > > > > > > > is
> > > > > > > > > > > > > > created
> > > > > > > > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe
> > all
> > > > > > topics
> > > > > > > > > whose
> > > > > > > > > > > > > > retention
> > > > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe
> > all
> > > > > > topics
> > > > > > > > > whose
> > > > > > > > > > > > > writable
> > > > > > > > > > > > > > > > group
> > > > > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > authorization),
> > > > > > > etc..
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One
> > possible
> > > > way
> > > > > > to
> > > > > > > > > > achieve
> > > > > > > > > > > > this
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > create-topic
> > > > > > > request,
> > > > > > > > > > whose
> > > > > > > > > > > > > value
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic;
> > then
> > > > > > > > > > describe-topics
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > > choose to
> > > > > > > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex,
> 2)
> > > > config
> > > > > > K-V
> > > > > > > > > > > matching,
> > > > > > > > > > > > > 3)
> > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> Thoughts?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu,
> > Mar
> > > 5,
> > > > > > 2015
> > > > > > > at
> > > > > > > > > > 4:37
> > > > > > > > > > > > PM,
> > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks
> > for
> > > > the
> > > > > > > > updated
> > > > > > > > > > > wiki.
> > > > > > > > > > > > A
> > > > > > > > > > > > > > few
> > > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1.
> Error
> > > > > > > description
> > > > > > > > in
> > > > > > > > > > > > > > response: I
> > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> several
> > > > > different
> > > > > > > > error
> > > > > > > > > > > cases
> > > > > > > > > > > > > > then
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes.
> In
> > > > > general
> > > > > > > the
> > > > > > > > > > > > errorCode
> > > > > > > > > > > > > > > > itself
> > > > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> sufficient
> > for
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > describing
> > > > the
> > > > > > > server
> > > > > > > > > > side
> > > > > > > > > > > > > > errors.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2.
> > Describe
> > > > > topic
> > > > > > > > > > request:
> > > > > > > > > > > it
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic
> > name
> > > > > regex
> > > > > > > for
> > > > > > > > > this
> > > > > > > > > > > > > > request.
> > > > > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the
> topic
> > > > > command
> > > > > > > is
> > > > > > > > to
> > > > > > > > > > > list
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic
> > name
> > > > > regex
> > > > > > > then
> > > > > > > > > we
> > > > > > > > > > > have
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > description
> > > > > info
> > > > > > > and
> > > > > > > > > then
> > > > > > > > > > > > > filter
> > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden
> on
> > > ZK.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3.
> Config
> > > > K-Vs
> > > > > in
> > > > > > > > > create
> > > > > > > > > > > > topic:
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe
> we
> > > can
> > > > > add
> > > > > > > > > another
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > K-V or
> > > > > > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with
> > config
> > > > K-V
> > > > > > in
> > > > > > > > > create
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request.
> This
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field
> can
> > > be
> > > > > > quite
> > > > > > > > > useful
> > > > > > > > > > > in
> > > > > > > > > > > > > > > storing
> > > > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who
> issue
> > > the
> > > > > > > create
> > > > > > > > > > > command,
> > > > > > > > > > > > > > etc,
> > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > multi-tenant
> > > > > > > setting.
> > > > > > > > > > Then
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on
> regex
> > of
> > > > the
> > > > > > > > > metadata
> > > > > > > > > > > > field.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4.
> Today
> > > all
> > > > > the
> > > > > > > > admin
> > > > > > > > > > > > > operations
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> return
> > once
> > > > it
> > > > > is
> > > > > > > > > written
> > > > > > > > > > > in
> > > > > > > > > > > > > ZK,
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> verification
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> request,
> > > etc.
> > > > > > With
> > > > > > > > > admin
> > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > synchronous
> > > > > > > requests;
> > > > > > > > > > when
> > > > > > > > > > > it
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > turned
> > > > > > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until
> the
> > > > > request
> > > > > > > has
> > > > > > > > > > been
> > > > > > > > > > > > > > > > completed. And
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token"
> > > field
> > > > > in
> > > > > > > the
> > > > > > > > > > > > response,
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > verification
> > > > > > > request"
> > > > > > > > > > with
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > given
> > > > > > > > > > > > > > > > > > token
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has
> been
> > > > > > completed.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1
> for
> > > > > > extending
> > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > information,
> > > > > and
> > > > > > > then
> > > > > > > > > we
> > > > > > > > > > > can
> > > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > ClusterMetadata
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> requests.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> Guozhang
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue,
> > Mar
> > > > 3,
> > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > 10:23
> > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> jjkoshy.w@gmail.com
> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks
> > for
> > > > > > sending
> > > > > > > > > that
> > > > > > > > > > > out
> > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > -
> > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it
> > today,
> > > so
> > > > > if
> > > > > > > > notes
> > > > > > > > > > can
> > > > > > > > > > > be
> > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On
> Mon,
> > > Mar
> > > > > 02,
> > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > > > > 09:16:13AM
> > > > > > > > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> Thanks
> > > for
> > > > > > > sending
> > > > > > > > > > this
> > > > > > > > > > > > out
> > > > > > > > > > > > > > Joe.
> > > > > > > > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> everyone
> > > :)
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On
> > Mon,
> > > > Mar
> > > > > 2,
> > > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > > 6:46
> > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > Hey, I
> > > > > just
> > > > > > > sent
> > > > > > > > > > out a
> > > > > > > > > > > > > > google
> > > > > > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > everyone I
> > > > > > > found
> > > > > > > > > > > working
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> let
> > me
> > > > > know
> > > > > > > and
> > > > > > > > > can
> > > > > > > > > > > > update
> > > > > > > > > > > > > > it,
> > > > > > > > > > > > > > > > np.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We
> > > > should
> > > > > do
> > > > > > > > this
> > > > > > > > > > > every
> > > > > > > > > > > > > > > Tuesday
> > > > > > > > > > > > > > > > @
> > > > > > > > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> help
> > > to
> > > > > > make a
> > > > > > > > > > google
> > > > > > > > > > > > > > account
> > > > > > > > > > > > > > > > so we
> > > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To
> > > > discuss
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in
> > > > > progress
> > > > > > > and
> > > > > > > > > > > related
> > > > > > > > > > > > > JIRA
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~
> > Joe
> > > > > Stein
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On
> > > Tue,
> > > > > Feb
> > > > > > > 24,
> > > > > > > > > 2015
> > > > > > > > > > > at
> > > > > > > > > > > > > 2:59
> > > > > > > > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > jay.kreps@gmail.com
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > Let's
> > > > > stay
> > > > > > on
> > > > > > > > > > Google
> > > > > > > > > > > > > > hangouts
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> sessions
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > available
> > > > > > on
> > > > > > > > > > youtube.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > -Jay
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> On
> > > Tue,
> > > > > Feb
> > > > > > > 24,
> > > > > > > > > > 2015
> > > > > > > > > > > at
> > > > > > > > > > > > > > 11:49
> > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > jholoman@cloudera.com
> > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > Jay /
> > > > > Joe
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > We're
> > > > > > happy
> > > > > > > > to
> > > > > > > > > > send
> > > > > > > > > > > > > out a
> > > > > > > > > > > > > > > > Webex
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > sessions
> > > > > > if
> > > > > > > > > there
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > interest and
> > > > > > > > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > Thanks
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > Jeff
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > On
> > > > Tue,
> > > > > > Feb
> > > > > > > > 24,
> > > > > > > > > > > 2015
> > > > > > > > > > > > at
> > > > > > > > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > Let's
> > > > > > try
> > > > > > > > to
> > > > > > > > > > get
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > technical
> > > > > > > > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > think
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > there
> > > > > > is
> > > > > > > > some
> > > > > > > > > > > > benefit
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> hopeful
> > > that
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> if
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > we
> > > > > post
> > > > > > > > > > > > instructions
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > give
> > > > > > > > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > > > working.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > > Tuesday
> > > > > > > at
> > > > > > > > > that
> > > > > > > > > > > > time
> > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > work
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > -Jay
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > On
> > > > > Tue,
> > > > > > > Feb
> > > > > > > > > 24,
> > > > > > > > > > > > 2015
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > 8:18
> > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > > Weekly
> > > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > maybe
> > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > ????
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > I
> > > > > > don't
> > > > > > > > > mind
> > > > > > > > > > > > google
> > > > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > whatever
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > so
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > we
> > > > > > know
> > > > > > > > the
> > > > > > > > > > > > apache
> > > > > > > > > > > > > > irc
> > > > > > > > > > > > > > > > > > channel
> > > > > > > > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see
> how
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> it
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > goes?
> > > > > > > We
> > > > > > > > > can
> > > > > > > > > > > pull
> > > > > > > > > > > > > > > > transcripts
> > > > > > > > > > > > > > > > > > > too
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need
> be
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > makes
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > it
> > > > > > > > helpful
> > > > > > > > > > for
> > > > > > > > > > > > > > things.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > ~
> > > > > > > > Joestein
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > On
> > > > > > Tue,
> > > > > > > > Feb
> > > > > > > > > > 24,
> > > > > > > > > > > > > 2015
> > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > 11:10
> > > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > jay.kreps@gmail.com
> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > We'd
> > > > > > > > > talked
> > > > > > > > > > > > about
> > > > > > > > > > > > > > > > doing a
> > > > > > > > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > about
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > > > > > generalizing
> > > > > > > > > > > > > that a
> > > > > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would
> be
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > good
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > for
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > > > everyone
> > > > > > > > > > > > > spending a
> > > > > > > > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > maybe
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > sync
> > > > > > > up
> > > > > > > > > > once
> > > > > > > > > > > a
> > > > > > > > > > > > > > week.
> > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> through
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > design
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > > stuff,
> > > > > > > > > make
> > > > > > > > > > > > sure
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > are on
> > > > > > > > > > > > > > > > > > > top
> > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > tricky
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > > issues,
> > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > We
> > > > > > > can
> > > > > > > > > make
> > > > > > > > > > > it
> > > > > > > > > > > > > > > publicly
> > > > > > > > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > who
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > > likes.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > Any
> > > > > > > > > > interest
> > > > > > > > > > > in
> > > > > > > > > > > > > > doing
> > > > > > > > > > > > > > > > this?
> > > > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> starting
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > next
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > week.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > -Jay
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > On
> > > > > > > Tue,
> > > > > > > > > Feb
> > > > > > > > > > > 24,
> > > > > > > > > > > > > > 2015
> > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > >
> > > > > > Hi
> > > > > > > > all,
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > >
> > > > > > > I've
> > > > > > > > > > > updated
> > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > page,
> > > > > > > > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > >
> > > > added
> > > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > >
> > > > > >
> > > > > > > some
> > > > > > > > > > > > > >
> > > > > > > > > > > > > ...
> > > > > > > > > > > > >
> > > > > > > > > > > > > [Message clipped]
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > --
> > > > > > > > > > > -- Guozhang
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> >
>



-- 
Thanks,
Neha

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
Andrii,


111.4 Yes, we can discuss this in the meeting today.

112. The question is when will a PartitionReassignment fail. Currently, it
can fail if the input is incorrect (e.g, invalid partition/replica).
However, this can be detected when the ReassignPartitionRequest is issued.
Once the process of partition reassignment is started, it will not fail.
The controller just waits until the process completes. We can improve the
status reporting of partition reassignment. However, I am not sure if that
needs to be tied to this KIP.

115.1 Yes, that makes sense. I misunderstood it. I thought
PreferredReplicaLeaderElectionInProgress and ReassignPartitionsInProgress
are used in the status check.

Thanks,

Jun


On Tue, Mar 24, 2015 at 3:46 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Jun,
>
> Thanks for such an accurate review!
>
> Most of your remarks I didn't fix in the previous change because
> I thought we'd finalize them today, during the call. Anyway, I've
> fixed them now.
> My comments to ensure I covered everything:
>
> 110. Fixed. Removed global error code.
>
> 111.1-3. Removed redundant TopicName, Partitions and ReplicationFactor
> 111.4 Yes, now we are in the situation when TopicMetadataResponse
> contains everything we need except topic level configs. I'm
> okay with both solutions, probably makes sense to discuss and pick
> the preferable one.
>
> 112. Need to think about it a bit more. Currently we may distinguish
> 3 states of specific partition being reassigned:
> a) Completed - absent in /reassign_partitions zk path and DescribeTopic/
> TopicMetadata shows expected assignment
> 2) In progress - present in /reassign_partitions
> 3) Failed - absent in /reassign_partitions but DescribeTopic/TopicMetadata
> replicas field doesn't correspond expected
> Now, if use only DescribeTopic to check reassignment status - how can we
> distinguish states "In Progress" and "Failed"?
> Also, there are some concerns that current reassignment status provided
> by ReaasignPartitionsCommand (and the same was intended in KIP-4) is
> very spurious, non-informative. Probably it makes sense to add additional
> data there (thus separate request may be useful), I'm considering different
> options now - btw, comments here are highly appreciated!
>
> 113-114. Okay, made it compliant with topic commands. About "The error code
> will
> then be per topic" - just a note: currently (ReassignPartitionsCommand)
> validation
> is done for the whole reassignment string, so in case of errors for
> specific
> partitions the reassignment is NOT started at all. Both options look fine
> for me,
> it's just the logic will be slightly changed.
>
> 115.1 Removed NotControllerReceivedAdminCommand. But why should we
> remove PreferredReplicaLeaderElectionInProgress and
> ReassignPartitionsInProgress?
> On Reassign/Preferred..Request we create admin zk path, we can of course
> update the path, but wouldn't it be safer if we simply refuse to start new
> reassignment in case one is in progress?
>
> 115.2 Yes, paragraph Protocol Errors covers only errors that are proposed
> to be added. I didn't want to list all errors that are to be added since
> anyway
> the final version will be clear after the development is completed.
>
> 100. Fixed, normalized json string in Create/Alter.
>
> Thanks,
> Andrii Biletskyi
>
> On Tue, Mar 24, 2015 at 12:58 AM, Jun Rao <ju...@confluent.io> wrote:
>
> > Andrii,
> >
> > I looked at the latest wiki for this KIP. I have a few more comments.
> >
> > 110. In CreateTopicResponse, AlterTopicResponse, DeleteTopicResponse and
> > DescribeTopicResponse, we probably don't need the global error code. Our
> > current convention is to just use the same global error code in each
> topic.
> > This simplifies the error checking on the client side.
> >
> > 111. DescribeTopicResponse:
> > 111.1 Our protocol definition doesn't support optional fields. So, we
> can't
> > make Leader an optional field.
> > 111.2 Do we really need the field Partitions and ReplicationFactor in
> > TopicConfigDetails?
> > The former is basically the size of the TopicPartitionDetails array and
> the
> > latter can be found from the size of the Replica array.
> > 111.3 Do we need TopicName in TopicDescription since it's already in
> > DescribeTopicResponse?
> > 111.4 Finally, DescribeTopicResponse is very similar to
> > TopicMetadataResponse. So, it probably makes sense just to keep one of
> them
> > in the future. Should we just use DescribeTopicRequest/Response to
> replace
> > TopicMetadataRequest/Response in the producer/consumer client in the
> future
> > (since DescribeTopicRequest doesn't trigger auto topic creation)? If so,
> we
> > will need to add the broker list in DescribeTopicResponse.
> >
> > 112. Thinking about this a bit more, I don't think we need a separate
> > VerifyReassignPartitionRequest/Response.
> > We can just use DescribeTopic to get the assigned replicas and check if
> > they match the target replica assignment in the client. The controller
> > propagates the metadata change after the reassignment completes for each
> > partition.
> >
> > 113. ReassignPartitionRequest: For consistency, we probably want to nest
> > the partition data under topic. So instead of
> >   ReassignPartitionRequest => [Topic PartitionId [ReplicaId]]
> > we can have sth like
> >   ReassignPartitionRequest => [Topic [PartitionId [ReplicaId]]]
> > The error code will then be per topic.
> >
> > 114. PreferredReplicaLeaderElectionRequest: Same as the above. Instead of
> >   PreferredReplicaLeaderElectionRequest => [Topic PartitionId]
> > we can have
> >   PreferredReplicaLeaderElectionRequest => [Topic [PartitionId]]
> > Again, the error code will be per topic.
> >
> > 115. ErrorCode:
> > 115.1 NotControllerReceivedAdminRequest,
> > PreferredReplicaLeaderElectionInProgress
> > and ReassignPartitionsInProgress are likely not needed any more.
> > 115.2 We probably need to add UnknownTopicOrPartitionCode since it's a
> > valid error code for some of the requests (e.g. describe, alter, etc)
> when
> > a non-existing topic is specified.
> >
> >
> > Previous comments.
> > 100. ReplicaAssignment in CreateTopicRequest and AlterTopicRequest are
> > still a json string. We need to flatten them.
> >
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Sat, Mar 21, 2015 at 2:03 AM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Guozhang,
> > >
> > > I'm not sure I understand how can we use those tools in CLI.
> > > First of all, those are *Test*Utils, e.g. waitUntilMetadataIsPropagated
> > > (which might be very useful for us) requires servers:Seq[KafkaServer]
> > > as an argument:
> > >
> > > def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic:
> > String,
> > > partition: Int, timeout: Long = 5000L): Int
> > >
> > > how can we have access it at runtime in one of the brokers?
> > >
> > > Secondly, how can user directly call these tools if zookeeper might be
> > > not accessible (which is used in the tools, right?) at all - e.g.
> behind
> > > the VPC
> > > in AWS. Furthermore, I think with KIP-4 we are trying to create an
> > > abstraction and
> > > a single point of zookeeper interactions, to eliminate direct calls to
> ZK
> > > either to get
> > > some cluster information or to change something.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Sat, Mar 21, 2015 at 3:37 AM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > Andrii,
> > > >
> > > > Actually the checking logic Jun mentioned is already implemented as
> > > > TestUtils.waitUntilXXX (LeaderIsElected, MetadataIsPropagated,
> etc...)
> > I
> > > > think we can extend these functions as CLI tools like TopicCommand so
> > > that
> > > > users re-implementing such endpoint can directly call something like
> > > > java.tools.WaitUntilXXX (of course this requires them to have javac
> > > > installed, which should be a reasonable requirement?)
> > > >
> > > > Guozhang
> > > >
> > > >
> > > >
> > > > On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Jun,
> > > > >
> > > > > Not that I was saying we need to make requests blocking on server,
> > > > > it was just to emphasize that with async requests a client
> > > > implementations
> > > > > may be a little bit more than just issue request - get the
> response.
> > > > > Thanks for the explanation, I understand now that we can go with
> > agreed
> > > > > solution though it may not be perfect.
> > > > > I believe this was one of the last controversial questions from the
> > > list.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > > On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > >
> > > > > > Andrii,
> > > > > >
> > > > > > A few points.
> > > > > >
> > > > > > 1. Create/Alter can typically complete quickly. So, it's possible
> > to
> > > > make
> > > > > > the request block until it's completed. However, currently, doing
> > > this
> > > > at
> > > > > > the broker is a bit involved. To make Create block, we will need
> to
> > > add
> > > > > > some callbacks in KafkaController. This is possible. However, the
> > > > > > controller logic currently is pretty completed. It would probably
> > be
> > > > > better
> > > > > > if we clean it up first before adding more complexity to it.
> Alter
> > is
> > > > > even
> > > > > > trickier. Adding partition is currently handled through
> > > > KafkaController.
> > > > > So
> > > > > > it can be dealt with in a similar way. However, Alter config is
> > done
> > > > > > completely differently. It doesn't go through the controller.
> > > Instead,
> > > > > each
> > > > > > broker listens to ZooKeeper directly. So, it's not clear if there
> > is
> > > an
> > > > > > easy way on the broker to figure out whether a config is applied
> on
> > > > every
> > > > > > broker.
> > > > > >
> > > > > > 2. Delete can potentially take long if a replica to be deleted is
> > > > > offline.
> > > > > > PreferredLeader/PartitionReassign can also take long. So, we
> can't
> > > > really
> > > > > > make those requests block on the broker.
> > > > > >
> > > > > > As you can see, at this moment it's not easy to make all admin
> > > requests
> > > > > > block on the broker. So, if we want the blocking feature in the
> > admin
> > > > > > utility in the short term, doing the completion check at the
> admin
> > > > client
> > > > > > is probably an easier route, even though it may not be ideal.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Jun,
> > > > > > >
> > > > > > > I see your point. But wouldn't that lead to a "fat" client
> > > > > > implementations?
> > > > > > > Suppose someone would like to implement client for Admin Wire
> > > > protocol.
> > > > > > > Not only people will have to code quite complicated logic like
> > > "send
> > > > > > > describe
> > > > > > > request to each broker" (again state machin?) but it will also
> > mean
> > > > > > people
> > > > > > > must understand internal kafka logic related to topic storage
> and
> > > how
> > > > > > > information is propageted from the controller to brokers.
> > > > > > > I see this like a dilemma between having a concise Wire
> Protocol
> > > and
> > > > > > > self-sufficient API to make client implementations simple.
> > > > > > > I don't have a win-win solution though.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > >
> > > > > > > > For 1), 2) and 3), blocking would probably mean that the new
> > > > metadata
> > > > > > is
> > > > > > > > propagated to every broker. To achieve that, the client can
> > keep
> > > > > > issuing
> > > > > > > > the describe topic request to every broker until it sees the
> > new
> > > > > > metadata
> > > > > > > > in the response.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >
> > > > > > > > > Hm, actually the ticket you linked, Guozhang, brings as
> back
> > > > > > > > > to the problem what should be considered a post-condition
> for
> > > > > > > > > each of the admin commands.
> > > > > > > > > In my understanding:
> > > > > > > > >
> > > > > > > > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > > > > > > > (Not the controller picked up changes from zk and
> broadcasted
> > > > > > > > > LeaderAndIsr and UpdateMetadata)
> > > > > > > > >
> > > > > > > > > 2) AlterTopic - same as 1) - broker changed assignment data
> > > > > > > > > in zookeeper or created admin path for topic config change
> > > > > > > > >
> > > > > > > > > 3) DeleteTopic - admin path /admin/delete_topics is created
> > > > > > > > >
> > > > > > > > > 4) ReassignPartitions and PreferredReplica - corresponding
> > > admin
> > > > > > > > > path is created
> > > > > > > > >
> > > > > > > > > Now what can be considered a completed operation from the
> > > > client's
> > > > > > > > > perspective?
> > > > > > > > > 1) Topic is created once corresponding data is in zk
> > > > > > > > > (I remember there were some thoughts that it'd be good to
> > > > consider
> > > > > > > > > topic created once all replicas receive information about
> it
> > > and
> > > > > thus
> > > > > > > > > clients can produce/consume from it, but as was discussed
> > this
> > > > > seems
> > > > > > > > > to be a hard thing to do)
> > > > > > > > >
> > > > > > > > > 2) Probably same as 1), so right after AlterTopic is issued
> > > > > > > > >
> > > > > > > > > 3) The topic has been removed from /brokers/topics
> > > > > > > > >
> > > > > > > > > 4) ReassignPartitions and PrefferedReplica were discussed
> > > > earlier -
> > > > > > > > > in short the former is completed once partition state info
> in
> > > zk
> > > > > > > matches
> > > > > > > > > reassignment request and admin path is empty, the latter -
> > once
> > > > > data
> > > > > > > > > in zk shows that head of assignned replicas of the
> partition
> > > and
> > > > > > leader
> > > > > > > > > is the same replica
> > > > > > > > >
> > > > > > > > > Thoughts?
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Andrii Biletskyi
> > > > > > > > >
> > > > > > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <
> > > > wangguoz@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > I think while loop is fine for supporting blocking, just
> > that
> > > > we
> > > > > > need
> > > > > > > > to
> > > > > > > > > > add back off to avoid bombarding brokers with
> DescribeTopic
> > > > > > requests.
> > > > > > > > > >
> > > > > > > > > > Also I have linked KAFKA-1125
> > > > > > > > > > <https://issues.apache.org/jira/browse/KAFKA-1125> to
> your
> > > > > > proposal,
> > > > > > > > and
> > > > > > > > > > when KAFKA-1694 is done this ticket can also be closed.
> > > > > > > > > >
> > > > > > > > > > Guozhang
> > > > > > > > > >
> > > > > > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > >
> > > > > > > > > > > Great.
> > > > > > > > > > > I want to elaborate this a bit more, to see we are on
> the
> > > > same
> > > > > > page
> > > > > > > > > > > concerning the client code.
> > > > > > > > > > >
> > > > > > > > > > > So with all topic commands being async a client
> > > (AdminClient
> > > > in
> > > > > > our
> > > > > > > > > > > case or any other other client people would like to
> > > > implement)
> > > > > to
> > > > > > > > > support
> > > > > > > > > > > a blocking operation (which seems to be a natural
> > use-case
> > > > e.g.
> > > > > > for
> > > > > > > > > topic
> > > > > > > > > > > creation): would have to do:
> > > > > > > > > > > 1. issue CreateTopicRequest
> > > > > > > > > > > 2. if successful, in a "while" loop send
> > > DescribeTopicRequest
> > > > > and
> > > > > > > > > > > break the loop once all topics are returned in response
> > (or
> > > > > upon
> > > > > > > > > > timeout).
> > > > > > > > > > > 3. if unsuccessful throw exception
> > > > > > > > > > > Would it be okay?
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <
> > jun@confluent.io
> > > >
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Andrii,
> > > > > > > > > > > >
> > > > > > > > > > > > I think you are right. It seems that only
> > > > ReassignPartitions
> > > > > > > needs
> > > > > > > > a
> > > > > > > > > > > > separate verification request.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > I like this idea too. Let's stick with that. I'll
> > > update
> > > > > KIP
> > > > > > > > > > > accordingly.
> > > > > > > > > > > > >
> > > > > > > > > > > > > I was also thinking we can avoid adding dedicated
> > > status
> > > > > > check
> > > > > > > > > > > > > requests for topic commands. - We have everything
> in
> > > > > > > > DescribeTopic
> > > > > > > > > > > > > for that! E.g.:
> > > > > > > > > > > > > User issued CreateTopic - to check the status
> client
> > > > sends
> > > > > > > > > > > DescribeTopic
> > > > > > > > > > > > > and checks whether is something returned for that
> > > topic.
> > > > > The
> > > > > > > same
> > > > > > > > > for
> > > > > > > > > > > > > alteration, deletion.
> > > > > > > > > > > > > Btw, PreferredReplica status can be also checked
> with
> > > > > > > > > > > > DescribeTopicRequest
> > > > > > > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > > > > > > For ReassignPartitions as discussed we'll need to
> > have
> > > a
> > > > > > > separate
> > > > > > > > > > > > Verify...
> > > > > > > > > > > > > request.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> > > > > > > > wangguoz@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > +1 on broker writing to ZK for async handling. I
> > was
> > > > > > thinking
> > > > > > > > > that
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > end state the admin requests would be eventually
> > sent
> > > > to
> > > > > > > > > controller
> > > > > > > > > > > > > either
> > > > > > > > > > > > > > through re-routing or clients discovering them,
> > > instead
> > > > > of
> > > > > > > > > letting
> > > > > > > > > > > > > > controller listen on ZK admin path. But thinking
> > > about
> > > > > it a
> > > > > > > > > second
> > > > > > > > > > > > time,
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > think it is actually simpler to let controller
> > manage
> > > > > > > > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > +1 as well. I think it helps to keep the
> > rerouting
> > > > > > approach
> > > > > > > > > > > > orthogonal
> > > > > > > > > > > > > > > to this KIP.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay
> > Kreps
> > > > > > wrote:
> > > > > > > > > > > > > > > > I'm +1 on Jun's suggestion as long as it can
> > work
> > > > for
> > > > > > all
> > > > > > > > the
> > > > > > > > > > > > > requests.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <
> > > > > > > jun@confluent.io
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > (a) Admin requests can be sent to and
> handled
> > > by
> > > > > any
> > > > > > > > > broker.
> > > > > > > > > > > > > > > > > (b) Admin requests are processed
> > > asynchronously,
> > > > at
> > > > > > > least
> > > > > > > > > for
> > > > > > > > > > > > now.
> > > > > > > > > > > > > > > That is,
> > > > > > > > > > > > > > > > > when the client gets a response, it just
> > means
> > > > that
> > > > > > the
> > > > > > > > > > request
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > initiated, but not necessarily completed.
> > Then,
> > > > > it's
> > > > > > up
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > client
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > issue another request to check the status
> for
> > > > > > > completion.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > To support (a), we were thinking of doing
> > > request
> > > > > > > > > forwarding
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > controller (utilizing KAFKA-1912). I am
> > making
> > > an
> > > > > > > > > alternative
> > > > > > > > > > > > > > proposal.
> > > > > > > > > > > > > > > > > Basically, the broker can just write to
> > > ZooKeeper
> > > > > to
> > > > > > > > inform
> > > > > > > > > > the
> > > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > about the request. For example, to handle
> > > > > > > > > > > partitionReassignment,
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > will just write the requested partitions to
> > > > > > > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > > > > > > (like what AdminUtils currently does) and
> > then
> > > > > send a
> > > > > > > > > > response
> > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > client. This shouldn't take long and the
> > > > > > implementation
> > > > > > > > > will
> > > > > > > > > > be
> > > > > > > > > > > > > > simpler
> > > > > > > > > > > > > > > > > than forwarding the requests to the
> > controller
> > > > > > through
> > > > > > > > RPC.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii
> > > > Biletskyi <
> > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Jun,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I might be wrong but didn't we agree we
> > will
> > > > let
> > > > > > any
> > > > > > > > > broker
> > > > > > > > > > > > from
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > cluster handle *long-running* admin
> > requests
> > > > (at
> > > > > > this
> > > > > > > > > time
> > > > > > > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > reassignPartitions), via zk admin path.
> > Thus
> > > > > > > > CreateTopics
> > > > > > > > > > etc
> > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun
> Rao <
> > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I think we agreed that those admin
> > requests
> > > > can
> > > > > > be
> > > > > > > > > issued
> > > > > > > > > > > to
> > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > broker.
> > > > > > > > > > > > > > > > > > > Because of that, there doesn't seem to
> > be a
> > > > > > strong
> > > > > > > > need
> > > > > > > > > > to
> > > > > > > > > > > > know
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > controller. So, perhaps we can proceed
> by
> > > not
> > > > > > > making
> > > > > > > > > any
> > > > > > > > > > > > change
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > format of TMR right now. When we start
> > > using
> > > > > > create
> > > > > > > > > topic
> > > > > > > > > > > > > request
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > producer, we will need a new version of
> > TMR
> > > > > that
> > > > > > > > > doesn't
> > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > As a first cut implementation, I think
> > the
> > > > > broker
> > > > > > > can
> > > > > > > > > > just
> > > > > > > > > > > > > write
> > > > > > > > > > > > > > > to ZK
> > > > > > > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > > > > > > requests, instead of forwarding them to
> > the
> > > > > > > > controller.
> > > > > > > > > > > This
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel
> > > Koshy
> > > > <
> > > > > > > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > For (1) yes we will circle back on
> that
> > > > > shortly
> > > > > > > > after
> > > > > > > > > > > > syncing
> > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > person. I think it is close to
> getting
> > > > > > committed
> > > > > > > > > > although
> > > > > > > > > > > > > > > development
> > > > > > > > > > > > > > > > > > > > for KAFKA-1927 can probably begin
> > without
> > > > it.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > There is one more item we covered at
> > the
> > > > > > hangout.
> > > > > > > > > i.e.,
> > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > want to add the coordinator to the
> > topic
> > > > > > metadata
> > > > > > > > > > > response
> > > > > > > > > > > > or
> > > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > There are two reasons I think we
> should
> > > try
> > > > > and
> > > > > > > > avoid
> > > > > > > > > > > > adding
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > > > > > > > > > - If we finally do request rerouting
> in
> > > > Kafka
> > > > > > > then
> > > > > > > > > the
> > > > > > > > > > > > field
> > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > >   little to no value. (It still helps
> > to
> > > > > have a
> > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > > >   ClusterMetadataRequest to query for
> > > > > > > cluster-wide
> > > > > > > > > > > > > information
> > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > >   'which broker is the controller?'
> as
> > > Joe
> > > > > > > > > mentioned.)
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I think it would be cleaner to have
> an
> > > > > explicit
> > > > > > > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > > > > > > that you can send to any broker in
> > order
> > > to
> > > > > > > obtain
> > > > > > > > > the
> > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > in the future possibly other
> > cluster-wide
> > > > > > > > > > information). I
> > > > > > > > > > > > > think
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > main argument against doing this and
> > > > instead
> > > > > > > adding
> > > > > > > > > it
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > metadata response was convenience -
> > i.e.,
> > > > you
> > > > > > > don't
> > > > > > > > > > have
> > > > > > > > > > > to
> > > > > > > > > > > > > > > discover
> > > > > > > > > > > > > > > > > > > > the controller in advance. However, I
> > > don't
> > > > > see
> > > > > > > > much
> > > > > > > > > > > actual
> > > > > > > > > > > > > > > > > > > > benefit/convenience in this and in
> fact
> > > > think
> > > > > > it
> > > > > > > > is a
> > > > > > > > > > > > > > non-issue.
> > > > > > > > > > > > > > > Let
> > > > > > > > > > > > > > > > > > > > me know if I'm overlooking something
> > > here.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > As an example, say we need to
> initiate
> > > > > > partition
> > > > > > > > > > > > reassignment
> > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > > > issuing the new
> > ReassignPartitionsRequest
> > > > to
> > > > > > the
> > > > > > > > > > > controller
> > > > > > > > > > > > > > > (assume
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > already have the desired manual
> > partition
> > > > > > > > > assignment).
> > > > > > > > > > > If
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > are to
> > > > > > > > > > > > > > > > > > > > augment topic metadata response then
> > the
> > > > flow
> > > > > > be
> > > > > > > > > > > something
> > > > > > > > > > > > > like
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > :
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > - Issue topic metadata request to any
> > > > broker
> > > > > > (and
> > > > > > > > > > > discover
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > > > > > > - Connect to controller if required
> > > (i.e.,
> > > > if
> > > > > > the
> > > > > > > > > > broker
> > > > > > > > > > > > > above
> > > > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > > > - Issue the partition reassignment
> > > request
> > > > to
> > > > > > the
> > > > > > > > > > > > controller.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > With an explicit cluster metadata
> > request
> > > > it
> > > > > > > would
> > > > > > > > > be:
> > > > > > > > > > > > > > > > > > > > - Issue cluster metadata request to
> any
> > > > > broker
> > > > > > > > > > > > > > > > > > > > - Connect to controller if required
> > > (i.e.,
> > > > if
> > > > > > the
> > > > > > > > > > broker
> > > > > > > > > > > > > above
> > > > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > > > - Issue the partition reassignment
> > > request
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > So it seems to add little practical
> > value
> > > > and
> > > > > > > > bloats
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > The other angle to this is the
> > following
> > > -
> > > > is
> > > > > > it
> > > > > > > a
> > > > > > > > > > matter
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > naming?
> > > > > > > > > > > > > > > > > > > > Should we just rename topic metadata
> > > > > > > > request/response
> > > > > > > > > > to
> > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > MetadataRequest/Response and add
> > cluster
> > > > > > metadata
> > > > > > > > to
> > > > > > > > > > it?
> > > > > > > > > > > By
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > same
> > > > > > > > > > > > > > > > > > > > token should we also allow querying
> for
> > > the
> > > > > > > > consumer
> > > > > > > > > > > > > > coordinator
> > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > in future transaction coordinator) as
> > > well?
> > > > > > This
> > > > > > > > > leads
> > > > > > > > > > > to a
> > > > > > > > > > > > > > > bloated
> > > > > > > > > > > > > > > > > > > > request which isn't very appealing
> and
> > > > > > altogether
> > > > > > > > > > > > confusing.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM
> > -0700,
> > > > Jun
> > > > > > Rao
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 1. I just realized that in order to
> > > start
> > > > > > > working
> > > > > > > > > on
> > > > > > > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > need to merge the changes to
> > > > > > > OffsetCommitRequest
> > > > > > > > > > (from
> > > > > > > > > > > > > 0.8.2)
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > > > > > > This is planned to be done as part
> of
> > > > > > > KAFKA-1634.
> > > > > > > > > So,
> > > > > > > > > > > we
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 2. Thinking about this a bit more,
> if
> > > the
> > > > > > > > semantic
> > > > > > > > > of
> > > > > > > > > > > > those
> > > > > > > > > > > > > > > "write"
> > > > > > > > > > > > > > > > > > > > > requests is async (i.e., after the
> > > client
> > > > > > gets
> > > > > > > a
> > > > > > > > > > > > response,
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > > > > > > that the operation is initiated,
> but
> > > not
> > > > > > > > > necessarily
> > > > > > > > > > > > > > > completed), we
> > > > > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > > > really need to forward the requests
> > to
> > > > the
> > > > > > > > > > controller.
> > > > > > > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > > > > > > receiving broker can just write the
> > > > > operation
> > > > > > > to
> > > > > > > > ZK
> > > > > > > > > > as
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > > > > > > line tool previously does. This
> will
> > > > > simplify
> > > > > > > the
> > > > > > > > > > > > > > > implementation.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 8. There is another implementation
> > > detail
> > > > > for
> > > > > > > > > > describe
> > > > > > > > > > > > > topic.
> > > > > > > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > want to read the topic config from
> > the
> > > > > broker
> > > > > > > > > cache,
> > > > > > > > > > > > > instead
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > > > > > > Currently, every broker reads the
> > > > > topic-level
> > > > > > > > > config
> > > > > > > > > > > for
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > > > However, it ignores those for
> topics
> > > not
> > > > > > hosted
> > > > > > > > on
> > > > > > > > > > > > itself.
> > > > > > > > > > > > > > So,
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > to change TopicConfigManager a bit
> so
> > > > that
> > > > > it
> > > > > > > > > caches
> > > > > > > > > > > the
> > > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM,
> > Andrii
> > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala
> requests
> > > > > > objects,
> > > > > > > > use
> > > > > > > > > > java
> > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > > > > > > >     A: Gwen kindly took that
> > > > > (KAFKA-1927).
> > > > > > > It's
> > > > > > > > > > > > important
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > speed
> > > > > > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > > > > > > >          there since this ticket
> > > blocks
> > > > > > other
> > > > > > > > > > > important
> > > > > > > > > > > > > > > changes.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute facility
> > vs
> > > > > client
> > > > > > > > > > > maintaining
> > > > > > > > > > > > > > > cluster
> > > > > > > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > > > > > > >     A: Jay has added pseudo code
> to
> > > > > > > KAFKA-1912
> > > > > > > > -
> > > > > > > > > > need
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > > > > > > >         easy to implement as a
> > > > > server-side
> > > > > > > > > feature
> > > > > > > > > > > > > > (comments
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 3. Q: Controller field in wire
> > > > protocol.
> > > > > > > > > > > > > > > > > > > > > >     A: This might be useful for
> > > > clients,
> > > > > > add
> > > > > > > > this
> > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic creation
> > from
> > > > TMR.
> > > > > > > > > > > > > > > > > > > > > >     A: I will add proposed by Jun
> > > > > solution
> > > > > > > > (using
> > > > > > > > > > > > > clientId
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR
> > vs
> > > > > > grabbing
> > > > > > > > all
> > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > > >     A: It was decided to try to
> > > gather
> > > > > all
> > > > > > > > > changes
> > > > > > > > > > to
> > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > > > > > > >         In case of TMR it worth
> > > > checking:
> > > > > > > > > > KAFKA-2020
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to
> > > deserialize
> > > > > > > user's
> > > > > > > > > > input
> > > > > > > > > > > in
> > > > > > > > > > > > > CLI
> > > > > > > > > > > > > > > tool.
> > > > > > > > > > > > > > > > > > > > > >     A: Use jackson for that,
> /tools
> > > > > project
> > > > > > > is
> > > > > > > > a
> > > > > > > > > > > > separate
> > > > > > > > > > > > > > > jar so
> > > > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions
> vs
> > > > > generic
> > > > > > > > status
> > > > > > > > > > > check
> > > > > > > > > > > > > > > command.
> > > > > > > > > > > > > > > > > > > > > >      A: For long-running requests
> > > like
> > > > > > > reassign
> > > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > > > > > > >          it makes sense to
> > introduce
> > > > it.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >  Please add, correct me if I
> missed
> > > > > > > something.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM,
> > > Andrii
> > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > wrote:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > You are right, I removed
> > > > > ClusterMetadata
> > > > > > > > > because
> > > > > > > > > > we
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > > > > > > what we need in TopicMetadata.
> > > Also,
> > > > as
> > > > > > Jay
> > > > > > > > > > pointed
> > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > earlier,
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > would like to have "orthogonal"
> > > API,
> > > > > but
> > > > > > at
> > > > > > > > the
> > > > > > > > > > > same
> > > > > > > > > > > > > time
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > But I like your idea and even
> > have
> > > > some
> > > > > > > other
> > > > > > > > > > > > arguments
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > > > > > > There is also
> > DescribeTopicRequest
> > > > > which
> > > > > > > was
> > > > > > > > > > > proposed
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > > > > > > it returns topic configs,
> > > partitions,
> > > > > > > > > replication
> > > > > > > > > > > > > factor
> > > > > > > > > > > > > > > plus
> > > > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > > > > > > leader replica. The later part
> is
> > > > > really
> > > > > > > > > already
> > > > > > > > > > > > there
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > > > > > > So again we'll have to add
> stuff
> > to
> > > > > TMR,
> > > > > > > not
> > > > > > > > to
> > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > > newly added requests. However,
> > this
> > > > way
> > > > > > > we'll
> > > > > > > > > end
> > > > > > > > > > > up
> > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > > > > > > request which returns cluster
> > > > metadata,
> > > > > > > topic
> > > > > > > > > > > > > replication
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > > > > plus partition replication
> data.
> > > > Seems
> > > > > > > > logical
> > > > > > > > > to
> > > > > > > > > > > > split
> > > > > > > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > > > > > > - ClusterMetadata (brokers +
> > > > > controller,
> > > > > > > > maybe
> > > > > > > > > > smth
> > > > > > > > > > > > > else)
> > > > > > > > > > > > > > > > > > > > > > > - TopicMetadata (topic info +
> > > > partition
> > > > > > > > > details)
> > > > > > > > > > > > > > > > > > > > > > > But since current TMR is
> involved
> > > in
> > > > > lots
> > > > > > > of
> > > > > > > > > > places
> > > > > > > > > > > > > > > (including
> > > > > > > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > > > > > as I understand) this might be
> > very
> > > > > > serious
> > > > > > > > > > change
> > > > > > > > > > > > and
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > > > > > > sense to stick with current
> > > approach.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29
> PM,
> > > Joel
> > > > > > > Koshy <
> > > > > > > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> I may be missing some context
> > but
> > > > > > > hopefully
> > > > > > > > > this
> > > > > > > > > > > > will
> > > > > > > > > > > > > > > also be
> > > > > > > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > > > > > > >> today: I thought the earlier
> > > > proposal
> > > > > > > where
> > > > > > > > > > there
> > > > > > > > > > > > was
> > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > > > > > > >> ClusterMetadata request was
> > > clearer
> > > > > and
> > > > > > > > > > explicit.
> > > > > > > > > > > > > During
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > >> this thread I think the
> > conclusion
> > > > was
> > > > > > > that
> > > > > > > > > the
> > > > > > > > > > > main
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > >> controller information and
> that
> > > can
> > > > be
> > > > > > > > rolled
> > > > > > > > > > into
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > >> response but that seems a bit
> > > > > irrelevant
> > > > > > > to
> > > > > > > > > > topic
> > > > > > > > > > > > > > > metadata.
> > > > > > > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > > > > > > >> think the full broker-list is
> > also
> > > > > > > > irrelevant
> > > > > > > > > to
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > >> it is already there and in
> use.
> > I
> > > > > think
> > > > > > > > there
> > > > > > > > > is
> > > > > > > > > > > > still
> > > > > > > > > > > > > > > room
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > > >> explicit ClusterMetadata
> request
> > > > since
> > > > > > > there
> > > > > > > > > may
> > > > > > > > > > > be
> > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > > > >> cluster-level information that
> > we
> > > > may
> > > > > > want
> > > > > > > > to
> > > > > > > > > > add
> > > > > > > > > > > > over
> > > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> have nothing to do with topic
> > > > > metadata).
> > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at
> > 02:45:30PM
> > > > > > +0200,
> > > > > > > > > Andrii
> > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > 101. Okay, if you say that
> > such
> > > > use
> > > > > > case
> > > > > > > > is
> > > > > > > > > > > > > > important. I
> > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > > > >> > using clientId for these
> > > purposes
> > > > is
> > > > > > > fine
> > > > > > > > -
> > > > > > > > > if
> > > > > > > > > > > we
> > > > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > > > > > > >> > as part of all Wire protocol
> > > > > messages,
> > > > > > > why
> > > > > > > > > not
> > > > > > > > > > > use
> > > > > > > > > > > > > > that.
> > > > > > > > > > > > > > > > > > > > > > >> > I will update KIP-4 page if
> > > nobody
> > > > > has
> > > > > > > > other
> > > > > > > > > > > ideas
> > > > > > > > > > > > > > > (which
> > > > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the
> > KIP
> > > > > > > > > accordingly.
> > > > > > > > > > I
> > > > > > > > > > > > > think
> > > > > > > > > > > > > > > we can
> > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > > > > > > >> > fine-grained error codes if
> > some
> > > > > error
> > > > > > > > code
> > > > > > > > > > > > received
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > > >> > won't give enough context to
> > > > return
> > > > > a
> > > > > > > > > > > descriptive
> > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > message
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > Look forward to discussing
> all
> > > > > > > outstanding
> > > > > > > > > > > issues
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > detail
> > > > > > > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at
> 10:59
> > > PM,
> > > > > Jun
> > > > > > > Rao
> > > > > > > > <
> > > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > 101. There may be a use
> case
> > > > where
> > > > > > you
> > > > > > > > > only
> > > > > > > > > > > want
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > > > > > > >> > > manually by admins.
> > Currently,
> > > > you
> > > > > > can
> > > > > > > > do
> > > > > > > > > > that
> > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > disabling
> > > > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> > > creation and issue topic
> > > > creation
> > > > > > from
> > > > > > > > the
> > > > > > > > > > > > > > > TopicCommand.
> > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > > > > > > >> > > topic creation completely
> on
> > > the
> > > > > > > broker
> > > > > > > > > and
> > > > > > > > > > > > don't
> > > > > > > > > > > > > > > have a
> > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > > > >> > > between topic creation
> > > requests
> > > > > from
> > > > > > > the
> > > > > > > > > > > regular
> > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > > > > > > >> > > can't support manual topic
> > > > > creation
> > > > > > > any
> > > > > > > > > > more.
> > > > > > > > > > > I
> > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > > > > > > >> > > way of distinguishing the
> > > > clients
> > > > > > > making
> > > > > > > > > the
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > creation
> > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > > > >> > > using clientId. For
> example,
> > > the
> > > > > > admin
> > > > > > > > > tool
> > > > > > > > > > > can
> > > > > > > > > > > > > set
> > > > > > > > > > > > > > > it to
> > > > > > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > > > > > > >> > > admin and the broker can
> > treat
> > > > > that
> > > > > > > > > clientId
> > > > > > > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > Also, there is a related
> > > > > discussion
> > > > > > in
> > > > > > > > > > > > KAFKA-2020.
> > > > > > > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > >> > > following in
> > > > > TopicMetadataResponse:
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > 1. If leader is not
> > available,
> > > > we
> > > > > > set
> > > > > > > > the
> > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > > > > > > >> > > 2. If a non-leader replica
> > is
> > > > not
> > > > > > > > > available,
> > > > > > > > > > > we
> > > > > > > > > > > > > take
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > > > >> > > the assigned replica list
> > and
> > > > isr
> > > > > in
> > > > > > > the
> > > > > > > > > > > > response.
> > > > > > > > > > > > > > As
> > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > > > > > > >> > > doing that, we set the
> > > partition
> > > > > > level
> > > > > > > > > error
> > > > > > > > > > > > code
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > This has a few problems.
> > > First,
> > > > > > > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > > > > > > >> > > an error, at least for the
> > > > normal
> > > > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > > > > > > >> > > to find out the leader.
> > > Second,
> > > > it
> > > > > > can
> > > > > > > > > > happen
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > both
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> > > another replica are not
> > > > available
> > > > > at
> > > > > > > the
> > > > > > > > > > same
> > > > > > > > > > > > > time.
> > > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > > > >> > > to indicate both. Third,
> > even
> > > > if a
> > > > > > > > replica
> > > > > > > > > > is
> > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > available,
> > > > > > > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > > > > > > >> > > useful to return its
> replica
> > > id
> > > > > > since
> > > > > > > > some
> > > > > > > > > > > > clients
> > > > > > > > > > > > > > > (e.g.
> > > > > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > One way to address this
> > issue
> > > is
> > > > > to
> > > > > > > > always
> > > > > > > > > > > > return
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > > > > > > >> > > leader, assigned replicas,
> > and
> > > > isr
> > > > > > > > > > regardless
> > > > > > > > > > > of
> > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > > > > > > >> > > broker is live or not.
> Since
> > > we
> > > > > also
> > > > > > > > > return
> > > > > > > > > > > the
> > > > > > > > > > > > > list
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > >> > > client can figure out
> > whether
> > > a
> > > > > > leader
> > > > > > > > or
> > > > > > > > > a
> > > > > > > > > > > > > replica
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > > > > > > >> > > accordingly. This way, we
> > > don't
> > > > > need
> > > > > > > to
> > > > > > > > > set
> > > > > > > > > > > the
> > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > > > >> > > when the leader or a
> replica
> > > is
> > > > > not
> > > > > > > > > > available.
> > > > > > > > > > > > > This
> > > > > > > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > > > > > > >> > > protocol, but does change
> > the
> > > > > > > semantics.
> > > > > > > > > > Since
> > > > > > > > > > > > we
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > > > > > > >> > > of TopicMetadataRequest
> > here,
> > > we
> > > > > can
> > > > > > > > > > > potentially
> > > > > > > > > > > > > > > piggyback
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > 102.1 For those types of
> > > errors
> > > > > due
> > > > > > to
> > > > > > > > > > invalid
> > > > > > > > > > > > > > input,
> > > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > >> > > guard it at parameter
> > > validation
> > > > > > time
> > > > > > > > and
> > > > > > > > > > > throw
> > > > > > > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > > > > > > >> > > without even sending the
> > > request
> > > > > to
> > > > > > > the
> > > > > > > > > > > broker?
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at
> > 10:37
> > > > AM,
> > > > > > > Andrii
> > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > >> > >
> andrii.biletskyi@stealth.ly
> > >
> > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > Answering your
> questions:
> > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > 101. If I understand you
> > > > > > correctly,
> > > > > > > > you
> > > > > > > > > > are
> > > > > > > > > > > > > saying
> > > > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > > > > > > >> > > > will be ported to
> TMR_V1)
> > > > won't
> > > > > be
> > > > > > > > able
> > > > > > > > > to
> > > > > > > > > > > > > > > automatically
> > > > > > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > > > >> > > > unconditionally remove
> > topic
> > > > > > > creation
> > > > > > > > > from
> > > > > > > > > > > > > there).
> > > > > > > > > > > > > > > But
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > > > > > > >> > > > Ok, about your proposal:
> > I'm
> > > > > not a
> > > > > > > big
> > > > > > > > > fan
> > > > > > > > > > > > too,
> > > > > > > > > > > > > > > when it
> > > > > > > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > > > > > > >> > > > clients directly in
> > protocol
> > > > > > schema.
> > > > > > > > And
> > > > > > > > > > > also
> > > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > > > > > > >> > > >
> auto.create.topics.enable
> > > is a
> > > > > > > server
> > > > > > > > > side
> > > > > > > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > > > > > > >> > > > in future versions, add
> > this
> > > > > > setting
> > > > > > > > to
> > > > > > > > > > > > producer
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > based
> > > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > > > > > > >> > > > UnknownTopic create
> topic
> > > > > > explicitly
> > > > > > > > by
> > > > > > > > > a
> > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's
> > because
> > > > we
> > > > > > want
> > > > > > > > to
> > > > > > > > > > > > support
> > > > > > > > > > > > > > > batching
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > > > > > > >> > > > want to give descriptive
> > > error
> > > > > > > > messages
> > > > > > > > > > for
> > > > > > > > > > > > > > clients.
> > > > > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > > > > > > >> > > > to construct such
> messages
> > > > (e.g.
> > > > > > > > > > AdminClient
> > > > > > > > > > > > > layer
> > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > > > > > > > >> > > > means two cases: either
> > > > invalid
> > > > > > > > number -
> > > > > > > > > > > e.g.
> > > > > > > > > > > > > -1;
> > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > > > > > > >> > > > partitions argument
> > wasn't)
> > > -
> > > > I
> > > > > > > > wrapped
> > > > > > > > > > > > > responses
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > > > > > > >> > > > other ideas, this was
> just
> > > > > initial
> > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree.
> I'll
> > > > change
> > > > > > > that
> > > > > > > > to
> > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at
> > 7:16
> > > > PM,
> > > > > > Jun
> > > > > > > > > Rao <
> > > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > 101. That's what I was
> > > > > thinking
> > > > > > > too,
> > > > > > > > > but
> > > > > > > > > > > it
> > > > > > > > > > > > > may
> > > > > > > > > > > > > > > not be
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > we can let it not
> > trigger
> > > > auto
> > > > > > > topic
> > > > > > > > > > > > creation.
> > > > > > > > > > > > > > > Then,
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > > > > > > UnknownTopicException,
> > > > > > > > > it
> > > > > > > > > > > can
> > > > > > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > createTopicRequest for
> > > auto
> > > > > > topic
> > > > > > > > > > > creation.
> > > > > > > > > > > > On
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > > > > > > >> > > > > never issue
> > > > > createTopicRequest.
> > > > > > > This
> > > > > > > > > > works
> > > > > > > > > > > > > when
> > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > > > > > > >> > > > > enabled on the broker
> > > side.
> > > > > > > > However, I
> > > > > > > > > > am
> > > > > > > > > > > > not
> > > > > > > > > > > > > > > sure how
> > > > > > > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > > > > > > >> > > > > when auto topic
> creation
> > > is
> > > > > > > disabled
> > > > > > > > > on
> > > > > > > > > > > the
> > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > side.
> > > > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > > > >> > > > > want to have a way to
> > > > manually
> > > > > > > > create
> > > > > > > > > a
> > > > > > > > > > > > topic,
> > > > > > > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > >> > > > > commands. However,
> then
> > we
> > > > > need
> > > > > > a
> > > > > > > > way
> > > > > > > > > to
> > > > > > > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > > > > > > >> > > > > issued from the
> producer
> > > > > clients
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > > > admin
> > > > > > > > > > > > > > > tools.
> > > > > > > > > > > > > > > > > > May
> > > > > > > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > new field in
> > > > > createTopicRequest
> > > > > > > and
> > > > > > > > > set
> > > > > > > > > > it
> > > > > > > > > > > > > > > differently
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > > > >> > > > > client and the admin
> > > client.
> > > > > > > > However,
> > > > > > > > > I
> > > > > > > > > > am
> > > > > > > > > > > > not
> > > > > > > > > > > > > > > sure if
> > > > > > > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring
> > > existing
> > > > > > > > requests
> > > > > > > > > > is a
> > > > > > > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > > > > > > >> > > > > posted some comments
> in
> > > > > > > KAFKA-1927.
> > > > > > > > We
> > > > > > > > > > > will
> > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > > > > > > >> > > > > first, before adding
> the
> > > new
> > > > > > logic
> > > > > > > > in
> > > > > > > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > 102. About the
> > > AdminClient:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit
> weird
> > > that
> > > > > we
> > > > > > > > return
> > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > we should either
> return
> > > > error
> > > > > > code
> > > > > > > > or
> > > > > > > > > > > throw
> > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > 102.2. We probably
> > > shouldn't
> > > > > > > > > explicitly
> > > > > > > > > > > use
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > Not every request
> > > evolution
> > > > > > > requires
> > > > > > > > > an
> > > > > > > > > > > api
> > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015
> at
> > > 4:08
> > > > > AM,
> > > > > > > > > Andrii
> > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > andrii.biletskyi@stealth.ly
> > > > >
> > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks for you
> > comments.
> > > > > > Answers
> > > > > > > > > > inline:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > 100. There are a few
> > > > fields
> > > > > > such
> > > > > > > > as
> > > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > > PartitionsSerialized
> > > > > > that
> > > > > > > > are
> > > > > > > > > > > > > > represented
> > > > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
> structures
> > > in
> > > > > > json.
> > > > > > > > > Could
> > > > > > > > > > we
> > > > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
> definition
> > as
> > > > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, now with Admin
> > > Client
> > > > > > this
> > > > > > > > > looks
> > > > > > > > > > a
> > > > > > > > > > > > bit
> > > > > > > > > > > > > > > weird.
> > > > > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > ReassignPartitionCommand
> > > > > > accepts
> > > > > > > > > input
> > > > > > > > > > > in
> > > > > > > > > > > > > > json,
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > interfaces
> unchanged,
> > > > where
> > > > > > > > > possible.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > If we port it to
> > > > > deserialized
> > > > > > > > > format,
> > > > > > > > > > in
> > > > > > > > > > > > CLI
> > > > > > > > > > > > > > > (/tools
> > > > > > > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > json library since
> > > /tools
> > > > is
> > > > > > > > written
> > > > > > > > > > in
> > > > > > > > > > > > java
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > provided by a user.
> > Can
> > > we
> > > > > > > quickly
> > > > > > > > > > agree
> > > > > > > > > > > > on
> > > > > > > > > > > > > > what
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON,
> > > whatever)?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> > > > > TopicMetadataRequest
> > > > > > > v1
> > > > > > > > > > still
> > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit
> weird
> > > now
> > > > > that
> > > > > > > we
> > > > > > > > > > have a
> > > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought about
> > how
> > > > the
> > > > > > new
> > > > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in
> > the
> > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For
> example,
> > > > > ideally,
> > > > > > > we
> > > > > > > > > > don't
> > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to
> trigger
> > > auto
> > > > > > topic
> > > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > I agree, this
> strange
> > > > logic
> > > > > > > should
> > > > > > > > > be
> > > > > > > > > > > > fixed.
> > > > > > > > > > > > > > > I'm not
> > > > > > > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > correct me if I'm
> > wrong,
> > > > but
> > > > > > it
> > > > > > > > > > doesn't
> > > > > > > > > > > > look
> > > > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > leverage AdminClient
> > for
> > > > > that
> > > > > > in
> > > > > > > > > > > Producer
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > > > > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant
> > > > getting
> > > > > > rid
> > > > > > > > of
> > > > > > > > > > > scala
> > > > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > > > HeartbeatRequestAndHeader
> > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap
> thing
> > > when
> > > > > > > adding
> > > > > > > > > the
> > > > > > > > > > > new
> > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the long
> > term
> > > > > plan
> > > > > > is
> > > > > > > > to
> > > > > > > > > > get
> > > > > > > > > > > > rid
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> > request/response
> > > in
> > > > > the
> > > > > > > > > client.
> > > > > > > > > > > > Since
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant number
> > of
> > > > new
> > > > > > > > > requests,
> > > > > > > > > > > > > perhaps
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the
> > existing
> > > > > scala
> > > > > > > > > requests
> > > > > > > > > > > > first
> > > > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks like I
> > > > > > misunderstood
> > > > > > > > the
> > > > > > > > > > > point
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > rework that. The
> only
> > > > thing
> > > > > is
> > > > > > > > that
> > > > > > > > > I
> > > > > > > > > > > > don't
> > > > > > > > > > > > > > see
> > > > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > least one existing
> > > > protocol
> > > > > > > > message.
> > > > > > > > > > > Thus,
> > > > > > > > > > > > > as
> > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > Re porting all
> > existing
> > > > > RQ/RP
> > > > > > in
> > > > > > > > > this
> > > > > > > > > > > > patch.
> > > > > > > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > requirement to have
> > > Admin
> > > > > KIP
> > > > > > > > done,
> > > > > > > > > > I'm
> > > > > > > > > > > > > afraid
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > There are 13
> protocol
> > > > > messages
> > > > > > > and
> > > > > > > > > all
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > intensive manual
> > > testing,
> > > > > no?
> > > > > > > I'm
> > > > > > > > > > afraid
> > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > not the
> > > > > > > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka core internals
> > :).
> > > > Let
> > > > > > me
> > > > > > > > know
> > > > > > > > > > > your
> > > > > > > > > > > > > > > thoughts
> > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > follow-up this
> issue (
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015
> > at
> > > > 6:40
> > > > > > AM,
> > > > > > > > Jun
> > > > > > > > > > > Rao <
> > > > > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > A few more
> comments.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 100. There are a
> few
> > > > > fields
> > > > > > > such
> > > > > > > > > as
> > > > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > > PartitionsSerialized
> > > > > > that
> > > > > > > > are
> > > > > > > > > > > > > > represented
> > > > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite
> structures
> > > in
> > > > > > json.
> > > > > > > > > Could
> > > > > > > > > > we
> > > > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol
> definition
> > as
> > > > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> > > > > > TopicMetadataRequest
> > > > > > > > v1
> > > > > > > > > > > still
> > > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit
> weird
> > > now
> > > > > that
> > > > > > > we
> > > > > > > > > > have a
> > > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought about
> > how
> > > > the
> > > > > > new
> > > > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in
> > the
> > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For
> example,
> > > > > ideally,
> > > > > > > we
> > > > > > > > > > don't
> > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to
> trigger
> > > auto
> > > > > > topic
> > > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay
> meant
> > > > > getting
> > > > > > > rid
> > > > > > > > > of
> > > > > > > > > > > > scala
> > > > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > > > HeartbeatRequestAndHeader
> > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap
> thing
> > > when
> > > > > > > adding
> > > > > > > > > the
> > > > > > > > > > > new
> > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the long
> > term
> > > > > plan
> > > > > > is
> > > > > > > > to
> > > > > > > > > > get
> > > > > > > > > > > > rid
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> > request/response
> > > in
> > > > > the
> > > > > > > > > client.
> > > > > > > > > > > > Since
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant number
> > of
> > > > new
> > > > > > > > > requests,
> > > > > > > > > > > > > perhaps
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the
> > existing
> > > > > scala
> > > > > > > > > requests
> > > > > > > > > > > > first
> > > > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12,
> 2015
> > > at
> > > > > 3:37
> > > > > > > PM,
> > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > andrii.biletskyi@stealth.ly
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > As said above -
> I
> > > list
> > > > > > again
> > > > > > > > all
> > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can see what's
> > left
> > > > and
> > > > > > > > finalize
> > > > > > > > > > all
> > > > > > > > > > > > > > pending
> > > > > > > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from
> Jay:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is much
> > > needed
> > > > > > > > > > > functionality,
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > really think
> these
> > > > > > protocols
> > > > > > > > > > > through.
> > > > > > > > > > > > We
> > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > of well
> > thought-out,
> > > > > > > > orthoganol
> > > > > > > > > > > apis.
> > > > > > > > > > > > > For
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > important to
> think
> > > > > through
> > > > > > > the
> > > > > > > > > end
> > > > > > > > > > > > state
> > > > > > > > > > > > > > > even if
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > won't implement
> in
> > > the
> > > > > > first
> > > > > > > > > > phase.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Definitely
> > behind
> > > > > this.
> > > > > > > > Would
> > > > > > > > > > > > > > appreciate
> > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > how this can be
> > > > > improved.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's please
> > > please
> > > > > > > please
> > > > > > > > > wait
> > > > > > > > > > > > until
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to the new java
> > > > protocol
> > > > > > > > > > > definitions.
> > > > > > > > > > > > If
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > objects that is
> > just
> > > > > > > > generating
> > > > > > > > > > more
> > > > > > > > > > > > > work
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> > > latest
> > > > > > > patch -
> > > > > > > > > > > removed
> > > > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This proposal
> > > > > > introduces
> > > > > > > a
> > > > > > > > > new
> > > > > > > > > > > type
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > inconsistent
> with
> > > > > > everything
> > > > > > > > > else
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > other marker
> > value.
> > > > You
> > > > > > > could
> > > > > > > > > > argue
> > > > > > > > > > > > > either
> > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > for consistency.
> > For
> > > > > > clients
> > > > > > > > > that
> > > > > > > > > > > > > > > implemented
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than our scala
> > code
> > > > > these
> > > > > > > > basic
> > > > > > > > > > > > > primitives
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> > > latest
> > > > > > > patch -
> > > > > > > > > > > removed
> > > > > > > > > > > > > > > MaybeOf
> > > > > > > > > > > > > > > > > > type
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 4.
> > ClusterMetadata:
> > > > This
> > > > > > > seems
> > > > > > > > > to
> > > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers, topics,
> > and
> > > > > > > > > partitions. I
> > > > > > > > > > > > think
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > ClusterMetadataRequest
> > > > > (or
> > > > > > > > just
> > > > > > > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > controller. Or
> are
> > > > there
> > > > > > > other
> > > > > > > > > > > things
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I agree.
> > Updated
> > > > the
> > > > > > KIP.
> > > > > > > > > Let's
> > > > > > > > > > > > > extends
> > > > > > > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > include
> > controller.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a
> > > tendency
> > > > to
> > > > > > try
> > > > > > > > to
> > > > > > > > > > > make a
> > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > particular
> nodes.
> > > This
> > > > > > adds
> > > > > > > a
> > > > > > > > > lot
> > > > > > > > > > of
> > > > > > > > > > > > > > burden
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > sounds easy but
> > each
> > > > > > > discovery
> > > > > > > > > can
> > > > > > > > > > > > fail
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > many
> > > > > > > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > full state
> machine
> > > to
> > > > do
> > > > > > > > > right). I
> > > > > > > > > > > > think
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > commands and
> > ideally
> > > > as
> > > > > > many
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > > > other
> > > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers and just
> > > > > redirect
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > there would be a
> > > > general
> > > > > > way
> > > > > > > > to
> > > > > > > > > > > > > > encapsulate
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a very
> > > > > interesting
> > > > > > > > idea,
> > > > > > > > > > but
> > > > > > > > > > > > > seems
> > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > feature (like
> > > > > performance
> > > > > > > > > > > > > considerations,
> > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I believe this
> > > > shouldn't
> > > > > > be
> > > > > > > a
> > > > > > > > > > > blocker.
> > > > > > > > > > > > > If
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point it won't
> > > affect
> > > > > > Admin
> > > > > > > > > > changes
> > > > > > > > > > > -
> > > > > > > > > > > > at
> > > > > > > > > > > > > > > least
> > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
> required.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We should
> > > probably
> > > > > > > > normalize
> > > > > > > > > > the
> > > > > > > > > > > > key
> > > > > > > > > > > > > > > value
> > > > > > > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than embedding a
> > new
> > > > > > > > formatting.
> > > > > > > > > > So
> > > > > > > > > > > > two
> > > > > > > > > > > > > > > strings
> > > > > > > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > internal equals
> > > sign.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> > > latest
> > > > > > > patch -
> > > > > > > > > > > > > normalized
> > > > > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the
> > > > postcondition
> > > > > of
> > > > > > > > these
> > > > > > > > > > > APIs
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the command has
> > been
> > > > > > > > completed?
> > > > > > > > > It
> > > > > > > > > > > is
> > > > > > > > > > > > a
> > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > been completed
> so
> > > you
> > > > > know
> > > > > > > > that
> > > > > > > > > if
> > > > > > > > > > > you
> > > > > > > > > > > > > > > create a
> > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it you won't get
> > an
> > > > > > > exception
> > > > > > > > > > about
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > being
> > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: For long
> > running
> > > > > > requests
> > > > > > > > > (like
> > > > > > > > > > > > > > reassign
> > > > > > > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command has
> begun
> > -
> > > so
> > > > > we
> > > > > > > > don't
> > > > > > > > > > > block
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > client.
> > > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic commands,
> > this
> > > > > will
> > > > > > be
> > > > > > > > > > > > refactored
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > immediately,
> since
> > > the
> > > > > > > > > Controller
> > > > > > > > > > > will
> > > > > > > > > > > > > > serve
> > > > > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up
> ticket
> > > > > > > KAFKA-1777).
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe
> topic
> > > and
> > > > > list
> > > > > > > > > topics
> > > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request. Is
> there
> > a
> > > > > reason
> > > > > > > to
> > > > > > > > > give
> > > > > > > > > > > > back
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like if we just
> > make
> > > > the
> > > > > > > > > > > > post-condition
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic is deleted
> > > that
> > > > > will
> > > > > > > get
> > > > > > > > > rid
> > > > > > > > > > > of
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be much more
> > > > intuitive.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> > > latest
> > > > > > > patch -
> > > > > > > > > > > removed
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> ListTopicsRequest.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we
> > > consider
> > > > > > > batching
> > > > > > > > > > these
> > > > > > > > > > > > > > > requests?
> > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > allow multiple
> > > > > operations
> > > > > > to
> > > > > > > > be
> > > > > > > > > > > > batched.
> > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we will get a
> lot
> > of
> > > > > code
> > > > > > > that
> > > > > > > > > > does
> > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > > > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this code will
> > work
> > > > > great
> > > > > > > when
> > > > > > > > > you
> > > > > > > > > > > > test
> > > > > > > > > > > > > > on 5
> > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the
> > KIP -
> > > > > > please
> > > > > > > > > check
> > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I think we
> > > should
> > > > > also
> > > > > > > > > discuss
> > > > > > > > > > > how
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > client api for
> > these
> > > > > > > > operations.
> > > > > > > > > > > > > Currently
> > > > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > is totally
> > sketchy.
> > > I
> > > > > > think
> > > > > > > we
> > > > > > > > > > > > probably
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > that exposes
> > > > > > administrative
> > > > > > > > > > > > > functionality.
> > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > properly test
> the
> > > new
> > > > > > apis,
> > > > > > > I
> > > > > > > > > > > suspect.
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the
> > KIP -
> > > > > > please
> > > > > > > > > check
> > > > > > > > > > > > "Admin
> > > > > > > > > > > > > > > Client"
> > > > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The other
> > > > > information
> > > > > > > that
> > > > > > > > > > would
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > information
> about
> > > > > > > > > partitions--how
> > > > > > > > > > > much
> > > > > > > > > > > > > > data
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the segment
> > offsets,
> > > > > what
> > > > > > is
> > > > > > > > the
> > > > > > > > > > > > log-end
> > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the compaction
> > > point,
> > > > > > etc. I
> > > > > > > > > think
> > > > > > > > > > > > that
> > > > > > > > > > > > > > done
> > > > > > > > > > > > > > > > > right
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > successor to the
> > > very
> > > > > > > awkward
> > > > > > > > > > > > > > OffsetRequest
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be resolved in a
> > > > > separate
> > > > > > > KIP
> > > > > > > > /
> > > > > > > > > > jira
> > > > > > > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 12. Generally we
> > can
> > > > do
> > > > > > good
> > > > > > > > > error
> > > > > > > > > > > > > > handling
> > > > > > > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > messages. I.e.
> > > > generally
> > > > > > the
> > > > > > > > > > client
> > > > > > > > > > > > has
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > context
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > an error that
> the
> > > > topic
> > > > > > > > doesn't
> > > > > > > > > > > exist
> > > > > > > > > > > > to
> > > > > > > > > > > > > > say
> > > > > > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than "error code
> > 14"
> > > > (or
> > > > > > > > > > whatever).
> > > > > > > > > > > > > Maybe
> > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this is hard? If
> > we
> > > > want
> > > > > > to
> > > > > > > > add
> > > > > > > > > > > > > > server-side
> > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > need to do this
> > in a
> > > > > > > > consistent
> > > > > > > > > > way
> > > > > > > > > > > > > across
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the
> > KIP -
> > > > > > please
> > > > > > > > > check
> > > > > > > > > > > > > > "Protocol
> > > > > > > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > comprehensive,
> > > > > > fine-grained
> > > > > > > > list
> > > > > > > > > > of
> > > > > > > > > > > > > error
> > > > > > > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from
> > > > Guozhang:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe
> topic
> > > > > > request:
> > > > > > > it
> > > > > > > > > > would
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > great to
> > > > > > > > > > > > > > > > > > go
> > > > > > > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex
> > for
> > > > > this
> > > > > > > > > request.
> > > > > > > > > > > For
> > > > > > > > > > > > > > > example,
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the topic
> command
> > is
> > > > to
> > > > > > list
> > > > > > > > all
> > > > > > > > > > > > topics
> > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex
> > > then
> > > > we
> > > > > > > have
> > > > > > > > to
> > > > > > > > > > > first
> > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > description info
> > and
> > > > > then
> > > > > > > > filter
> > > > > > > > > > at
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs
> in
> > > > > create
> > > > > > > > topic:
> > > > > > > > > > > this
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > related
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > maybe we can add
> > > > another
> > > > > > > > > metadata
> > > > > > > > > > > K-V
> > > > > > > > > > > > or
> > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > with config K-V
> in
> > > > > create
> > > > > > > > topic
> > > > > > > > > > like
> > > > > > > > > > > > we
> > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > field can be
> quite
> > > > > useful
> > > > > > in
> > > > > > > > > > storing
> > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > who issue the
> > create
> > > > > > > command,
> > > > > > > > > etc,
> > > > > > > > > > > > which
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant
> > > setting.
> > > > > Then
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > on regex of the
> > > > metadata
> > > > > > > > field.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: As discussed
> it
> > > is
> > > > > very
> > > > > > > > > > > interesting
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we have some
> basic
> > > > > > > > functionality
> > > > > > > > > > > > there.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today all
> the
> > > > admin
> > > > > > > > > operations
> > > > > > > > > > > are
> > > > > > > > > > > > > > > async in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > return once it
> is
> > > > > written
> > > > > > in
> > > > > > > > ZK,
> > > > > > > > > > and
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > is why
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > > > > > > testUtil.waitForTopicCreated()
> > > > > > > > > > > /
> > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request, etc.
> With
> > > > admin
> > > > > > > > > requests
> > > > > > > > > > we
> > > > > > > > > > > > > could
> > > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > synchronous
> > > requests;
> > > > > when
> > > > > > > it
> > > > > > > > is
> > > > > > > > > > > > turned
> > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > until the
> request
> > > has
> > > > > been
> > > > > > > > > > > completed.
> > > > > > > > > > > > > And
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > "token" field in
> > the
> > > > > > > response,
> > > > > > > > > and
> > > > > > > > > > > > then
> > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > verification
> > > request"
> > > > > with
> > > > > > > the
> > > > > > > > > > given
> > > > > > > > > > > > > token
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > has been
> > completed.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I see your
> > point.
> > > > My
> > > > > > idea
> > > > > > > > was
> > > > > > > > > > to
> > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long running
> > > request,
> > > > > > where
> > > > > > > > > > needed.
> > > > > > > > > > > We
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > do it
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > concern is that
> > > > > > introducing
> > > > > > > a
> > > > > > > > > > token
> > > > > > > > > > > we
> > > > > > > > > > > > > > again
> > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to do similar
> > thing
> > > > > > > > introducing
> > > > > > > > > > > single
> > > > > > > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this idea
> because
> > we
> > > > > > wanted
> > > > > > > to
> > > > > > > > > > have
> > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > a) have fixed
> > schema
> > > > but
> > > > > > > > > introduce
> > > > > > > > > > > > each
> > > > > > > > > > > > > > > time new
> > > > > > > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long-running
> > > requests
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > b) use one
> request
> > > for
> > > > > > > > > > verification
> > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > generalize
> > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine with
> > > whatever
> > > > > > > > decision
> > > > > > > > > > > > > community
> > > > > > > > > > > > > > > come
> > > > > > > > > > > > > > > > > to.
> > > > > > > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comment from
> Gwen:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 16. Specifically
> > for
> > > > > > > > ownership,
> > > > > > > > > I
> > > > > > > > > > > > think
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like you are
> > > > describing
> > > > > > ACL)
> > > > > > > > via
> > > > > > > > > > an
> > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I remember
> KIP-11
> > > > > > described
> > > > > > > > > this,
> > > > > > > > > > > but
> > > > > > > > > > > > I
> > > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no
> > problem.
> > > > Not
> > > > > > > sure
> > > > > > > > > > though
> > > > > > > > > > > > how
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be
> committed
> > > > first
> > > > > > and
> > > > > > > > > > include
> > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I added
> > this
> > > > > note
> > > > > > to
> > > > > > > > > "Open
> > > > > > > > > > > > > > > Questions"
> > > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13,
> > 2015
> > > > at
> > > > > > > 12:34
> > > > > > > > > AM,
> > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > andrii.biletskyi@stealth.ly
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Today I
> uploaded
> > > the
> > > > > > patch
> > > > > > > > > that
> > > > > > > > > > > > covers
> > > > > > > > > > > > > > > some of
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - removed
> > MaybeOf
> > > > > > optional
> > > > > > > > > type
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - switched to
> > java
> > > > > > > protocol
> > > > > > > > > > > > > definitions
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - simplified
> > > > messages
> > > > > > > > > > (normalized
> > > > > > > > > > > > > > configs,
> > > > > > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I also updated
> > the
> > > > > KIP-4
> > > > > > > > with
> > > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin
> > > > > Operations
> > > > > > > ->
> > > > > > > > > > > updated
> > > > > > > > > > > > > Wire
> > > > > > > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove
> > > > > ClusterMetadata
> > > > > > > ->
> > > > > > > > > > > changed
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > extend
> > > > > > > > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin Client
> > ->
> > > > > > updated
> > > > > > > my
> > > > > > > > > > > initial
> > > > > > > > > > > > > > > proposal
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Error codes
> ->
> > > > > > proposed
> > > > > > > > > > > > fine-grained
> > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > AdminRequestFailed
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I will also
> > send a
> > > > > > > separate
> > > > > > > > > > email
> > > > > > > > > > > to
> > > > > > > > > > > > > > > cover all
> > > > > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii
> Biletskyi
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar
> 12,
> > > 2015
> > > > > at
> > > > > > > 9:26
> > > > > > > > > PM,
> > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11
> (
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> It actually
> > > > specifies
> > > > > > > > changes
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are
> > > > > > consistent
> > > > > > > in
> > > > > > > > > > this
> > > > > > > > > > > > > regard
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar
> 12,
> > > > 2015
> > > > > at
> > > > > > > > 12:21
> > > > > > > > > > PM,
> > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> Specifically
> > > for
> > > > > > > > > ownership, I
> > > > > > > > > > > > think
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > like you
> are
> > > > > > describing
> > > > > > > > > ACL)
> > > > > > > > > > > via
> > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I remember
> > > KIP-11
> > > > > > > > described
> > > > > > > > > > > this,
> > > > > > > > > > > > > > but I
> > > > > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> Regardless, I
> > > > think
> > > > > > > KIP-4
> > > > > > > > > > > focuses
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > exists from
> > > Kafka
> > > > > > > > brokers,
> > > > > > > > > > not
> > > > > > > > > > > on
> > > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > should
> exist
> > > but
> > > > > > > doesn't
> > > > > > > > > yet?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar
> > 12,
> > > > > 2015
> > > > > > at
> > > > > > > > > 6:37
> > > > > > > > > > > AM,
> > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just want
> to
> > > > > > > elaborate a
> > > > > > > > > bit
> > > > > > > > > > > > more
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > describe-topic
> > > > > based
> > > > > > > on
> > > > > > > > > > > config /
> > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on
> > KAFKA-1694.
> > > > The
> > > > > > > main
> > > > > > > > > > > > motivation
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> mechanisms,
> > > > which
> > > > > I
> > > > > > > > think
> > > > > > > > > is
> > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > architecture:
> > > > > today
> > > > > > > > anyone
> > > > > > > > > > can
> > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> there is
> no
> > > > > concept
> > > > > > or
> > > > > > > > > > > > "ownership"
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> users. For
> > > > > example,
> > > > > > at
> > > > > > > > > > > LinkedIn
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some
> casual
> > > > topic
> > > > > > name
> > > > > > > > > > prefix,
> > > > > > > > > > > > > which
> > > > > > > > > > > > > > > is a
> > > > > > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale
> our
> > > > > > > customers.
> > > > > > > > It
> > > > > > > > > > > would
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe
> all
> > > > > topics
> > > > > > > that
> > > > > > > > > is
> > > > > > > > > > > > > created
> > > > > > > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe
> all
> > > > > topics
> > > > > > > > whose
> > > > > > > > > > > > > retention
> > > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe
> all
> > > > > topics
> > > > > > > > whose
> > > > > > > > > > > > writable
> > > > > > > > > > > > > > > group
> > > > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > authorization),
> > > > > > etc..
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One
> possible
> > > way
> > > > > to
> > > > > > > > > achieve
> > > > > > > > > > > this
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> create-topic
> > > > > > request,
> > > > > > > > > whose
> > > > > > > > > > > > value
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic;
> then
> > > > > > > > > describe-topics
> > > > > > > > > > > can
> > > > > > > > > > > > > > > choose to
> > > > > > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2)
> > > config
> > > > > K-V
> > > > > > > > > > matching,
> > > > > > > > > > > > 3)
> > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu,
> Mar
> > 5,
> > > > > 2015
> > > > > > at
> > > > > > > > > 4:37
> > > > > > > > > > > PM,
> > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks
> for
> > > the
> > > > > > > updated
> > > > > > > > > > wiki.
> > > > > > > > > > > A
> > > > > > > > > > > > > few
> > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error
> > > > > > description
> > > > > > > in
> > > > > > > > > > > > > response: I
> > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> several
> > > > different
> > > > > > > error
> > > > > > > > > > cases
> > > > > > > > > > > > > then
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In
> > > > general
> > > > > > the
> > > > > > > > > > > errorCode
> > > > > > > > > > > > > > > itself
> > > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> sufficient
> for
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> describing
> > > the
> > > > > > server
> > > > > > > > > side
> > > > > > > > > > > > > errors.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2.
> Describe
> > > > topic
> > > > > > > > > request:
> > > > > > > > > > it
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic
> name
> > > > regex
> > > > > > for
> > > > > > > > this
> > > > > > > > > > > > > request.
> > > > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic
> > > > command
> > > > > > is
> > > > > > > to
> > > > > > > > > > list
> > > > > > > > > > > > all
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic
> name
> > > > regex
> > > > > > then
> > > > > > > > we
> > > > > > > > > > have
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> description
> > > > info
> > > > > > and
> > > > > > > > then
> > > > > > > > > > > > filter
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on
> > ZK.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config
> > > K-Vs
> > > > in
> > > > > > > > create
> > > > > > > > > > > topic:
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we
> > can
> > > > add
> > > > > > > > another
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > K-V or
> > > > > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with
> config
> > > K-V
> > > > > in
> > > > > > > > create
> > > > > > > > > > > topic
> > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field can
> > be
> > > > > quite
> > > > > > > > useful
> > > > > > > > > > in
> > > > > > > > > > > > > > storing
> > > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue
> > the
> > > > > > create
> > > > > > > > > > command,
> > > > > > > > > > > > > etc,
> > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > multi-tenant
> > > > > > setting.
> > > > > > > > > Then
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex
> of
> > > the
> > > > > > > > metadata
> > > > > > > > > > > field.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today
> > all
> > > > the
> > > > > > > admin
> > > > > > > > > > > > operations
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> return
> once
> > > it
> > > > is
> > > > > > > > written
> > > > > > > > > > in
> > > > > > > > > > > > ZK,
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> request,
> > etc.
> > > > > With
> > > > > > > > admin
> > > > > > > > > > > > requests
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> synchronous
> > > > > > requests;
> > > > > > > > > when
> > > > > > > > > > it
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > turned
> > > > > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until the
> > > > request
> > > > > > has
> > > > > > > > > been
> > > > > > > > > > > > > > > completed. And
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token"
> > field
> > > > in
> > > > > > the
> > > > > > > > > > > response,
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > verification
> > > > > > request"
> > > > > > > > > with
> > > > > > > > > > > the
> > > > > > > > > > > > > > given
> > > > > > > > > > > > > > > > > token
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has been
> > > > > completed.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for
> > > > > extending
> > > > > > > > > > Metadata
> > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > information,
> > > > and
> > > > > > then
> > > > > > > > we
> > > > > > > > > > can
> > > > > > > > > > > > > remove
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> ClusterMetadata
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue,
> Mar
> > > 3,
> > > > > 2015
> > > > > > > at
> > > > > > > > > > 10:23
> > > > > > > > > > > > AM,
> > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com
> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks
> for
> > > > > sending
> > > > > > > > that
> > > > > > > > > > out
> > > > > > > > > > > > Joe
> > > > > > > > > > > > > -
> > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it
> today,
> > so
> > > > if
> > > > > > > notes
> > > > > > > > > can
> > > > > > > > > > be
> > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon,
> > Mar
> > > > 02,
> > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > > > 09:16:13AM
> > > > > > > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks
> > for
> > > > > > sending
> > > > > > > > > this
> > > > > > > > > > > out
> > > > > > > > > > > > > Joe.
> > > > > > > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone
> > :)
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On
> Mon,
> > > Mar
> > > > 2,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > 6:46
> > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> Hey, I
> > > > just
> > > > > > sent
> > > > > > > > > out a
> > > > > > > > > > > > > google
> > > > > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > everyone I
> > > > > > found
> > > > > > > > > > working
> > > > > > > > > > > > on
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let
> me
> > > > know
> > > > > > and
> > > > > > > > can
> > > > > > > > > > > update
> > > > > > > > > > > > > it,
> > > > > > > > > > > > > > > np.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We
> > > should
> > > > do
> > > > > > > this
> > > > > > > > > > every
> > > > > > > > > > > > > > Tuesday
> > > > > > > > > > > > > > > @
> > > > > > > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help
> > to
> > > > > make a
> > > > > > > > > google
> > > > > > > > > > > > > account
> > > > > > > > > > > > > > > so we
> > > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To
> > > discuss
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in
> > > > progress
> > > > > > and
> > > > > > > > > > related
> > > > > > > > > > > > JIRA
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~
> Joe
> > > > Stein
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On
> > Tue,
> > > > Feb
> > > > > > 24,
> > > > > > > > 2015
> > > > > > > > > > at
> > > > > > > > > > > > 2:59
> > > > > > > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > jay.kreps@gmail.com
> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> Let's
> > > > stay
> > > > > on
> > > > > > > > > Google
> > > > > > > > > > > > > hangouts
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > available
> > > > > on
> > > > > > > > > youtube.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> -Jay
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On
> > Tue,
> > > > Feb
> > > > > > 24,
> > > > > > > > > 2015
> > > > > > > > > > at
> > > > > > > > > > > > > 11:49
> > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > jholoman@cloudera.com
> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > Jay /
> > > > Joe
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > We're
> > > > > happy
> > > > > > > to
> > > > > > > > > send
> > > > > > > > > > > > out a
> > > > > > > > > > > > > > > Webex
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > sessions
> > > > > if
> > > > > > > > there
> > > > > > > > > > is
> > > > > > > > > > > > > > > interest and
> > > > > > > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > Thanks
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > Jeff
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> On
> > > Tue,
> > > > > Feb
> > > > > > > 24,
> > > > > > > > > > 2015
> > > > > > > > > > > at
> > > > > > > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > Let's
> > > > > try
> > > > > > > to
> > > > > > > > > get
> > > > > > > > > > > the
> > > > > > > > > > > > > > > technical
> > > > > > > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > think
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > there
> > > > > is
> > > > > > > some
> > > > > > > > > > > benefit
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful
> > that
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > we
> > > > post
> > > > > > > > > > > instructions
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > give
> > > > > > > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > working.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > Tuesday
> > > > > > at
> > > > > > > > that
> > > > > > > > > > > time
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > work
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > -Jay
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > On
> > > > Tue,
> > > > > > Feb
> > > > > > > > 24,
> > > > > > > > > > > 2015
> > > > > > > > > > > > at
> > > > > > > > > > > > > > > 8:18
> > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > > Weekly
> > > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > > great
> > > > > > > > > > > > > > > maybe
> > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> ????
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > I
> > > > > don't
> > > > > > > > mind
> > > > > > > > > > > google
> > > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > whatever
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> so
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > we
> > > > > know
> > > > > > > the
> > > > > > > > > > > apache
> > > > > > > > > > > > > irc
> > > > > > > > > > > > > > > > > channel
> > > > > > > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > goes?
> > > > > > We
> > > > > > > > can
> > > > > > > > > > pull
> > > > > > > > > > > > > > > transcripts
> > > > > > > > > > > > > > > > > > too
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > makes
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > it
> > > > > > > helpful
> > > > > > > > > for
> > > > > > > > > > > > > things.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > ~
> > > > > > > Joestein
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > On
> > > > > Tue,
> > > > > > > Feb
> > > > > > > > > 24,
> > > > > > > > > > > > 2015
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > 11:10
> > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > jay.kreps@gmail.com
> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > We'd
> > > > > > > > talked
> > > > > > > > > > > about
> > > > > > > > > > > > > > > doing a
> > > > > > > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > about
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > > > > > generalizing
> > > > > > > > > > > > that a
> > > > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > good
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > for
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > > > everyone
> > > > > > > > > > > > spending a
> > > > > > > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > maybe
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > sync
> > > > > > up
> > > > > > > > > once
> > > > > > > > > > a
> > > > > > > > > > > > > week.
> > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > design
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > > stuff,
> > > > > > > > make
> > > > > > > > > > > sure
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > are on
> > > > > > > > > > > > > > > > > > top
> > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > tricky
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > > issues,
> > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > We
> > > > > > can
> > > > > > > > make
> > > > > > > > > > it
> > > > > > > > > > > > > > publicly
> > > > > > > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> who
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > > likes.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > Any
> > > > > > > > > interest
> > > > > > > > > > in
> > > > > > > > > > > > > doing
> > > > > > > > > > > > > > > this?
> > > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> next
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > week.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > -Jay
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > On
> > > > > > Tue,
> > > > > > > > Feb
> > > > > > > > > > 24,
> > > > > > > > > > > > > 2015
> > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > >
> > > > > Hi
> > > > > > > all,
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > >
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > >
> > > > > > I've
> > > > > > > > > > updated
> > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > page,
> > > > > > > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > added
> > > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > >
> > > > >
> > > > > > some
> > > > > > > > > > > > >
> > > > > > > > > > > > ...
> > > > > > > > > > > >
> > > > > > > > > > > > [Message clipped]
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > --
> > > > > > > > > > -- Guozhang
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Jun,

Thanks for such an accurate review!

Most of your remarks I didn't fix in the previous change because
I thought we'd finalize them today, during the call. Anyway, I've
fixed them now.
My comments to ensure I covered everything:

110. Fixed. Removed global error code.

111.1-3. Removed redundant TopicName, Partitions and ReplicationFactor
111.4 Yes, now we are in the situation when TopicMetadataResponse
contains everything we need except topic level configs. I'm
okay with both solutions, probably makes sense to discuss and pick
the preferable one.

112. Need to think about it a bit more. Currently we may distinguish
3 states of specific partition being reassigned:
a) Completed - absent in /reassign_partitions zk path and DescribeTopic/
TopicMetadata shows expected assignment
2) In progress - present in /reassign_partitions
3) Failed - absent in /reassign_partitions but DescribeTopic/TopicMetadata
replicas field doesn't correspond expected
Now, if use only DescribeTopic to check reassignment status - how can we
distinguish states "In Progress" and "Failed"?
Also, there are some concerns that current reassignment status provided
by ReaasignPartitionsCommand (and the same was intended in KIP-4) is
very spurious, non-informative. Probably it makes sense to add additional
data there (thus separate request may be useful), I'm considering different
options now - btw, comments here are highly appreciated!

113-114. Okay, made it compliant with topic commands. About "The error code
will
then be per topic" - just a note: currently (ReassignPartitionsCommand)
validation
is done for the whole reassignment string, so in case of errors for
specific
partitions the reassignment is NOT started at all. Both options look fine
for me,
it's just the logic will be slightly changed.

115.1 Removed NotControllerReceivedAdminCommand. But why should we
remove PreferredReplicaLeaderElectionInProgress and
ReassignPartitionsInProgress?
On Reassign/Preferred..Request we create admin zk path, we can of course
update the path, but wouldn't it be safer if we simply refuse to start new
reassignment in case one is in progress?

115.2 Yes, paragraph Protocol Errors covers only errors that are proposed
to be added. I didn't want to list all errors that are to be added since
anyway
the final version will be clear after the development is completed.

100. Fixed, normalized json string in Create/Alter.

Thanks,
Andrii Biletskyi

On Tue, Mar 24, 2015 at 12:58 AM, Jun Rao <ju...@confluent.io> wrote:

> Andrii,
>
> I looked at the latest wiki for this KIP. I have a few more comments.
>
> 110. In CreateTopicResponse, AlterTopicResponse, DeleteTopicResponse and
> DescribeTopicResponse, we probably don't need the global error code. Our
> current convention is to just use the same global error code in each topic.
> This simplifies the error checking on the client side.
>
> 111. DescribeTopicResponse:
> 111.1 Our protocol definition doesn't support optional fields. So, we can't
> make Leader an optional field.
> 111.2 Do we really need the field Partitions and ReplicationFactor in
> TopicConfigDetails?
> The former is basically the size of the TopicPartitionDetails array and the
> latter can be found from the size of the Replica array.
> 111.3 Do we need TopicName in TopicDescription since it's already in
> DescribeTopicResponse?
> 111.4 Finally, DescribeTopicResponse is very similar to
> TopicMetadataResponse. So, it probably makes sense just to keep one of them
> in the future. Should we just use DescribeTopicRequest/Response to replace
> TopicMetadataRequest/Response in the producer/consumer client in the future
> (since DescribeTopicRequest doesn't trigger auto topic creation)? If so, we
> will need to add the broker list in DescribeTopicResponse.
>
> 112. Thinking about this a bit more, I don't think we need a separate
> VerifyReassignPartitionRequest/Response.
> We can just use DescribeTopic to get the assigned replicas and check if
> they match the target replica assignment in the client. The controller
> propagates the metadata change after the reassignment completes for each
> partition.
>
> 113. ReassignPartitionRequest: For consistency, we probably want to nest
> the partition data under topic. So instead of
>   ReassignPartitionRequest => [Topic PartitionId [ReplicaId]]
> we can have sth like
>   ReassignPartitionRequest => [Topic [PartitionId [ReplicaId]]]
> The error code will then be per topic.
>
> 114. PreferredReplicaLeaderElectionRequest: Same as the above. Instead of
>   PreferredReplicaLeaderElectionRequest => [Topic PartitionId]
> we can have
>   PreferredReplicaLeaderElectionRequest => [Topic [PartitionId]]
> Again, the error code will be per topic.
>
> 115. ErrorCode:
> 115.1 NotControllerReceivedAdminRequest,
> PreferredReplicaLeaderElectionInProgress
> and ReassignPartitionsInProgress are likely not needed any more.
> 115.2 We probably need to add UnknownTopicOrPartitionCode since it's a
> valid error code for some of the requests (e.g. describe, alter, etc) when
> a non-existing topic is specified.
>
>
> Previous comments.
> 100. ReplicaAssignment in CreateTopicRequest and AlterTopicRequest are
> still a json string. We need to flatten them.
>
>
> Thanks,
>
> Jun
>
>
> On Sat, Mar 21, 2015 at 2:03 AM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Guozhang,
> >
> > I'm not sure I understand how can we use those tools in CLI.
> > First of all, those are *Test*Utils, e.g. waitUntilMetadataIsPropagated
> > (which might be very useful for us) requires servers:Seq[KafkaServer]
> > as an argument:
> >
> > def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic:
> String,
> > partition: Int, timeout: Long = 5000L): Int
> >
> > how can we have access it at runtime in one of the brokers?
> >
> > Secondly, how can user directly call these tools if zookeeper might be
> > not accessible (which is used in the tools, right?) at all - e.g. behind
> > the VPC
> > in AWS. Furthermore, I think with KIP-4 we are trying to create an
> > abstraction and
> > a single point of zookeeper interactions, to eliminate direct calls to ZK
> > either to get
> > some cluster information or to change something.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> > On Sat, Mar 21, 2015 at 3:37 AM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > Andrii,
> > >
> > > Actually the checking logic Jun mentioned is already implemented as
> > > TestUtils.waitUntilXXX (LeaderIsElected, MetadataIsPropagated, etc...)
> I
> > > think we can extend these functions as CLI tools like TopicCommand so
> > that
> > > users re-implementing such endpoint can directly call something like
> > > java.tools.WaitUntilXXX (of course this requires them to have javac
> > > installed, which should be a reasonable requirement?)
> > >
> > > Guozhang
> > >
> > >
> > >
> > > On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Jun,
> > > >
> > > > Not that I was saying we need to make requests blocking on server,
> > > > it was just to emphasize that with async requests a client
> > > implementations
> > > > may be a little bit more than just issue request - get the response.
> > > > Thanks for the explanation, I understand now that we can go with
> agreed
> > > > solution though it may not be perfect.
> > > > I believe this was one of the last controversial questions from the
> > list.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Andrii,
> > > > >
> > > > > A few points.
> > > > >
> > > > > 1. Create/Alter can typically complete quickly. So, it's possible
> to
> > > make
> > > > > the request block until it's completed. However, currently, doing
> > this
> > > at
> > > > > the broker is a bit involved. To make Create block, we will need to
> > add
> > > > > some callbacks in KafkaController. This is possible. However, the
> > > > > controller logic currently is pretty completed. It would probably
> be
> > > > better
> > > > > if we clean it up first before adding more complexity to it. Alter
> is
> > > > even
> > > > > trickier. Adding partition is currently handled through
> > > KafkaController.
> > > > So
> > > > > it can be dealt with in a similar way. However, Alter config is
> done
> > > > > completely differently. It doesn't go through the controller.
> > Instead,
> > > > each
> > > > > broker listens to ZooKeeper directly. So, it's not clear if there
> is
> > an
> > > > > easy way on the broker to figure out whether a config is applied on
> > > every
> > > > > broker.
> > > > >
> > > > > 2. Delete can potentially take long if a replica to be deleted is
> > > > offline.
> > > > > PreferredLeader/PartitionReassign can also take long. So, we can't
> > > really
> > > > > make those requests block on the broker.
> > > > >
> > > > > As you can see, at this moment it's not easy to make all admin
> > requests
> > > > > block on the broker. So, if we want the blocking feature in the
> admin
> > > > > utility in the short term, doing the completion check at the admin
> > > client
> > > > > is probably an easier route, even though it may not be ideal.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Jun,
> > > > > >
> > > > > > I see your point. But wouldn't that lead to a "fat" client
> > > > > implementations?
> > > > > > Suppose someone would like to implement client for Admin Wire
> > > protocol.
> > > > > > Not only people will have to code quite complicated logic like
> > "send
> > > > > > describe
> > > > > > request to each broker" (again state machin?) but it will also
> mean
> > > > > people
> > > > > > must understand internal kafka logic related to topic storage and
> > how
> > > > > > information is propageted from the controller to brokers.
> > > > > > I see this like a dilemma between having a concise Wire Protocol
> > and
> > > > > > self-sufficient API to make client implementations simple.
> > > > > > I don't have a win-win solution though.
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > >
> > > > > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > >
> > > > > > > For 1), 2) and 3), blocking would probably mean that the new
> > > metadata
> > > > > is
> > > > > > > propagated to every broker. To achieve that, the client can
> keep
> > > > > issuing
> > > > > > > the describe topic request to every broker until it sees the
> new
> > > > > metadata
> > > > > > > in the response.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >
> > > > > > > > Hm, actually the ticket you linked, Guozhang, brings as back
> > > > > > > > to the problem what should be considered a post-condition for
> > > > > > > > each of the admin commands.
> > > > > > > > In my understanding:
> > > > > > > >
> > > > > > > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > > > > > > (Not the controller picked up changes from zk and broadcasted
> > > > > > > > LeaderAndIsr and UpdateMetadata)
> > > > > > > >
> > > > > > > > 2) AlterTopic - same as 1) - broker changed assignment data
> > > > > > > > in zookeeper or created admin path for topic config change
> > > > > > > >
> > > > > > > > 3) DeleteTopic - admin path /admin/delete_topics is created
> > > > > > > >
> > > > > > > > 4) ReassignPartitions and PreferredReplica - corresponding
> > admin
> > > > > > > > path is created
> > > > > > > >
> > > > > > > > Now what can be considered a completed operation from the
> > > client's
> > > > > > > > perspective?
> > > > > > > > 1) Topic is created once corresponding data is in zk
> > > > > > > > (I remember there were some thoughts that it'd be good to
> > > consider
> > > > > > > > topic created once all replicas receive information about it
> > and
> > > > thus
> > > > > > > > clients can produce/consume from it, but as was discussed
> this
> > > > seems
> > > > > > > > to be a hard thing to do)
> > > > > > > >
> > > > > > > > 2) Probably same as 1), so right after AlterTopic is issued
> > > > > > > >
> > > > > > > > 3) The topic has been removed from /brokers/topics
> > > > > > > >
> > > > > > > > 4) ReassignPartitions and PrefferedReplica were discussed
> > > earlier -
> > > > > > > > in short the former is completed once partition state info in
> > zk
> > > > > > matches
> > > > > > > > reassignment request and admin path is empty, the latter -
> once
> > > > data
> > > > > > > > in zk shows that head of assignned replicas of the partition
> > and
> > > > > leader
> > > > > > > > is the same replica
> > > > > > > >
> > > > > > > > Thoughts?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Andrii Biletskyi
> > > > > > > >
> > > > > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <
> > > wangguoz@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > I think while loop is fine for supporting blocking, just
> that
> > > we
> > > > > need
> > > > > > > to
> > > > > > > > > add back off to avoid bombarding brokers with DescribeTopic
> > > > > requests.
> > > > > > > > >
> > > > > > > > > Also I have linked KAFKA-1125
> > > > > > > > > <https://issues.apache.org/jira/browse/KAFKA-1125> to your
> > > > > proposal,
> > > > > > > and
> > > > > > > > > when KAFKA-1694 is done this ticket can also be closed.
> > > > > > > > >
> > > > > > > > > Guozhang
> > > > > > > > >
> > > > > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >
> > > > > > > > > > Great.
> > > > > > > > > > I want to elaborate this a bit more, to see we are on the
> > > same
> > > > > page
> > > > > > > > > > concerning the client code.
> > > > > > > > > >
> > > > > > > > > > So with all topic commands being async a client
> > (AdminClient
> > > in
> > > > > our
> > > > > > > > > > case or any other other client people would like to
> > > implement)
> > > > to
> > > > > > > > support
> > > > > > > > > > a blocking operation (which seems to be a natural
> use-case
> > > e.g.
> > > > > for
> > > > > > > > topic
> > > > > > > > > > creation): would have to do:
> > > > > > > > > > 1. issue CreateTopicRequest
> > > > > > > > > > 2. if successful, in a "while" loop send
> > DescribeTopicRequest
> > > > and
> > > > > > > > > > break the loop once all topics are returned in response
> (or
> > > > upon
> > > > > > > > > timeout).
> > > > > > > > > > 3. if unsuccessful throw exception
> > > > > > > > > > Would it be okay?
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Andrii Biletskyi
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <
> jun@confluent.io
> > >
> > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Andrii,
> > > > > > > > > > >
> > > > > > > > > > > I think you are right. It seems that only
> > > ReassignPartitions
> > > > > > needs
> > > > > > > a
> > > > > > > > > > > separate verification request.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Guys,
> > > > > > > > > > > > I like this idea too. Let's stick with that. I'll
> > update
> > > > KIP
> > > > > > > > > > accordingly.
> > > > > > > > > > > >
> > > > > > > > > > > > I was also thinking we can avoid adding dedicated
> > status
> > > > > check
> > > > > > > > > > > > requests for topic commands. - We have everything in
> > > > > > > DescribeTopic
> > > > > > > > > > > > for that! E.g.:
> > > > > > > > > > > > User issued CreateTopic - to check the status client
> > > sends
> > > > > > > > > > DescribeTopic
> > > > > > > > > > > > and checks whether is something returned for that
> > topic.
> > > > The
> > > > > > same
> > > > > > > > for
> > > > > > > > > > > > alteration, deletion.
> > > > > > > > > > > > Btw, PreferredReplica status can be also checked with
> > > > > > > > > > > DescribeTopicRequest
> > > > > > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > > > > > For ReassignPartitions as discussed we'll need to
> have
> > a
> > > > > > separate
> > > > > > > > > > > Verify...
> > > > > > > > > > > > request.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> > > > > > > wangguoz@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > +1 on broker writing to ZK for async handling. I
> was
> > > > > thinking
> > > > > > > > that
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > end state the admin requests would be eventually
> sent
> > > to
> > > > > > > > controller
> > > > > > > > > > > > either
> > > > > > > > > > > > > through re-routing or clients discovering them,
> > instead
> > > > of
> > > > > > > > letting
> > > > > > > > > > > > > controller listen on ZK admin path. But thinking
> > about
> > > > it a
> > > > > > > > second
> > > > > > > > > > > time,
> > > > > > > > > > > > I
> > > > > > > > > > > > > think it is actually simpler to let controller
> manage
> > > > > > > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > > > > > > jjkoshy.w@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > +1 as well. I think it helps to keep the
> rerouting
> > > > > approach
> > > > > > > > > > > orthogonal
> > > > > > > > > > > > > > to this KIP.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay
> Kreps
> > > > > wrote:
> > > > > > > > > > > > > > > I'm +1 on Jun's suggestion as long as it can
> work
> > > for
> > > > > all
> > > > > > > the
> > > > > > > > > > > > requests.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <
> > > > > > jun@confluent.io
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > (a) Admin requests can be sent to and handled
> > by
> > > > any
> > > > > > > > broker.
> > > > > > > > > > > > > > > > (b) Admin requests are processed
> > asynchronously,
> > > at
> > > > > > least
> > > > > > > > for
> > > > > > > > > > > now.
> > > > > > > > > > > > > > That is,
> > > > > > > > > > > > > > > > when the client gets a response, it just
> means
> > > that
> > > > > the
> > > > > > > > > request
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > initiated, but not necessarily completed.
> Then,
> > > > it's
> > > > > up
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > client
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > issue another request to check the status for
> > > > > > completion.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > To support (a), we were thinking of doing
> > request
> > > > > > > > forwarding
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > controller (utilizing KAFKA-1912). I am
> making
> > an
> > > > > > > > alternative
> > > > > > > > > > > > > proposal.
> > > > > > > > > > > > > > > > Basically, the broker can just write to
> > ZooKeeper
> > > > to
> > > > > > > inform
> > > > > > > > > the
> > > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > about the request. For example, to handle
> > > > > > > > > > partitionReassignment,
> > > > > > > > > > > > the
> > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > will just write the requested partitions to
> > > > > > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > > > > > (like what AdminUtils currently does) and
> then
> > > > send a
> > > > > > > > > response
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > client. This shouldn't take long and the
> > > > > implementation
> > > > > > > > will
> > > > > > > > > be
> > > > > > > > > > > > > simpler
> > > > > > > > > > > > > > > > than forwarding the requests to the
> controller
> > > > > through
> > > > > > > RPC.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii
> > > Biletskyi <
> > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Jun,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I might be wrong but didn't we agree we
> will
> > > let
> > > > > any
> > > > > > > > broker
> > > > > > > > > > > from
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > cluster handle *long-running* admin
> requests
> > > (at
> > > > > this
> > > > > > > > time
> > > > > > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > reassignPartitions), via zk admin path.
> Thus
> > > > > > > CreateTopics
> > > > > > > > > etc
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I think we agreed that those admin
> requests
> > > can
> > > > > be
> > > > > > > > issued
> > > > > > > > > > to
> > > > > > > > > > > > any
> > > > > > > > > > > > > > > > broker.
> > > > > > > > > > > > > > > > > > Because of that, there doesn't seem to
> be a
> > > > > strong
> > > > > > > need
> > > > > > > > > to
> > > > > > > > > > > know
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > controller. So, perhaps we can proceed by
> > not
> > > > > > making
> > > > > > > > any
> > > > > > > > > > > change
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > format of TMR right now. When we start
> > using
> > > > > create
> > > > > > > > topic
> > > > > > > > > > > > request
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > producer, we will need a new version of
> TMR
> > > > that
> > > > > > > > doesn't
> > > > > > > > > > > > trigger
> > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > As a first cut implementation, I think
> the
> > > > broker
> > > > > > can
> > > > > > > > > just
> > > > > > > > > > > > write
> > > > > > > > > > > > > > to ZK
> > > > > > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > >
> > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > > > > > requests, instead of forwarding them to
> the
> > > > > > > controller.
> > > > > > > > > > This
> > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel
> > Koshy
> > > <
> > > > > > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > For (1) yes we will circle back on that
> > > > shortly
> > > > > > > after
> > > > > > > > > > > syncing
> > > > > > > > > > > > > up
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > person. I think it is close to getting
> > > > > committed
> > > > > > > > > although
> > > > > > > > > > > > > > development
> > > > > > > > > > > > > > > > > > > for KAFKA-1927 can probably begin
> without
> > > it.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > There is one more item we covered at
> the
> > > > > hangout.
> > > > > > > > i.e.,
> > > > > > > > > > > > whether
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > want to add the coordinator to the
> topic
> > > > > metadata
> > > > > > > > > > response
> > > > > > > > > > > or
> > > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > There are two reasons I think we should
> > try
> > > > and
> > > > > > > avoid
> > > > > > > > > > > adding
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > > > > > > > > - If we finally do request rerouting in
> > > Kafka
> > > > > > then
> > > > > > > > the
> > > > > > > > > > > field
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > >   little to no value. (It still helps
> to
> > > > have a
> > > > > > > > > separate
> > > > > > > > > > > > > > > > > > >   ClusterMetadataRequest to query for
> > > > > > cluster-wide
> > > > > > > > > > > > information
> > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > >   'which broker is the controller?' as
> > Joe
> > > > > > > > mentioned.)
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > I think it would be cleaner to have an
> > > > explicit
> > > > > > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > > > > > that you can send to any broker in
> order
> > to
> > > > > > obtain
> > > > > > > > the
> > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > in the future possibly other
> cluster-wide
> > > > > > > > > information). I
> > > > > > > > > > > > think
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > main argument against doing this and
> > > instead
> > > > > > adding
> > > > > > > > it
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > metadata response was convenience -
> i.e.,
> > > you
> > > > > > don't
> > > > > > > > > have
> > > > > > > > > > to
> > > > > > > > > > > > > > discover
> > > > > > > > > > > > > > > > > > > the controller in advance. However, I
> > don't
> > > > see
> > > > > > > much
> > > > > > > > > > actual
> > > > > > > > > > > > > > > > > > > benefit/convenience in this and in fact
> > > think
> > > > > it
> > > > > > > is a
> > > > > > > > > > > > > non-issue.
> > > > > > > > > > > > > > Let
> > > > > > > > > > > > > > > > > > > me know if I'm overlooking something
> > here.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > As an example, say we need to initiate
> > > > > partition
> > > > > > > > > > > reassignment
> > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > > issuing the new
> ReassignPartitionsRequest
> > > to
> > > > > the
> > > > > > > > > > controller
> > > > > > > > > > > > > > (assume
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > already have the desired manual
> partition
> > > > > > > > assignment).
> > > > > > > > > > If
> > > > > > > > > > > we
> > > > > > > > > > > > > > are to
> > > > > > > > > > > > > > > > > > > augment topic metadata response then
> the
> > > flow
> > > > > be
> > > > > > > > > > something
> > > > > > > > > > > > like
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > :
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > - Issue topic metadata request to any
> > > broker
> > > > > (and
> > > > > > > > > > discover
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > > > > > - Connect to controller if required
> > (i.e.,
> > > if
> > > > > the
> > > > > > > > > broker
> > > > > > > > > > > > above
> > > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > > - Issue the partition reassignment
> > request
> > > to
> > > > > the
> > > > > > > > > > > controller.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > With an explicit cluster metadata
> request
> > > it
> > > > > > would
> > > > > > > > be:
> > > > > > > > > > > > > > > > > > > - Issue cluster metadata request to any
> > > > broker
> > > > > > > > > > > > > > > > > > > - Connect to controller if required
> > (i.e.,
> > > if
> > > > > the
> > > > > > > > > broker
> > > > > > > > > > > > above
> > > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > > - Issue the partition reassignment
> > request
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > So it seems to add little practical
> value
> > > and
> > > > > > > bloats
> > > > > > > > > > topic
> > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > The other angle to this is the
> following
> > -
> > > is
> > > > > it
> > > > > > a
> > > > > > > > > matter
> > > > > > > > > > > of
> > > > > > > > > > > > > > naming?
> > > > > > > > > > > > > > > > > > > Should we just rename topic metadata
> > > > > > > request/response
> > > > > > > > > to
> > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > MetadataRequest/Response and add
> cluster
> > > > > metadata
> > > > > > > to
> > > > > > > > > it?
> > > > > > > > > > By
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > same
> > > > > > > > > > > > > > > > > > > token should we also allow querying for
> > the
> > > > > > > consumer
> > > > > > > > > > > > > coordinator
> > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > in future transaction coordinator) as
> > well?
> > > > > This
> > > > > > > > leads
> > > > > > > > > > to a
> > > > > > > > > > > > > > bloated
> > > > > > > > > > > > > > > > > > > request which isn't very appealing and
> > > > > altogether
> > > > > > > > > > > confusing.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM
> -0700,
> > > Jun
> > > > > Rao
> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 1. I just realized that in order to
> > start
> > > > > > working
> > > > > > > > on
> > > > > > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > need to merge the changes to
> > > > > > OffsetCommitRequest
> > > > > > > > > (from
> > > > > > > > > > > > 0.8.2)
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > > > > > This is planned to be done as part of
> > > > > > KAFKA-1634.
> > > > > > > > So,
> > > > > > > > > > we
> > > > > > > > > > > > will
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 2. Thinking about this a bit more, if
> > the
> > > > > > > semantic
> > > > > > > > of
> > > > > > > > > > > those
> > > > > > > > > > > > > > "write"
> > > > > > > > > > > > > > > > > > > > requests is async (i.e., after the
> > client
> > > > > gets
> > > > > > a
> > > > > > > > > > > response,
> > > > > > > > > > > > it
> > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > > > > > that the operation is initiated, but
> > not
> > > > > > > > necessarily
> > > > > > > > > > > > > > completed), we
> > > > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > > really need to forward the requests
> to
> > > the
> > > > > > > > > controller.
> > > > > > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > > > > > receiving broker can just write the
> > > > operation
> > > > > > to
> > > > > > > ZK
> > > > > > > > > as
> > > > > > > > > > > the
> > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > > > > > line tool previously does. This will
> > > > simplify
> > > > > > the
> > > > > > > > > > > > > > implementation.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 8. There is another implementation
> > detail
> > > > for
> > > > > > > > > describe
> > > > > > > > > > > > topic.
> > > > > > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > want to read the topic config from
> the
> > > > broker
> > > > > > > > cache,
> > > > > > > > > > > > instead
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > > > > > Currently, every broker reads the
> > > > topic-level
> > > > > > > > config
> > > > > > > > > > for
> > > > > > > > > > > > all
> > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > > However, it ignores those for topics
> > not
> > > > > hosted
> > > > > > > on
> > > > > > > > > > > itself.
> > > > > > > > > > > > > So,
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > to change TopicConfigManager a bit so
> > > that
> > > > it
> > > > > > > > caches
> > > > > > > > > > the
> > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM,
> Andrii
> > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala requests
> > > > > objects,
> > > > > > > use
> > > > > > > > > java
> > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > > > > > >     A: Gwen kindly took that
> > > > (KAFKA-1927).
> > > > > > It's
> > > > > > > > > > > important
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > speed
> > > > > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > > > > > >          there since this ticket
> > blocks
> > > > > other
> > > > > > > > > > important
> > > > > > > > > > > > > > changes.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute facility
> vs
> > > > client
> > > > > > > > > > maintaining
> > > > > > > > > > > > > > cluster
> > > > > > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > > > > > >     A: Jay has added pseudo code to
> > > > > > KAFKA-1912
> > > > > > > -
> > > > > > > > > need
> > > > > > > > > > > to
> > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > > > > > >         easy to implement as a
> > > > server-side
> > > > > > > > feature
> > > > > > > > > > > > > (comments
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 3. Q: Controller field in wire
> > > protocol.
> > > > > > > > > > > > > > > > > > > > >     A: This might be useful for
> > > clients,
> > > > > add
> > > > > > > this
> > > > > > > > > to
> > > > > > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic creation
> from
> > > TMR.
> > > > > > > > > > > > > > > > > > > > >     A: I will add proposed by Jun
> > > > solution
> > > > > > > (using
> > > > > > > > > > > > clientId
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR
> vs
> > > > > grabbing
> > > > > > > all
> > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > >     A: It was decided to try to
> > gather
> > > > all
> > > > > > > > changes
> > > > > > > > > to
> > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > > > > > >         In case of TMR it worth
> > > checking:
> > > > > > > > > KAFKA-2020
> > > > > > > > > > > and
> > > > > > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to
> > deserialize
> > > > > > user's
> > > > > > > > > input
> > > > > > > > > > in
> > > > > > > > > > > > CLI
> > > > > > > > > > > > > > tool.
> > > > > > > > > > > > > > > > > > > > >     A: Use jackson for that, /tools
> > > > project
> > > > > > is
> > > > > > > a
> > > > > > > > > > > separate
> > > > > > > > > > > > > > jar so
> > > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs
> > > > generic
> > > > > > > status
> > > > > > > > > > check
> > > > > > > > > > > > > > command.
> > > > > > > > > > > > > > > > > > > > >      A: For long-running requests
> > like
> > > > > > reassign
> > > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > > > > > >          it makes sense to
> introduce
> > > it.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >  Please add, correct me if I missed
> > > > > > something.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM,
> > Andrii
> > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > You are right, I removed
> > > > ClusterMetadata
> > > > > > > > because
> > > > > > > > > we
> > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > > > > > what we need in TopicMetadata.
> > Also,
> > > as
> > > > > Jay
> > > > > > > > > pointed
> > > > > > > > > > > out
> > > > > > > > > > > > > > > > earlier,
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > would like to have "orthogonal"
> > API,
> > > > but
> > > > > at
> > > > > > > the
> > > > > > > > > > same
> > > > > > > > > > > > time
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > But I like your idea and even
> have
> > > some
> > > > > > other
> > > > > > > > > > > arguments
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > > > > > There is also
> DescribeTopicRequest
> > > > which
> > > > > > was
> > > > > > > > > > proposed
> > > > > > > > > > > > in
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > > > > > it returns topic configs,
> > partitions,
> > > > > > > > replication
> > > > > > > > > > > > factor
> > > > > > > > > > > > > > plus
> > > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > > > > > leader replica. The later part is
> > > > really
> > > > > > > > already
> > > > > > > > > > > there
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > > > > > So again we'll have to add stuff
> to
> > > > TMR,
> > > > > > not
> > > > > > > to
> > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > > newly added requests. However,
> this
> > > way
> > > > > > we'll
> > > > > > > > end
> > > > > > > > > > up
> > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > > > > > request which returns cluster
> > > metadata,
> > > > > > topic
> > > > > > > > > > > > replication
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > > > plus partition replication data.
> > > Seems
> > > > > > > logical
> > > > > > > > to
> > > > > > > > > > > split
> > > > > > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > > > > > - ClusterMetadata (brokers +
> > > > controller,
> > > > > > > maybe
> > > > > > > > > smth
> > > > > > > > > > > > else)
> > > > > > > > > > > > > > > > > > > > > > - TopicMetadata (topic info +
> > > partition
> > > > > > > > details)
> > > > > > > > > > > > > > > > > > > > > > But since current TMR is involved
> > in
> > > > lots
> > > > > > of
> > > > > > > > > places
> > > > > > > > > > > > > > (including
> > > > > > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > > > > as I understand) this might be
> very
> > > > > serious
> > > > > > > > > change
> > > > > > > > > > > and
> > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > > > > > sense to stick with current
> > approach.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM,
> > Joel
> > > > > > Koshy <
> > > > > > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> I may be missing some context
> but
> > > > > > hopefully
> > > > > > > > this
> > > > > > > > > > > will
> > > > > > > > > > > > > > also be
> > > > > > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > > > > > >> today: I thought the earlier
> > > proposal
> > > > > > where
> > > > > > > > > there
> > > > > > > > > > > was
> > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > > > > > >> ClusterMetadata request was
> > clearer
> > > > and
> > > > > > > > > explicit.
> > > > > > > > > > > > During
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > >> this thread I think the
> conclusion
> > > was
> > > > > > that
> > > > > > > > the
> > > > > > > > > > main
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > >> controller information and that
> > can
> > > be
> > > > > > > rolled
> > > > > > > > > into
> > > > > > > > > > > the
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > >> response but that seems a bit
> > > > irrelevant
> > > > > > to
> > > > > > > > > topic
> > > > > > > > > > > > > > metadata.
> > > > > > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > > > > > >> think the full broker-list is
> also
> > > > > > > irrelevant
> > > > > > > > to
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > >> it is already there and in use.
> I
> > > > think
> > > > > > > there
> > > > > > > > is
> > > > > > > > > > > still
> > > > > > > > > > > > > > room
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > > >> explicit ClusterMetadata request
> > > since
> > > > > > there
> > > > > > > > may
> > > > > > > > > > be
> > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > > >> cluster-level information that
> we
> > > may
> > > > > want
> > > > > > > to
> > > > > > > > > add
> > > > > > > > > > > over
> > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> have nothing to do with topic
> > > > metadata).
> > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at
> 02:45:30PM
> > > > > +0200,
> > > > > > > > Andrii
> > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > 101. Okay, if you say that
> such
> > > use
> > > > > case
> > > > > > > is
> > > > > > > > > > > > > important. I
> > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > > >> > using clientId for these
> > purposes
> > > is
> > > > > > fine
> > > > > > > -
> > > > > > > > if
> > > > > > > > > > we
> > > > > > > > > > > > > > already
> > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > > > > > >> > as part of all Wire protocol
> > > > messages,
> > > > > > why
> > > > > > > > not
> > > > > > > > > > use
> > > > > > > > > > > > > that.
> > > > > > > > > > > > > > > > > > > > > >> > I will update KIP-4 page if
> > nobody
> > > > has
> > > > > > > other
> > > > > > > > > > ideas
> > > > > > > > > > > > > > (which
> > > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the
> KIP
> > > > > > > > accordingly.
> > > > > > > > > I
> > > > > > > > > > > > think
> > > > > > > > > > > > > > we can
> > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > > > > > >> > fine-grained error codes if
> some
> > > > error
> > > > > > > code
> > > > > > > > > > > received
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > >> > won't give enough context to
> > > return
> > > > a
> > > > > > > > > > descriptive
> > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > message
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > Look forward to discussing all
> > > > > > outstanding
> > > > > > > > > > issues
> > > > > > > > > > > in
> > > > > > > > > > > > > > detail
> > > > > > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59
> > PM,
> > > > Jun
> > > > > > Rao
> > > > > > > <
> > > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > 101. There may be a use case
> > > where
> > > > > you
> > > > > > > > only
> > > > > > > > > > want
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > > > > > >> > > manually by admins.
> Currently,
> > > you
> > > > > can
> > > > > > > do
> > > > > > > > > that
> > > > > > > > > > > by
> > > > > > > > > > > > > > > > disabling
> > > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > >> > > creation and issue topic
> > > creation
> > > > > from
> > > > > > > the
> > > > > > > > > > > > > > TopicCommand.
> > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > > > > > >> > > topic creation completely on
> > the
> > > > > > broker
> > > > > > > > and
> > > > > > > > > > > don't
> > > > > > > > > > > > > > have a
> > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > > >> > > between topic creation
> > requests
> > > > from
> > > > > > the
> > > > > > > > > > regular
> > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > > > > > >> > > can't support manual topic
> > > > creation
> > > > > > any
> > > > > > > > > more.
> > > > > > > > > > I
> > > > > > > > > > > > was
> > > > > > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > > > > > >> > > way of distinguishing the
> > > clients
> > > > > > making
> > > > > > > > the
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > > creation
> > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > > >> > > using clientId. For example,
> > the
> > > > > admin
> > > > > > > > tool
> > > > > > > > > > can
> > > > > > > > > > > > set
> > > > > > > > > > > > > > it to
> > > > > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > > > > > >> > > admin and the broker can
> treat
> > > > that
> > > > > > > > clientId
> > > > > > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > Also, there is a related
> > > > discussion
> > > > > in
> > > > > > > > > > > KAFKA-2020.
> > > > > > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > >> > > following in
> > > > TopicMetadataResponse:
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > 1. If leader is not
> available,
> > > we
> > > > > set
> > > > > > > the
> > > > > > > > > > > > partition
> > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > > > > > >> > > 2. If a non-leader replica
> is
> > > not
> > > > > > > > available,
> > > > > > > > > > we
> > > > > > > > > > > > take
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > > >> > > the assigned replica list
> and
> > > isr
> > > > in
> > > > > > the
> > > > > > > > > > > response.
> > > > > > > > > > > > > As
> > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > > > > > >> > > doing that, we set the
> > partition
> > > > > level
> > > > > > > > error
> > > > > > > > > > > code
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > This has a few problems.
> > First,
> > > > > > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > > > > > >> > > an error, at least for the
> > > normal
> > > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > > > > > >> > > to find out the leader.
> > Second,
> > > it
> > > > > can
> > > > > > > > > happen
> > > > > > > > > > > that
> > > > > > > > > > > > > > both
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> > > another replica are not
> > > available
> > > > at
> > > > > > the
> > > > > > > > > same
> > > > > > > > > > > > time.
> > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > > >> > > to indicate both. Third,
> even
> > > if a
> > > > > > > replica
> > > > > > > > > is
> > > > > > > > > > > not
> > > > > > > > > > > > > > > > available,
> > > > > > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > > > > > >> > > useful to return its replica
> > id
> > > > > since
> > > > > > > some
> > > > > > > > > > > clients
> > > > > > > > > > > > > > (e.g.
> > > > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > One way to address this
> issue
> > is
> > > > to
> > > > > > > always
> > > > > > > > > > > return
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > > > > > >> > > leader, assigned replicas,
> and
> > > isr
> > > > > > > > > regardless
> > > > > > > > > > of
> > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > > > > > >> > > broker is live or not. Since
> > we
> > > > also
> > > > > > > > return
> > > > > > > > > > the
> > > > > > > > > > > > list
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > >> > > client can figure out
> whether
> > a
> > > > > leader
> > > > > > > or
> > > > > > > > a
> > > > > > > > > > > > replica
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > > > > > >> > > accordingly. This way, we
> > don't
> > > > need
> > > > > > to
> > > > > > > > set
> > > > > > > > > > the
> > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > > >> > > when the leader or a replica
> > is
> > > > not
> > > > > > > > > available.
> > > > > > > > > > > > This
> > > > > > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > > > > > >> > > protocol, but does change
> the
> > > > > > semantics.
> > > > > > > > > Since
> > > > > > > > > > > we
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > > > > > >> > > of TopicMetadataRequest
> here,
> > we
> > > > can
> > > > > > > > > > potentially
> > > > > > > > > > > > > > piggyback
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > 102.1 For those types of
> > errors
> > > > due
> > > > > to
> > > > > > > > > invalid
> > > > > > > > > > > > > input,
> > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > >> > > guard it at parameter
> > validation
> > > > > time
> > > > > > > and
> > > > > > > > > > throw
> > > > > > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > > > > > >> > > without even sending the
> > request
> > > > to
> > > > > > the
> > > > > > > > > > broker?
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at
> 10:37
> > > AM,
> > > > > > Andrii
> > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly
> >
> > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > 101. If I understand you
> > > > > correctly,
> > > > > > > you
> > > > > > > > > are
> > > > > > > > > > > > saying
> > > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > > > > > >> > > > will be ported to TMR_V1)
> > > won't
> > > > be
> > > > > > > able
> > > > > > > > to
> > > > > > > > > > > > > > automatically
> > > > > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > > >> > > > unconditionally remove
> topic
> > > > > > creation
> > > > > > > > from
> > > > > > > > > > > > there).
> > > > > > > > > > > > > > But
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > > > > > >> > > > Ok, about your proposal:
> I'm
> > > > not a
> > > > > > big
> > > > > > > > fan
> > > > > > > > > > > too,
> > > > > > > > > > > > > > when it
> > > > > > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > > > > > >> > > > clients directly in
> protocol
> > > > > schema.
> > > > > > > And
> > > > > > > > > > also
> > > > > > > > > > > > I'm
> > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > > > > > >> > > > auto.create.topics.enable
> > is a
> > > > > > server
> > > > > > > > side
> > > > > > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > > > > > >> > > > in future versions, add
> this
> > > > > setting
> > > > > > > to
> > > > > > > > > > > producer
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > based
> > > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > > > > > >> > > > UnknownTopic create topic
> > > > > explicitly
> > > > > > > by
> > > > > > > > a
> > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's
> because
> > > we
> > > > > want
> > > > > > > to
> > > > > > > > > > > support
> > > > > > > > > > > > > > batching
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > > > > > >> > > > want to give descriptive
> > error
> > > > > > > messages
> > > > > > > > > for
> > > > > > > > > > > > > clients.
> > > > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > > > > > >> > > > to construct such messages
> > > (e.g.
> > > > > > > > > AdminClient
> > > > > > > > > > > > layer
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > > > > > > >> > > > means two cases: either
> > > invalid
> > > > > > > number -
> > > > > > > > > > e.g.
> > > > > > > > > > > > -1;
> > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > > > > > >> > > > partitions argument
> wasn't)
> > -
> > > I
> > > > > > > wrapped
> > > > > > > > > > > > responses
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > > > > > >> > > > other ideas, this was just
> > > > initial
> > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll
> > > change
> > > > > > that
> > > > > > > to
> > > > > > > > > > > > probably
> > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at
> 7:16
> > > PM,
> > > > > Jun
> > > > > > > > Rao <
> > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > 101. That's what I was
> > > > thinking
> > > > > > too,
> > > > > > > > but
> > > > > > > > > > it
> > > > > > > > > > > > may
> > > > > > > > > > > > > > not be
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > > > > > >> > > > > we can let it not
> trigger
> > > auto
> > > > > > topic
> > > > > > > > > > > creation.
> > > > > > > > > > > > > > Then,
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > > > > > UnknownTopicException,
> > > > > > > > it
> > > > > > > > > > can
> > > > > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > > > > > >> > > > > createTopicRequest for
> > auto
> > > > > topic
> > > > > > > > > > creation.
> > > > > > > > > > > On
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > > > > > >> > > > > never issue
> > > > createTopicRequest.
> > > > > > This
> > > > > > > > > works
> > > > > > > > > > > > when
> > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > > > > > >> > > > > enabled on the broker
> > side.
> > > > > > > However, I
> > > > > > > > > am
> > > > > > > > > > > not
> > > > > > > > > > > > > > sure how
> > > > > > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > > > > > >> > > > > when auto topic creation
> > is
> > > > > > disabled
> > > > > > > > on
> > > > > > > > > > the
> > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > side.
> > > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > > >> > > > > want to have a way to
> > > manually
> > > > > > > create
> > > > > > > > a
> > > > > > > > > > > topic,
> > > > > > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > >> > > > > commands. However, then
> we
> > > > need
> > > > > a
> > > > > > > way
> > > > > > > > to
> > > > > > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > > > > > >> > > > > issued from the producer
> > > > clients
> > > > > > and
> > > > > > > > the
> > > > > > > > > > > admin
> > > > > > > > > > > > > > tools.
> > > > > > > > > > > > > > > > > May
> > > > > > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > >> > > > > new field in
> > > > createTopicRequest
> > > > > > and
> > > > > > > > set
> > > > > > > > > it
> > > > > > > > > > > > > > differently
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > > >> > > > > client and the admin
> > client.
> > > > > > > However,
> > > > > > > > I
> > > > > > > > > am
> > > > > > > > > > > not
> > > > > > > > > > > > > > sure if
> > > > > > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring
> > existing
> > > > > > > requests
> > > > > > > > > is a
> > > > > > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > > > > > >> > > > > posted some comments in
> > > > > > KAFKA-1927.
> > > > > > > We
> > > > > > > > > > will
> > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > > > > > >> > > > > first, before adding the
> > new
> > > > > logic
> > > > > > > in
> > > > > > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > 102. About the
> > AdminClient:
> > > > > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird
> > that
> > > > we
> > > > > > > return
> > > > > > > > > > > > exception
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > >> > > > > we should either return
> > > error
> > > > > code
> > > > > > > or
> > > > > > > > > > throw
> > > > > > > > > > > an
> > > > > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > > > > > >> > > > > 102.2. We probably
> > shouldn't
> > > > > > > > explicitly
> > > > > > > > > > use
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > > > > > >> > > > > Not every request
> > evolution
> > > > > > requires
> > > > > > > > an
> > > > > > > > > > api
> > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at
> > 4:08
> > > > AM,
> > > > > > > > Andrii
> > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > >> > > > >
> > andrii.biletskyi@stealth.ly
> > > >
> > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks for you
> comments.
> > > > > Answers
> > > > > > > > > inline:
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > 100. There are a few
> > > fields
> > > > > such
> > > > > > > as
> > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > PartitionsSerialized
> > > > > that
> > > > > > > are
> > > > > > > > > > > > > represented
> > > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite structures
> > in
> > > > > json.
> > > > > > > > Could
> > > > > > > > > we
> > > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition
> as
> > > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, now with Admin
> > Client
> > > > > this
> > > > > > > > looks
> > > > > > > > > a
> > > > > > > > > > > bit
> > > > > > > > > > > > > > weird.
> > > > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> ReassignPartitionCommand
> > > > > accepts
> > > > > > > > input
> > > > > > > > > > in
> > > > > > > > > > > > > json,
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > > > > > >> > > > > > interfaces unchanged,
> > > where
> > > > > > > > possible.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > If we port it to
> > > > deserialized
> > > > > > > > format,
> > > > > > > > > in
> > > > > > > > > > > CLI
> > > > > > > > > > > > > > (/tools
> > > > > > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > > > > > >> > > > > > json library since
> > /tools
> > > is
> > > > > > > written
> > > > > > > > > in
> > > > > > > > > > > java
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > > > > > >> > > > > > provided by a user.
> Can
> > we
> > > > > > quickly
> > > > > > > > > agree
> > > > > > > > > > > on
> > > > > > > > > > > > > what
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON,
> > whatever)?
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> > > > TopicMetadataRequest
> > > > > > v1
> > > > > > > > > still
> > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird
> > now
> > > > that
> > > > > > we
> > > > > > > > > have a
> > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought about
> how
> > > the
> > > > > new
> > > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in
> the
> > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> > > > ideally,
> > > > > > we
> > > > > > > > > don't
> > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger
> > auto
> > > > > topic
> > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > I agree, this strange
> > > logic
> > > > > > should
> > > > > > > > be
> > > > > > > > > > > fixed.
> > > > > > > > > > > > > > I'm not
> > > > > > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > > > > > >> > > > > > correct me if I'm
> wrong,
> > > but
> > > > > it
> > > > > > > > > doesn't
> > > > > > > > > > > look
> > > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > > > > > >> > > > > > leverage AdminClient
> for
> > > > that
> > > > > in
> > > > > > > > > > Producer
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > > > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant
> > > getting
> > > > > rid
> > > > > > > of
> > > > > > > > > > scala
> > > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > > HeartbeatRequestAndHeader
> > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing
> > when
> > > > > > adding
> > > > > > > > the
> > > > > > > > > > new
> > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the long
> term
> > > > plan
> > > > > is
> > > > > > > to
> > > > > > > > > get
> > > > > > > > > > > rid
> > > > > > > > > > > > of
> > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> request/response
> > in
> > > > the
> > > > > > > > client.
> > > > > > > > > > > Since
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant number
> of
> > > new
> > > > > > > > requests,
> > > > > > > > > > > > perhaps
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the
> existing
> > > > scala
> > > > > > > > requests
> > > > > > > > > > > first
> > > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks like I
> > > > > misunderstood
> > > > > > > the
> > > > > > > > > > point
> > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > > > > > >> > > > > > rework that. The only
> > > thing
> > > > is
> > > > > > > that
> > > > > > > > I
> > > > > > > > > > > don't
> > > > > > > > > > > > > see
> > > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > > > > > >> > > > > > least one existing
> > > protocol
> > > > > > > message.
> > > > > > > > > > Thus,
> > > > > > > > > > > > as
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > Re porting all
> existing
> > > > RQ/RP
> > > > > in
> > > > > > > > this
> > > > > > > > > > > patch.
> > > > > > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > > > > > > >> > > > > > requirement to have
> > Admin
> > > > KIP
> > > > > > > done,
> > > > > > > > > I'm
> > > > > > > > > > > > afraid
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > There are 13 protocol
> > > > messages
> > > > > > and
> > > > > > > > all
> > > > > > > > > > > that
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > > > > > > >> > > > > > intensive manual
> > testing,
> > > > no?
> > > > > > I'm
> > > > > > > > > afraid
> > > > > > > > > > > I'm
> > > > > > > > > > > > > > not the
> > > > > > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka core internals
> :).
> > > Let
> > > > > me
> > > > > > > know
> > > > > > > > > > your
> > > > > > > > > > > > > > thoughts
> > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015
> at
> > > 6:40
> > > > > AM,
> > > > > > > Jun
> > > > > > > > > > Rao <
> > > > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > 100. There are a few
> > > > fields
> > > > > > such
> > > > > > > > as
> > > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> > PartitionsSerialized
> > > > > that
> > > > > > > are
> > > > > > > > > > > > > represented
> > > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > composite structures
> > in
> > > > > json.
> > > > > > > > Could
> > > > > > > > > we
> > > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition
> as
> > > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> > > > > TopicMetadataRequest
> > > > > > > v1
> > > > > > > > > > still
> > > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird
> > now
> > > > that
> > > > > > we
> > > > > > > > > have a
> > > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought about
> how
> > > the
> > > > > new
> > > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in
> the
> > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> > > > ideally,
> > > > > > we
> > > > > > > > > don't
> > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger
> > auto
> > > > > topic
> > > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant
> > > > getting
> > > > > > rid
> > > > > > > > of
> > > > > > > > > > > scala
> > > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > > HeartbeatRequestAndHeader
> > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing
> > when
> > > > > > adding
> > > > > > > > the
> > > > > > > > > > new
> > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the long
> term
> > > > plan
> > > > > is
> > > > > > > to
> > > > > > > > > get
> > > > > > > > > > > rid
> > > > > > > > > > > > of
> > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > java
> request/response
> > in
> > > > the
> > > > > > > > client.
> > > > > > > > > > > Since
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > significant number
> of
> > > new
> > > > > > > > requests,
> > > > > > > > > > > > perhaps
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the
> existing
> > > > scala
> > > > > > > > requests
> > > > > > > > > > > first
> > > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015
> > at
> > > > 3:37
> > > > > > PM,
> > > > > > > > > > Andrii
> > > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > andrii.biletskyi@stealth.ly
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > As said above - I
> > list
> > > > > again
> > > > > > > all
> > > > > > > > > > > > comments
> > > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can see what's
> left
> > > and
> > > > > > > finalize
> > > > > > > > > all
> > > > > > > > > > > > > pending
> > > > > > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is much
> > needed
> > > > > > > > > > functionality,
> > > > > > > > > > > > but
> > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > really think these
> > > > > protocols
> > > > > > > > > > through.
> > > > > > > > > > > We
> > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > of well
> thought-out,
> > > > > > > orthoganol
> > > > > > > > > > apis.
> > > > > > > > > > > > For
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > important to think
> > > > through
> > > > > > the
> > > > > > > > end
> > > > > > > > > > > state
> > > > > > > > > > > > > > even if
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > won't implement in
> > the
> > > > > first
> > > > > > > > > phase.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Definitely
> behind
> > > > this.
> > > > > > > Would
> > > > > > > > > > > > > appreciate
> > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > how this can be
> > > > improved.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's please
> > please
> > > > > > please
> > > > > > > > wait
> > > > > > > > > > > until
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to the new java
> > > protocol
> > > > > > > > > > definitions.
> > > > > > > > > > > If
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > objects that is
> just
> > > > > > > generating
> > > > > > > > > more
> > > > > > > > > > > > work
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> > latest
> > > > > > patch -
> > > > > > > > > > removed
> > > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This proposal
> > > > > introduces
> > > > > > a
> > > > > > > > new
> > > > > > > > > > type
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > inconsistent with
> > > > > everything
> > > > > > > > else
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > other marker
> value.
> > > You
> > > > > > could
> > > > > > > > > argue
> > > > > > > > > > > > either
> > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > for consistency.
> For
> > > > > clients
> > > > > > > > that
> > > > > > > > > > > > > > implemented
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than our scala
> code
> > > > these
> > > > > > > basic
> > > > > > > > > > > > primitives
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> > latest
> > > > > > patch -
> > > > > > > > > > removed
> > > > > > > > > > > > > > MaybeOf
> > > > > > > > > > > > > > > > > type
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 4.
> ClusterMetadata:
> > > This
> > > > > > seems
> > > > > > > > to
> > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers, topics,
> and
> > > > > > > > partitions. I
> > > > > > > > > > > think
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > ClusterMetadataRequest
> > > > (or
> > > > > > > just
> > > > > > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > controller. Or are
> > > there
> > > > > > other
> > > > > > > > > > things
> > > > > > > > > > > we
> > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I agree.
> Updated
> > > the
> > > > > KIP.
> > > > > > > > Let's
> > > > > > > > > > > > extends
> > > > > > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > include
> controller.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a
> > tendency
> > > to
> > > > > try
> > > > > > > to
> > > > > > > > > > make a
> > > > > > > > > > > > lot
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > particular nodes.
> > This
> > > > > adds
> > > > > > a
> > > > > > > > lot
> > > > > > > > > of
> > > > > > > > > > > > > burden
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > sounds easy but
> each
> > > > > > discovery
> > > > > > > > can
> > > > > > > > > > > fail
> > > > > > > > > > > > in
> > > > > > > > > > > > > > many
> > > > > > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > full state machine
> > to
> > > do
> > > > > > > > right). I
> > > > > > > > > > > think
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > commands and
> ideally
> > > as
> > > > > many
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > > > other
> > > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers and just
> > > > redirect
> > > > > to
> > > > > > > the
> > > > > > > > > > > > > controller
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > there would be a
> > > general
> > > > > way
> > > > > > > to
> > > > > > > > > > > > > encapsulate
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a very
> > > > interesting
> > > > > > > idea,
> > > > > > > > > but
> > > > > > > > > > > > seems
> > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > feature (like
> > > > performance
> > > > > > > > > > > > considerations,
> > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I believe this
> > > shouldn't
> > > > > be
> > > > > > a
> > > > > > > > > > blocker.
> > > > > > > > > > > > If
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point it won't
> > affect
> > > > > Admin
> > > > > > > > > changes
> > > > > > > > > > -
> > > > > > > > > > > at
> > > > > > > > > > > > > > least
> > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We should
> > probably
> > > > > > > normalize
> > > > > > > > > the
> > > > > > > > > > > key
> > > > > > > > > > > > > > value
> > > > > > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than embedding a
> new
> > > > > > > formatting.
> > > > > > > > > So
> > > > > > > > > > > two
> > > > > > > > > > > > > > strings
> > > > > > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > internal equals
> > sign.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> > latest
> > > > > > patch -
> > > > > > > > > > > > normalized
> > > > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the
> > > postcondition
> > > > of
> > > > > > > these
> > > > > > > > > > APIs
> > > > > > > > > > > > that
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the command has
> been
> > > > > > > completed?
> > > > > > > > It
> > > > > > > > > > is
> > > > > > > > > > > a
> > > > > > > > > > > > > lot
> > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > been completed so
> > you
> > > > know
> > > > > > > that
> > > > > > > > if
> > > > > > > > > > you
> > > > > > > > > > > > > > create a
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it you won't get
> an
> > > > > > exception
> > > > > > > > > about
> > > > > > > > > > > > there
> > > > > > > > > > > > > > being
> > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: For long
> running
> > > > > requests
> > > > > > > > (like
> > > > > > > > > > > > > reassign
> > > > > > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command has begun
> -
> > so
> > > > we
> > > > > > > don't
> > > > > > > > > > block
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > client.
> > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic commands,
> this
> > > > will
> > > > > be
> > > > > > > > > > > refactored
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > immediately, since
> > the
> > > > > > > > Controller
> > > > > > > > > > will
> > > > > > > > > > > > > serve
> > > > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up ticket
> > > > > > KAFKA-1777).
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe topic
> > and
> > > > list
> > > > > > > > topics
> > > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request. Is there
> a
> > > > reason
> > > > > > to
> > > > > > > > give
> > > > > > > > > > > back
> > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like if we just
> make
> > > the
> > > > > > > > > > > post-condition
> > > > > > > > > > > > of
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic is deleted
> > that
> > > > will
> > > > > > get
> > > > > > > > rid
> > > > > > > > > > of
> > > > > > > > > > > > the
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be much more
> > > intuitive.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> > latest
> > > > > > patch -
> > > > > > > > > > removed
> > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we
> > consider
> > > > > > batching
> > > > > > > > > these
> > > > > > > > > > > > > > requests?
> > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > allow multiple
> > > > operations
> > > > > to
> > > > > > > be
> > > > > > > > > > > batched.
> > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we will get a lot
> of
> > > > code
> > > > > > that
> > > > > > > > > does
> > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this code will
> work
> > > > great
> > > > > > when
> > > > > > > > you
> > > > > > > > > > > test
> > > > > > > > > > > > > on 5
> > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the
> KIP -
> > > > > please
> > > > > > > > check
> > > > > > > > > > > "Topic
> > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I think we
> > should
> > > > also
> > > > > > > > discuss
> > > > > > > > > > how
> > > > > > > > > > > > we
> > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > client api for
> these
> > > > > > > operations.
> > > > > > > > > > > > Currently
> > > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > is totally
> sketchy.
> > I
> > > > > think
> > > > > > we
> > > > > > > > > > > probably
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > that exposes
> > > > > administrative
> > > > > > > > > > > > functionality.
> > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > properly test the
> > new
> > > > > apis,
> > > > > > I
> > > > > > > > > > suspect.
> > > > > > > > > > > > We
> > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the
> KIP -
> > > > > please
> > > > > > > > check
> > > > > > > > > > > "Admin
> > > > > > > > > > > > > > Client"
> > > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The other
> > > > information
> > > > > > that
> > > > > > > > > would
> > > > > > > > > > > be
> > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > information about
> > > > > > > > partitions--how
> > > > > > > > > > much
> > > > > > > > > > > > > data
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the segment
> offsets,
> > > > what
> > > > > is
> > > > > > > the
> > > > > > > > > > > log-end
> > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the compaction
> > point,
> > > > > etc. I
> > > > > > > > think
> > > > > > > > > > > that
> > > > > > > > > > > > > done
> > > > > > > > > > > > > > > > right
> > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > successor to the
> > very
> > > > > > awkward
> > > > > > > > > > > > > OffsetRequest
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > be resolved in a
> > > > separate
> > > > > > KIP
> > > > > > > /
> > > > > > > > > jira
> > > > > > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 12. Generally we
> can
> > > do
> > > > > good
> > > > > > > > error
> > > > > > > > > > > > > handling
> > > > > > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > messages. I.e.
> > > generally
> > > > > the
> > > > > > > > > client
> > > > > > > > > > > has
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > context
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > an error that the
> > > topic
> > > > > > > doesn't
> > > > > > > > > > exist
> > > > > > > > > > > to
> > > > > > > > > > > > > say
> > > > > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > than "error code
> 14"
> > > (or
> > > > > > > > > whatever).
> > > > > > > > > > > > Maybe
> > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this is hard? If
> we
> > > want
> > > > > to
> > > > > > > add
> > > > > > > > > > > > > server-side
> > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > need to do this
> in a
> > > > > > > consistent
> > > > > > > > > way
> > > > > > > > > > > > across
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the
> KIP -
> > > > > please
> > > > > > > > check
> > > > > > > > > > > > > "Protocol
> > > > > > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > comprehensive,
> > > > > fine-grained
> > > > > > > list
> > > > > > > > > of
> > > > > > > > > > > > error
> > > > > > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from
> > > Guozhang:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe topic
> > > > > request:
> > > > > > it
> > > > > > > > > would
> > > > > > > > > > > be
> > > > > > > > > > > > > > great to
> > > > > > > > > > > > > > > > > go
> > > > > > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex
> for
> > > > this
> > > > > > > > request.
> > > > > > > > > > For
> > > > > > > > > > > > > > example,
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the topic command
> is
> > > to
> > > > > list
> > > > > > > all
> > > > > > > > > > > topics
> > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex
> > then
> > > we
> > > > > > have
> > > > > > > to
> > > > > > > > > > first
> > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > description info
> and
> > > > then
> > > > > > > filter
> > > > > > > > > at
> > > > > > > > > > > the
> > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in
> > > > create
> > > > > > > topic:
> > > > > > > > > > this
> > > > > > > > > > > is
> > > > > > > > > > > > > > related
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > maybe we can add
> > > another
> > > > > > > > metadata
> > > > > > > > > > K-V
> > > > > > > > > > > or
> > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > with config K-V in
> > > > create
> > > > > > > topic
> > > > > > > > > like
> > > > > > > > > > > we
> > > > > > > > > > > > > did
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > field can be quite
> > > > useful
> > > > > in
> > > > > > > > > storing
> > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > who issue the
> create
> > > > > > command,
> > > > > > > > etc,
> > > > > > > > > > > which
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant
> > setting.
> > > > Then
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > describe
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > on regex of the
> > > metadata
> > > > > > > field.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: As discussed it
> > is
> > > > very
> > > > > > > > > > interesting
> > > > > > > > > > > > but
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > we have some basic
> > > > > > > functionality
> > > > > > > > > > > there.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today all the
> > > admin
> > > > > > > > operations
> > > > > > > > > > are
> > > > > > > > > > > > > > async in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > return once it is
> > > > written
> > > > > in
> > > > > > > ZK,
> > > > > > > > > and
> > > > > > > > > > > > that
> > > > > > > > > > > > > > is why
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > > > > > testUtil.waitForTopicCreated()
> > > > > > > > > > /
> > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > request, etc. With
> > > admin
> > > > > > > > requests
> > > > > > > > > we
> > > > > > > > > > > > could
> > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > synchronous
> > requests;
> > > > when
> > > > > > it
> > > > > > > is
> > > > > > > > > > > turned
> > > > > > > > > > > > > on,
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > until the request
> > has
> > > > been
> > > > > > > > > > completed.
> > > > > > > > > > > > And
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > "token" field in
> the
> > > > > > response,
> > > > > > > > and
> > > > > > > > > > > then
> > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > verification
> > request"
> > > > with
> > > > > > the
> > > > > > > > > given
> > > > > > > > > > > > token
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > has been
> completed.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I see your
> point.
> > > My
> > > > > idea
> > > > > > > was
> > > > > > > > > to
> > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long running
> > request,
> > > > > where
> > > > > > > > > needed.
> > > > > > > > > > We
> > > > > > > > > > > > can
> > > > > > > > > > > > > > do it
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > concern is that
> > > > > introducing
> > > > > > a
> > > > > > > > > token
> > > > > > > > > > we
> > > > > > > > > > > > > again
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > to do similar
> thing
> > > > > > > introducing
> > > > > > > > > > single
> > > > > > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > this idea because
> we
> > > > > wanted
> > > > > > to
> > > > > > > > > have
> > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > a) have fixed
> schema
> > > but
> > > > > > > > introduce
> > > > > > > > > > > each
> > > > > > > > > > > > > > time new
> > > > > > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > long-running
> > requests
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > b) use one request
> > for
> > > > > > > > > verification
> > > > > > > > > > > but
> > > > > > > > > > > > > > > > generalize
> > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine with
> > whatever
> > > > > > > decision
> > > > > > > > > > > > community
> > > > > > > > > > > > > > come
> > > > > > > > > > > > > > > > to.
> > > > > > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > 16. Specifically
> for
> > > > > > > ownership,
> > > > > > > > I
> > > > > > > > > > > think
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > like you are
> > > describing
> > > > > ACL)
> > > > > > > via
> > > > > > > > > an
> > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > I remember KIP-11
> > > > > described
> > > > > > > > this,
> > > > > > > > > > but
> > > > > > > > > > > I
> > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no
> problem.
> > > Not
> > > > > > sure
> > > > > > > > > though
> > > > > > > > > > > how
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be committed
> > > first
> > > > > and
> > > > > > > > > include
> > > > > > > > > > > > > changes
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I added
> this
> > > > note
> > > > > to
> > > > > > > > "Open
> > > > > > > > > > > > > > Questions"
> > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13,
> 2015
> > > at
> > > > > > 12:34
> > > > > > > > AM,
> > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > andrii.biletskyi@stealth.ly
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Today I uploaded
> > the
> > > > > patch
> > > > > > > > that
> > > > > > > > > > > covers
> > > > > > > > > > > > > > some of
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - removed
> MaybeOf
> > > > > optional
> > > > > > > > type
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - switched to
> java
> > > > > > protocol
> > > > > > > > > > > > definitions
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - simplified
> > > messages
> > > > > > > > > (normalized
> > > > > > > > > > > > > configs,
> > > > > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I also updated
> the
> > > > KIP-4
> > > > > > > with
> > > > > > > > > > > > respective
> > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin
> > > > Operations
> > > > > > ->
> > > > > > > > > > updated
> > > > > > > > > > > > Wire
> > > > > > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove
> > > > ClusterMetadata
> > > > > > ->
> > > > > > > > > > changed
> > > > > > > > > > > to
> > > > > > > > > > > > > > extend
> > > > > > > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin Client
> ->
> > > > > updated
> > > > > > my
> > > > > > > > > > initial
> > > > > > > > > > > > > > proposal
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Error codes ->
> > > > > proposed
> > > > > > > > > > > fine-grained
> > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> AdminRequestFailed
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I will also
> send a
> > > > > > separate
> > > > > > > > > email
> > > > > > > > > > to
> > > > > > > > > > > > > > cover all
> > > > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12,
> > 2015
> > > > at
> > > > > > 9:26
> > > > > > > > PM,
> > > > > > > > > > > Gwen
> > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> It actually
> > > specifies
> > > > > > > changes
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are
> > > > > consistent
> > > > > > in
> > > > > > > > > this
> > > > > > > > > > > > regard
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12,
> > > 2015
> > > > at
> > > > > > > 12:21
> > > > > > > > > PM,
> > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Specifically
> > for
> > > > > > > > ownership, I
> > > > > > > > > > > think
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > like you are
> > > > > describing
> > > > > > > > ACL)
> > > > > > > > > > via
> > > > > > > > > > > an
> > > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I remember
> > KIP-11
> > > > > > > described
> > > > > > > > > > this,
> > > > > > > > > > > > > but I
> > > > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I
> > > think
> > > > > > KIP-4
> > > > > > > > > > focuses
> > > > > > > > > > > > on
> > > > > > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > exists from
> > Kafka
> > > > > > > brokers,
> > > > > > > > > not
> > > > > > > > > > on
> > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > should exist
> > but
> > > > > > doesn't
> > > > > > > > yet?
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar
> 12,
> > > > 2015
> > > > > at
> > > > > > > > 6:37
> > > > > > > > > > AM,
> > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just want to
> > > > > > elaborate a
> > > > > > > > bit
> > > > > > > > > > > more
> > > > > > > > > > > > on
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> describe-topic
> > > > based
> > > > > > on
> > > > > > > > > > config /
> > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on
> KAFKA-1694.
> > > The
> > > > > > main
> > > > > > > > > > > motivation
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms,
> > > which
> > > > I
> > > > > > > think
> > > > > > > > is
> > > > > > > > > > > quite
> > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> architecture:
> > > > today
> > > > > > > anyone
> > > > > > > > > can
> > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> there is no
> > > > concept
> > > > > or
> > > > > > > > > > > "ownership"
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> users. For
> > > > example,
> > > > > at
> > > > > > > > > > LinkedIn
> > > > > > > > > > > we
> > > > > > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some casual
> > > topic
> > > > > name
> > > > > > > > > prefix,
> > > > > > > > > > > > which
> > > > > > > > > > > > > > is a
> > > > > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale our
> > > > > > customers.
> > > > > > > It
> > > > > > > > > > would
> > > > > > > > > > > > be
> > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > > > topics
> > > > > > that
> > > > > > > > is
> > > > > > > > > > > > created
> > > > > > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > > > topics
> > > > > > > whose
> > > > > > > > > > > > retention
> > > > > > > > > > > > > > time
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > > > topics
> > > > > > > whose
> > > > > > > > > > > writable
> > > > > > > > > > > > > > group
> > > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > authorization),
> > > > > etc..
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One possible
> > way
> > > > to
> > > > > > > > achieve
> > > > > > > > > > this
> > > > > > > > > > > > is
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> create-topic
> > > > > request,
> > > > > > > > whose
> > > > > > > > > > > value
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic; then
> > > > > > > > describe-topics
> > > > > > > > > > can
> > > > > > > > > > > > > > choose to
> > > > > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2)
> > config
> > > > K-V
> > > > > > > > > matching,
> > > > > > > > > > > 3)
> > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar
> 5,
> > > > 2015
> > > > > at
> > > > > > > > 4:37
> > > > > > > > > > PM,
> > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for
> > the
> > > > > > updated
> > > > > > > > > wiki.
> > > > > > > > > > A
> > > > > > > > > > > > few
> > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error
> > > > > description
> > > > > > in
> > > > > > > > > > > > response: I
> > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> several
> > > different
> > > > > > error
> > > > > > > > > cases
> > > > > > > > > > > > then
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In
> > > general
> > > > > the
> > > > > > > > > > errorCode
> > > > > > > > > > > > > > itself
> > > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> describing
> > the
> > > > > server
> > > > > > > > side
> > > > > > > > > > > > errors.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe
> > > topic
> > > > > > > > request:
> > > > > > > > > it
> > > > > > > > > > > > would
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name
> > > regex
> > > > > for
> > > > > > > this
> > > > > > > > > > > > request.
> > > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic
> > > command
> > > > > is
> > > > > > to
> > > > > > > > > list
> > > > > > > > > > > all
> > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name
> > > regex
> > > > > then
> > > > > > > we
> > > > > > > > > have
> > > > > > > > > > > to
> > > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> description
> > > info
> > > > > and
> > > > > > > then
> > > > > > > > > > > filter
> > > > > > > > > > > > at
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on
> ZK.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config
> > K-Vs
> > > in
> > > > > > > create
> > > > > > > > > > topic:
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we
> can
> > > add
> > > > > > > another
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > K-V or
> > > > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with config
> > K-V
> > > > in
> > > > > > > create
> > > > > > > > > > topic
> > > > > > > > > > > > > like
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field can
> be
> > > > quite
> > > > > > > useful
> > > > > > > > > in
> > > > > > > > > > > > > storing
> > > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue
> the
> > > > > create
> > > > > > > > > command,
> > > > > > > > > > > > etc,
> > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> multi-tenant
> > > > > setting.
> > > > > > > > Then
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of
> > the
> > > > > > > metadata
> > > > > > > > > > field.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today
> all
> > > the
> > > > > > admin
> > > > > > > > > > > operations
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> return once
> > it
> > > is
> > > > > > > written
> > > > > > > > > in
> > > > > > > > > > > ZK,
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> request,
> etc.
> > > > With
> > > > > > > admin
> > > > > > > > > > > requests
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous
> > > > > requests;
> > > > > > > > when
> > > > > > > > > it
> > > > > > > > > > > is
> > > > > > > > > > > > > > turned
> > > > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until the
> > > request
> > > > > has
> > > > > > > > been
> > > > > > > > > > > > > > completed. And
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token"
> field
> > > in
> > > > > the
> > > > > > > > > > response,
> > > > > > > > > > > > and
> > > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> verification
> > > > > request"
> > > > > > > > with
> > > > > > > > > > the
> > > > > > > > > > > > > given
> > > > > > > > > > > > > > > > token
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has been
> > > > completed.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for
> > > > extending
> > > > > > > > > Metadata
> > > > > > > > > > > > > request
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> information,
> > > and
> > > > > then
> > > > > > > we
> > > > > > > > > can
> > > > > > > > > > > > remove
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar
> > 3,
> > > > 2015
> > > > > > at
> > > > > > > > > 10:23
> > > > > > > > > > > AM,
> > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for
> > > > sending
> > > > > > > that
> > > > > > > > > out
> > > > > > > > > > > Joe
> > > > > > > > > > > > -
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it today,
> so
> > > if
> > > > > > notes
> > > > > > > > can
> > > > > > > > > be
> > > > > > > > > > > > sent
> > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon,
> Mar
> > > 02,
> > > > > 2015
> > > > > > > at
> > > > > > > > > > > > 09:16:13AM
> > > > > > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks
> for
> > > > > sending
> > > > > > > > this
> > > > > > > > > > out
> > > > > > > > > > > > Joe.
> > > > > > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone
> :)
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon,
> > Mar
> > > 2,
> > > > > > 2015
> > > > > > > at
> > > > > > > > > > 6:46
> > > > > > > > > > > > AM,
> > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I
> > > just
> > > > > sent
> > > > > > > > out a
> > > > > > > > > > > > google
> > > > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > everyone I
> > > > > found
> > > > > > > > > working
> > > > > > > > > > > on
> > > > > > > > > > > > a
> > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me
> > > know
> > > > > and
> > > > > > > can
> > > > > > > > > > update
> > > > > > > > > > > > it,
> > > > > > > > > > > > > > np.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We
> > should
> > > do
> > > > > > this
> > > > > > > > > every
> > > > > > > > > > > > > Tuesday
> > > > > > > > > > > > > > @
> > > > > > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help
> to
> > > > make a
> > > > > > > > google
> > > > > > > > > > > > account
> > > > > > > > > > > > > > so we
> > > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To
> > discuss
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in
> > > progress
> > > > > and
> > > > > > > > > related
> > > > > > > > > > > JIRA
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe
> > > Stein
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On
> Tue,
> > > Feb
> > > > > 24,
> > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > 2:59
> > > > > > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> jay.kreps@gmail.com
> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's
> > > stay
> > > > on
> > > > > > > > Google
> > > > > > > > > > > > hangouts
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > available
> > > > on
> > > > > > > > youtube.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On
> Tue,
> > > Feb
> > > > > 24,
> > > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > > 11:49
> > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > jholoman@cloudera.com
> > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> Jay /
> > > Joe
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> We're
> > > > happy
> > > > > > to
> > > > > > > > send
> > > > > > > > > > > out a
> > > > > > > > > > > > > > Webex
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > sessions
> > > > if
> > > > > > > there
> > > > > > > > > is
> > > > > > > > > > > > > > interest and
> > > > > > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > Thanks
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> Jeff
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On
> > Tue,
> > > > Feb
> > > > > > 24,
> > > > > > > > > 2015
> > > > > > > > > > at
> > > > > > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > Let's
> > > > try
> > > > > > to
> > > > > > > > get
> > > > > > > > > > the
> > > > > > > > > > > > > > technical
> > > > > > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> think
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > there
> > > > is
> > > > > > some
> > > > > > > > > > benefit
> > > > > > > > > > > > to
> > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful
> that
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> we
> > > post
> > > > > > > > > > instructions
> > > > > > > > > > > > and
> > > > > > > > > > > > > > give
> > > > > > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > working.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > Tuesday
> > > > > at
> > > > > > > that
> > > > > > > > > > time
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > work
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > -Jay
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> On
> > > Tue,
> > > > > Feb
> > > > > > > 24,
> > > > > > > > > > 2015
> > > > > > > > > > > at
> > > > > > > > > > > > > > 8:18
> > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > Weekly
> > > > > > > would
> > > > > > > > be
> > > > > > > > > > > great
> > > > > > > > > > > > > > maybe
> > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > I
> > > > don't
> > > > > > > mind
> > > > > > > > > > google
> > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > whatever
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > we
> > > > know
> > > > > > the
> > > > > > > > > > apache
> > > > > > > > > > > > irc
> > > > > > > > > > > > > > > > channel
> > > > > > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > goes?
> > > > > We
> > > > > > > can
> > > > > > > > > pull
> > > > > > > > > > > > > > transcripts
> > > > > > > > > > > > > > > > > too
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > makes
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > it
> > > > > > helpful
> > > > > > > > for
> > > > > > > > > > > > things.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > ~
> > > > > > Joestein
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > On
> > > > Tue,
> > > > > > Feb
> > > > > > > > 24,
> > > > > > > > > > > 2015
> > > > > > > > > > > > at
> > > > > > > > > > > > > > 11:10
> > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > jay.kreps@gmail.com
> > > > > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > We'd
> > > > > > > talked
> > > > > > > > > > about
> > > > > > > > > > > > > > doing a
> > > > > > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> about
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > > > > > generalizing
> > > > > > > > > > > that a
> > > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> good
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > for
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > > > everyone
> > > > > > > > > > > spending a
> > > > > > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > maybe
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > sync
> > > > > up
> > > > > > > > once
> > > > > > > > > a
> > > > > > > > > > > > week.
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> design
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > > stuff,
> > > > > > > make
> > > > > > > > > > sure
> > > > > > > > > > > we
> > > > > > > > > > > > > > are on
> > > > > > > > > > > > > > > > > top
> > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > tricky
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > > issues,
> > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > We
> > > > > can
> > > > > > > make
> > > > > > > > > it
> > > > > > > > > > > > > publicly
> > > > > > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > > likes.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > Any
> > > > > > > > interest
> > > > > > > > > in
> > > > > > > > > > > > doing
> > > > > > > > > > > > > > this?
> > > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > week.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > -Jay
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > On
> > > > > Tue,
> > > > > > > Feb
> > > > > > > > > 24,
> > > > > > > > > > > > 2015
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > >
> > > > Hi
> > > > > > all,
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > >
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > >
> > > > > I've
> > > > > > > > > updated
> > > > > > > > > > > KIP
> > > > > > > > > > > > > > page,
> > > > > > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > added
> > > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > >
> > > > > some
> > > > > > > > > > > >
> > > > > > > > > > > ...
> > > > > > > > > > >
> > > > > > > > > > > [Message clipped]
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > --
> > > > > > > > > -- Guozhang
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
Andrii,

I looked at the latest wiki for this KIP. I have a few more comments.

110. In CreateTopicResponse, AlterTopicResponse, DeleteTopicResponse and
DescribeTopicResponse, we probably don't need the global error code. Our
current convention is to just use the same global error code in each topic.
This simplifies the error checking on the client side.

111. DescribeTopicResponse:
111.1 Our protocol definition doesn't support optional fields. So, we can't
make Leader an optional field.
111.2 Do we really need the field Partitions and ReplicationFactor in
TopicConfigDetails?
The former is basically the size of the TopicPartitionDetails array and the
latter can be found from the size of the Replica array.
111.3 Do we need TopicName in TopicDescription since it's already in
DescribeTopicResponse?
111.4 Finally, DescribeTopicResponse is very similar to
TopicMetadataResponse. So, it probably makes sense just to keep one of them
in the future. Should we just use DescribeTopicRequest/Response to replace
TopicMetadataRequest/Response in the producer/consumer client in the future
(since DescribeTopicRequest doesn't trigger auto topic creation)? If so, we
will need to add the broker list in DescribeTopicResponse.

112. Thinking about this a bit more, I don't think we need a separate
VerifyReassignPartitionRequest/Response.
We can just use DescribeTopic to get the assigned replicas and check if
they match the target replica assignment in the client. The controller
propagates the metadata change after the reassignment completes for each
partition.

113. ReassignPartitionRequest: For consistency, we probably want to nest
the partition data under topic. So instead of
  ReassignPartitionRequest => [Topic PartitionId [ReplicaId]]
we can have sth like
  ReassignPartitionRequest => [Topic [PartitionId [ReplicaId]]]
The error code will then be per topic.

114. PreferredReplicaLeaderElectionRequest: Same as the above. Instead of
  PreferredReplicaLeaderElectionRequest => [Topic PartitionId]
we can have
  PreferredReplicaLeaderElectionRequest => [Topic [PartitionId]]
Again, the error code will be per topic.

115. ErrorCode:
115.1 NotControllerReceivedAdminRequest,
PreferredReplicaLeaderElectionInProgress
and ReassignPartitionsInProgress are likely not needed any more.
115.2 We probably need to add UnknownTopicOrPartitionCode since it's a
valid error code for some of the requests (e.g. describe, alter, etc) when
a non-existing topic is specified.


Previous comments.
100. ReplicaAssignment in CreateTopicRequest and AlterTopicRequest are
still a json string. We need to flatten them.


Thanks,

Jun


On Sat, Mar 21, 2015 at 2:03 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Guozhang,
>
> I'm not sure I understand how can we use those tools in CLI.
> First of all, those are *Test*Utils, e.g. waitUntilMetadataIsPropagated
> (which might be very useful for us) requires servers:Seq[KafkaServer]
> as an argument:
>
> def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String,
> partition: Int, timeout: Long = 5000L): Int
>
> how can we have access it at runtime in one of the brokers?
>
> Secondly, how can user directly call these tools if zookeeper might be
> not accessible (which is used in the tools, right?) at all - e.g. behind
> the VPC
> in AWS. Furthermore, I think with KIP-4 we are trying to create an
> abstraction and
> a single point of zookeeper interactions, to eliminate direct calls to ZK
> either to get
> some cluster information or to change something.
>
> Thanks,
> Andrii Biletskyi
>
> On Sat, Mar 21, 2015 at 3:37 AM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Andrii,
> >
> > Actually the checking logic Jun mentioned is already implemented as
> > TestUtils.waitUntilXXX (LeaderIsElected, MetadataIsPropagated, etc...) I
> > think we can extend these functions as CLI tools like TopicCommand so
> that
> > users re-implementing such endpoint can directly call something like
> > java.tools.WaitUntilXXX (of course this requires them to have javac
> > installed, which should be a reasonable requirement?)
> >
> > Guozhang
> >
> >
> >
> > On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Jun,
> > >
> > > Not that I was saying we need to make requests blocking on server,
> > > it was just to emphasize that with async requests a client
> > implementations
> > > may be a little bit more than just issue request - get the response.
> > > Thanks for the explanation, I understand now that we can go with agreed
> > > solution though it may not be perfect.
> > > I believe this was one of the last controversial questions from the
> list.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Andrii,
> > > >
> > > > A few points.
> > > >
> > > > 1. Create/Alter can typically complete quickly. So, it's possible to
> > make
> > > > the request block until it's completed. However, currently, doing
> this
> > at
> > > > the broker is a bit involved. To make Create block, we will need to
> add
> > > > some callbacks in KafkaController. This is possible. However, the
> > > > controller logic currently is pretty completed. It would probably be
> > > better
> > > > if we clean it up first before adding more complexity to it. Alter is
> > > even
> > > > trickier. Adding partition is currently handled through
> > KafkaController.
> > > So
> > > > it can be dealt with in a similar way. However, Alter config is done
> > > > completely differently. It doesn't go through the controller.
> Instead,
> > > each
> > > > broker listens to ZooKeeper directly. So, it's not clear if there is
> an
> > > > easy way on the broker to figure out whether a config is applied on
> > every
> > > > broker.
> > > >
> > > > 2. Delete can potentially take long if a replica to be deleted is
> > > offline.
> > > > PreferredLeader/PartitionReassign can also take long. So, we can't
> > really
> > > > make those requests block on the broker.
> > > >
> > > > As you can see, at this moment it's not easy to make all admin
> requests
> > > > block on the broker. So, if we want the blocking feature in the admin
> > > > utility in the short term, doing the completion check at the admin
> > client
> > > > is probably an easier route, even though it may not be ideal.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Jun,
> > > > >
> > > > > I see your point. But wouldn't that lead to a "fat" client
> > > > implementations?
> > > > > Suppose someone would like to implement client for Admin Wire
> > protocol.
> > > > > Not only people will have to code quite complicated logic like
> "send
> > > > > describe
> > > > > request to each broker" (again state machin?) but it will also mean
> > > > people
> > > > > must understand internal kafka logic related to topic storage and
> how
> > > > > information is propageted from the controller to brokers.
> > > > > I see this like a dilemma between having a concise Wire Protocol
> and
> > > > > self-sufficient API to make client implementations simple.
> > > > > I don't have a win-win solution though.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > >
> > > > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > >
> > > > > > For 1), 2) and 3), blocking would probably mean that the new
> > metadata
> > > > is
> > > > > > propagated to every broker. To achieve that, the client can keep
> > > > issuing
> > > > > > the describe topic request to every broker until it sees the new
> > > > metadata
> > > > > > in the response.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Hm, actually the ticket you linked, Guozhang, brings as back
> > > > > > > to the problem what should be considered a post-condition for
> > > > > > > each of the admin commands.
> > > > > > > In my understanding:
> > > > > > >
> > > > > > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > > > > > (Not the controller picked up changes from zk and broadcasted
> > > > > > > LeaderAndIsr and UpdateMetadata)
> > > > > > >
> > > > > > > 2) AlterTopic - same as 1) - broker changed assignment data
> > > > > > > in zookeeper or created admin path for topic config change
> > > > > > >
> > > > > > > 3) DeleteTopic - admin path /admin/delete_topics is created
> > > > > > >
> > > > > > > 4) ReassignPartitions and PreferredReplica - corresponding
> admin
> > > > > > > path is created
> > > > > > >
> > > > > > > Now what can be considered a completed operation from the
> > client's
> > > > > > > perspective?
> > > > > > > 1) Topic is created once corresponding data is in zk
> > > > > > > (I remember there were some thoughts that it'd be good to
> > consider
> > > > > > > topic created once all replicas receive information about it
> and
> > > thus
> > > > > > > clients can produce/consume from it, but as was discussed this
> > > seems
> > > > > > > to be a hard thing to do)
> > > > > > >
> > > > > > > 2) Probably same as 1), so right after AlterTopic is issued
> > > > > > >
> > > > > > > 3) The topic has been removed from /brokers/topics
> > > > > > >
> > > > > > > 4) ReassignPartitions and PrefferedReplica were discussed
> > earlier -
> > > > > > > in short the former is completed once partition state info in
> zk
> > > > > matches
> > > > > > > reassignment request and admin path is empty, the latter - once
> > > data
> > > > > > > in zk shows that head of assignned replicas of the partition
> and
> > > > leader
> > > > > > > is the same replica
> > > > > > >
> > > > > > > Thoughts?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <
> > wangguoz@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > I think while loop is fine for supporting blocking, just that
> > we
> > > > need
> > > > > > to
> > > > > > > > add back off to avoid bombarding brokers with DescribeTopic
> > > > requests.
> > > > > > > >
> > > > > > > > Also I have linked KAFKA-1125
> > > > > > > > <https://issues.apache.org/jira/browse/KAFKA-1125> to your
> > > > proposal,
> > > > > > and
> > > > > > > > when KAFKA-1694 is done this ticket can also be closed.
> > > > > > > >
> > > > > > > > Guozhang
> > > > > > > >
> > > > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >
> > > > > > > > > Great.
> > > > > > > > > I want to elaborate this a bit more, to see we are on the
> > same
> > > > page
> > > > > > > > > concerning the client code.
> > > > > > > > >
> > > > > > > > > So with all topic commands being async a client
> (AdminClient
> > in
> > > > our
> > > > > > > > > case or any other other client people would like to
> > implement)
> > > to
> > > > > > > support
> > > > > > > > > a blocking operation (which seems to be a natural use-case
> > e.g.
> > > > for
> > > > > > > topic
> > > > > > > > > creation): would have to do:
> > > > > > > > > 1. issue CreateTopicRequest
> > > > > > > > > 2. if successful, in a "while" loop send
> DescribeTopicRequest
> > > and
> > > > > > > > > break the loop once all topics are returned in response (or
> > > upon
> > > > > > > > timeout).
> > > > > > > > > 3. if unsuccessful throw exception
> > > > > > > > > Would it be okay?
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Andrii Biletskyi
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Andrii,
> > > > > > > > > >
> > > > > > > > > > I think you are right. It seems that only
> > ReassignPartitions
> > > > > needs
> > > > > > a
> > > > > > > > > > separate verification request.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > >
> > > > > > > > > > > Guys,
> > > > > > > > > > > I like this idea too. Let's stick with that. I'll
> update
> > > KIP
> > > > > > > > > accordingly.
> > > > > > > > > > >
> > > > > > > > > > > I was also thinking we can avoid adding dedicated
> status
> > > > check
> > > > > > > > > > > requests for topic commands. - We have everything in
> > > > > > DescribeTopic
> > > > > > > > > > > for that! E.g.:
> > > > > > > > > > > User issued CreateTopic - to check the status client
> > sends
> > > > > > > > > DescribeTopic
> > > > > > > > > > > and checks whether is something returned for that
> topic.
> > > The
> > > > > same
> > > > > > > for
> > > > > > > > > > > alteration, deletion.
> > > > > > > > > > > Btw, PreferredReplica status can be also checked with
> > > > > > > > > > DescribeTopicRequest
> > > > > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > > > > For ReassignPartitions as discussed we'll need to have
> a
> > > > > separate
> > > > > > > > > > Verify...
> > > > > > > > > > > request.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> > > > > > wangguoz@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > +1 on broker writing to ZK for async handling. I was
> > > > thinking
> > > > > > > that
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > end state the admin requests would be eventually sent
> > to
> > > > > > > controller
> > > > > > > > > > > either
> > > > > > > > > > > > through re-routing or clients discovering them,
> instead
> > > of
> > > > > > > letting
> > > > > > > > > > > > controller listen on ZK admin path. But thinking
> about
> > > it a
> > > > > > > second
> > > > > > > > > > time,
> > > > > > > > > > > I
> > > > > > > > > > > > think it is actually simpler to let controller manage
> > > > > > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > > > > > >
> > > > > > > > > > > > Guozhang
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > > > > > jjkoshy.w@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > +1 as well. I think it helps to keep the rerouting
> > > > approach
> > > > > > > > > > orthogonal
> > > > > > > > > > > > > to this KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps
> > > > wrote:
> > > > > > > > > > > > > > I'm +1 on Jun's suggestion as long as it can work
> > for
> > > > all
> > > > > > the
> > > > > > > > > > > requests.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <
> > > > > jun@confluent.io
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > (a) Admin requests can be sent to and handled
> by
> > > any
> > > > > > > broker.
> > > > > > > > > > > > > > > (b) Admin requests are processed
> asynchronously,
> > at
> > > > > least
> > > > > > > for
> > > > > > > > > > now.
> > > > > > > > > > > > > That is,
> > > > > > > > > > > > > > > when the client gets a response, it just means
> > that
> > > > the
> > > > > > > > request
> > > > > > > > > > is
> > > > > > > > > > > > > > > initiated, but not necessarily completed. Then,
> > > it's
> > > > up
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > client
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > issue another request to check the status for
> > > > > completion.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > To support (a), we were thinking of doing
> request
> > > > > > > forwarding
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > > > controller (utilizing KAFKA-1912). I am making
> an
> > > > > > > alternative
> > > > > > > > > > > > proposal.
> > > > > > > > > > > > > > > Basically, the broker can just write to
> ZooKeeper
> > > to
> > > > > > inform
> > > > > > > > the
> > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > about the request. For example, to handle
> > > > > > > > > partitionReassignment,
> > > > > > > > > > > the
> > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > will just write the requested partitions to
> > > > > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > > > > (like what AdminUtils currently does) and then
> > > send a
> > > > > > > > response
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > > client. This shouldn't take long and the
> > > > implementation
> > > > > > > will
> > > > > > > > be
> > > > > > > > > > > > simpler
> > > > > > > > > > > > > > > than forwarding the requests to the controller
> > > > through
> > > > > > RPC.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii
> > Biletskyi <
> > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jun,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I might be wrong but didn't we agree we will
> > let
> > > > any
> > > > > > > broker
> > > > > > > > > > from
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > cluster handle *long-running* admin requests
> > (at
> > > > this
> > > > > > > time
> > > > > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > reassignPartitions), via zk admin path. Thus
> > > > > > CreateTopics
> > > > > > > > etc
> > > > > > > > > > > > should
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> > > > > > > > jun@confluent.io>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I think we agreed that those admin requests
> > can
> > > > be
> > > > > > > issued
> > > > > > > > > to
> > > > > > > > > > > any
> > > > > > > > > > > > > > > broker.
> > > > > > > > > > > > > > > > > Because of that, there doesn't seem to be a
> > > > strong
> > > > > > need
> > > > > > > > to
> > > > > > > > > > know
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > controller. So, perhaps we can proceed by
> not
> > > > > making
> > > > > > > any
> > > > > > > > > > change
> > > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > format of TMR right now. When we start
> using
> > > > create
> > > > > > > topic
> > > > > > > > > > > request
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > producer, we will need a new version of TMR
> > > that
> > > > > > > doesn't
> > > > > > > > > > > trigger
> > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > As a first cut implementation, I think the
> > > broker
> > > > > can
> > > > > > > > just
> > > > > > > > > > > write
> > > > > > > > > > > > > to ZK
> > > > > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > > > > requests, instead of forwarding them to the
> > > > > > controller.
> > > > > > > > > This
> > > > > > > > > > > will
> > > > > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel
> Koshy
> > <
> > > > > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > For (1) yes we will circle back on that
> > > shortly
> > > > > > after
> > > > > > > > > > syncing
> > > > > > > > > > > > up
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > person. I think it is close to getting
> > > > committed
> > > > > > > > although
> > > > > > > > > > > > > development
> > > > > > > > > > > > > > > > > > for KAFKA-1927 can probably begin without
> > it.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > There is one more item we covered at the
> > > > hangout.
> > > > > > > i.e.,
> > > > > > > > > > > whether
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > want to add the coordinator to the topic
> > > > metadata
> > > > > > > > > response
> > > > > > > > > > or
> > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > There are two reasons I think we should
> try
> > > and
> > > > > > avoid
> > > > > > > > > > adding
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > > > > > > > - If we finally do request rerouting in
> > Kafka
> > > > > then
> > > > > > > the
> > > > > > > > > > field
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > >   little to no value. (It still helps to
> > > have a
> > > > > > > > separate
> > > > > > > > > > > > > > > > > >   ClusterMetadataRequest to query for
> > > > > cluster-wide
> > > > > > > > > > > information
> > > > > > > > > > > > > such
> > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > >   'which broker is the controller?' as
> Joe
> > > > > > > mentioned.)
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I think it would be cleaner to have an
> > > explicit
> > > > > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > > > > that you can send to any broker in order
> to
> > > > > obtain
> > > > > > > the
> > > > > > > > > > > > controller
> > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > in the future possibly other cluster-wide
> > > > > > > > information). I
> > > > > > > > > > > think
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > main argument against doing this and
> > instead
> > > > > adding
> > > > > > > it
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > metadata response was convenience - i.e.,
> > you
> > > > > don't
> > > > > > > > have
> > > > > > > > > to
> > > > > > > > > > > > > discover
> > > > > > > > > > > > > > > > > > the controller in advance. However, I
> don't
> > > see
> > > > > > much
> > > > > > > > > actual
> > > > > > > > > > > > > > > > > > benefit/convenience in this and in fact
> > think
> > > > it
> > > > > > is a
> > > > > > > > > > > > non-issue.
> > > > > > > > > > > > > Let
> > > > > > > > > > > > > > > > > > me know if I'm overlooking something
> here.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > As an example, say we need to initiate
> > > > partition
> > > > > > > > > > reassignment
> > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > issuing the new ReassignPartitionsRequest
> > to
> > > > the
> > > > > > > > > controller
> > > > > > > > > > > > > (assume
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > already have the desired manual partition
> > > > > > > assignment).
> > > > > > > > > If
> > > > > > > > > > we
> > > > > > > > > > > > > are to
> > > > > > > > > > > > > > > > > > augment topic metadata response then the
> > flow
> > > > be
> > > > > > > > > something
> > > > > > > > > > > like
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > :
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > - Issue topic metadata request to any
> > broker
> > > > (and
> > > > > > > > > discover
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > > > > - Connect to controller if required
> (i.e.,
> > if
> > > > the
> > > > > > > > broker
> > > > > > > > > > > above
> > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > - Issue the partition reassignment
> request
> > to
> > > > the
> > > > > > > > > > controller.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > With an explicit cluster metadata request
> > it
> > > > > would
> > > > > > > be:
> > > > > > > > > > > > > > > > > > - Issue cluster metadata request to any
> > > broker
> > > > > > > > > > > > > > > > > > - Connect to controller if required
> (i.e.,
> > if
> > > > the
> > > > > > > > broker
> > > > > > > > > > > above
> > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > - Issue the partition reassignment
> request
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > So it seems to add little practical value
> > and
> > > > > > bloats
> > > > > > > > > topic
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > The other angle to this is the following
> -
> > is
> > > > it
> > > > > a
> > > > > > > > matter
> > > > > > > > > > of
> > > > > > > > > > > > > naming?
> > > > > > > > > > > > > > > > > > Should we just rename topic metadata
> > > > > > request/response
> > > > > > > > to
> > > > > > > > > > just
> > > > > > > > > > > > > > > > > > MetadataRequest/Response and add cluster
> > > > metadata
> > > > > > to
> > > > > > > > it?
> > > > > > > > > By
> > > > > > > > > > > > that
> > > > > > > > > > > > > same
> > > > > > > > > > > > > > > > > > token should we also allow querying for
> the
> > > > > > consumer
> > > > > > > > > > > > coordinator
> > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > in future transaction coordinator) as
> well?
> > > > This
> > > > > > > leads
> > > > > > > > > to a
> > > > > > > > > > > > > bloated
> > > > > > > > > > > > > > > > > > request which isn't very appealing and
> > > > altogether
> > > > > > > > > > confusing.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700,
> > Jun
> > > > Rao
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 1. I just realized that in order to
> start
> > > > > working
> > > > > > > on
> > > > > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > need to merge the changes to
> > > > > OffsetCommitRequest
> > > > > > > > (from
> > > > > > > > > > > 0.8.2)
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > > > > This is planned to be done as part of
> > > > > KAFKA-1634.
> > > > > > > So,
> > > > > > > > > we
> > > > > > > > > > > will
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 2. Thinking about this a bit more, if
> the
> > > > > > semantic
> > > > > > > of
> > > > > > > > > > those
> > > > > > > > > > > > > "write"
> > > > > > > > > > > > > > > > > > > requests is async (i.e., after the
> client
> > > > gets
> > > > > a
> > > > > > > > > > response,
> > > > > > > > > > > it
> > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > > > > that the operation is initiated, but
> not
> > > > > > > necessarily
> > > > > > > > > > > > > completed), we
> > > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > really need to forward the requests to
> > the
> > > > > > > > controller.
> > > > > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > > > > receiving broker can just write the
> > > operation
> > > > > to
> > > > > > ZK
> > > > > > > > as
> > > > > > > > > > the
> > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > > > > line tool previously does. This will
> > > simplify
> > > > > the
> > > > > > > > > > > > > implementation.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 8. There is another implementation
> detail
> > > for
> > > > > > > > describe
> > > > > > > > > > > topic.
> > > > > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > want to read the topic config from the
> > > broker
> > > > > > > cache,
> > > > > > > > > > > instead
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > > > > Currently, every broker reads the
> > > topic-level
> > > > > > > config
> > > > > > > > > for
> > > > > > > > > > > all
> > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > However, it ignores those for topics
> not
> > > > hosted
> > > > > > on
> > > > > > > > > > itself.
> > > > > > > > > > > > So,
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > to change TopicConfigManager a bit so
> > that
> > > it
> > > > > > > caches
> > > > > > > > > the
> > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii
> > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala requests
> > > > objects,
> > > > > > use
> > > > > > > > java
> > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > > > > >     A: Gwen kindly took that
> > > (KAFKA-1927).
> > > > > It's
> > > > > > > > > > important
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > speed
> > > > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > > > > >          there since this ticket
> blocks
> > > > other
> > > > > > > > > important
> > > > > > > > > > > > > changes.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute facility vs
> > > client
> > > > > > > > > maintaining
> > > > > > > > > > > > > cluster
> > > > > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > > > > >     A: Jay has added pseudo code to
> > > > > KAFKA-1912
> > > > > > -
> > > > > > > > need
> > > > > > > > > > to
> > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > > > > >         easy to implement as a
> > > server-side
> > > > > > > feature
> > > > > > > > > > > > (comments
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 3. Q: Controller field in wire
> > protocol.
> > > > > > > > > > > > > > > > > > > >     A: This might be useful for
> > clients,
> > > > add
> > > > > > this
> > > > > > > > to
> > > > > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic creation from
> > TMR.
> > > > > > > > > > > > > > > > > > > >     A: I will add proposed by Jun
> > > solution
> > > > > > (using
> > > > > > > > > > > clientId
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs
> > > > grabbing
> > > > > > all
> > > > > > > > > > > protocol
> > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > >     A: It was decided to try to
> gather
> > > all
> > > > > > > changes
> > > > > > > > to
> > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > > > > >         In case of TMR it worth
> > checking:
> > > > > > > > KAFKA-2020
> > > > > > > > > > and
> > > > > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to
> deserialize
> > > > > user's
> > > > > > > > input
> > > > > > > > > in
> > > > > > > > > > > CLI
> > > > > > > > > > > > > tool.
> > > > > > > > > > > > > > > > > > > >     A: Use jackson for that, /tools
> > > project
> > > > > is
> > > > > > a
> > > > > > > > > > separate
> > > > > > > > > > > > > jar so
> > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs
> > > generic
> > > > > > status
> > > > > > > > > check
> > > > > > > > > > > > > command.
> > > > > > > > > > > > > > > > > > > >      A: For long-running requests
> like
> > > > > reassign
> > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > > > > >          it makes sense to introduce
> > it.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >  Please add, correct me if I missed
> > > > > something.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM,
> Andrii
> > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > You are right, I removed
> > > ClusterMetadata
> > > > > > > because
> > > > > > > > we
> > > > > > > > > > > have
> > > > > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > > > > what we need in TopicMetadata.
> Also,
> > as
> > > > Jay
> > > > > > > > pointed
> > > > > > > > > > out
> > > > > > > > > > > > > > > earlier,
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > would like to have "orthogonal"
> API,
> > > but
> > > > at
> > > > > > the
> > > > > > > > > same
> > > > > > > > > > > time
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > But I like your idea and even have
> > some
> > > > > other
> > > > > > > > > > arguments
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > > > > There is also DescribeTopicRequest
> > > which
> > > > > was
> > > > > > > > > proposed
> > > > > > > > > > > in
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > > > > it returns topic configs,
> partitions,
> > > > > > > replication
> > > > > > > > > > > factor
> > > > > > > > > > > > > plus
> > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > > > > leader replica. The later part is
> > > really
> > > > > > > already
> > > > > > > > > > there
> > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > > > > So again we'll have to add stuff to
> > > TMR,
> > > > > not
> > > > > > to
> > > > > > > > > > > duplicate
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > newly added requests. However, this
> > way
> > > > > we'll
> > > > > > > end
> > > > > > > > > up
> > > > > > > > > > > with
> > > > > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > > > > request which returns cluster
> > metadata,
> > > > > topic
> > > > > > > > > > > replication
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > > plus partition replication data.
> > Seems
> > > > > > logical
> > > > > > > to
> > > > > > > > > > split
> > > > > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > > > > - ClusterMetadata (brokers +
> > > controller,
> > > > > > maybe
> > > > > > > > smth
> > > > > > > > > > > else)
> > > > > > > > > > > > > > > > > > > > > - TopicMetadata (topic info +
> > partition
> > > > > > > details)
> > > > > > > > > > > > > > > > > > > > > But since current TMR is involved
> in
> > > lots
> > > > > of
> > > > > > > > places
> > > > > > > > > > > > > (including
> > > > > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > > > as I understand) this might be very
> > > > serious
> > > > > > > > change
> > > > > > > > > > and
> > > > > > > > > > > it
> > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > > > > sense to stick with current
> approach.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM,
> Joel
> > > > > Koshy <
> > > > > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> I may be missing some context but
> > > > > hopefully
> > > > > > > this
> > > > > > > > > > will
> > > > > > > > > > > > > also be
> > > > > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > > > > >> today: I thought the earlier
> > proposal
> > > > > where
> > > > > > > > there
> > > > > > > > > > was
> > > > > > > > > > > an
> > > > > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > > > > >> ClusterMetadata request was
> clearer
> > > and
> > > > > > > > explicit.
> > > > > > > > > > > During
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > >> this thread I think the conclusion
> > was
> > > > > that
> > > > > > > the
> > > > > > > > > main
> > > > > > > > > > > > need
> > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> controller information and that
> can
> > be
> > > > > > rolled
> > > > > > > > into
> > > > > > > > > > the
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > >> response but that seems a bit
> > > irrelevant
> > > > > to
> > > > > > > > topic
> > > > > > > > > > > > > metadata.
> > > > > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > > > > >> think the full broker-list is also
> > > > > > irrelevant
> > > > > > > to
> > > > > > > > > > topic
> > > > > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > >> it is already there and in use. I
> > > think
> > > > > > there
> > > > > > > is
> > > > > > > > > > still
> > > > > > > > > > > > > room
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > >> explicit ClusterMetadata request
> > since
> > > > > there
> > > > > > > may
> > > > > > > > > be
> > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > >> cluster-level information that we
> > may
> > > > want
> > > > > > to
> > > > > > > > add
> > > > > > > > > > over
> > > > > > > > > > > > > time
> > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> have nothing to do with topic
> > > metadata).
> > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM
> > > > +0200,
> > > > > > > Andrii
> > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > 101. Okay, if you say that such
> > use
> > > > case
> > > > > > is
> > > > > > > > > > > > important. I
> > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > >> > using clientId for these
> purposes
> > is
> > > > > fine
> > > > > > -
> > > > > > > if
> > > > > > > > > we
> > > > > > > > > > > > > already
> > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > > > > >> > as part of all Wire protocol
> > > messages,
> > > > > why
> > > > > > > not
> > > > > > > > > use
> > > > > > > > > > > > that.
> > > > > > > > > > > > > > > > > > > > >> > I will update KIP-4 page if
> nobody
> > > has
> > > > > > other
> > > > > > > > > ideas
> > > > > > > > > > > > > (which
> > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP
> > > > > > > accordingly.
> > > > > > > > I
> > > > > > > > > > > think
> > > > > > > > > > > > > we can
> > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > > > > >> > fine-grained error codes if some
> > > error
> > > > > > code
> > > > > > > > > > received
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > >> > won't give enough context to
> > return
> > > a
> > > > > > > > > descriptive
> > > > > > > > > > > > error
> > > > > > > > > > > > > > > > message
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > Look forward to discussing all
> > > > > outstanding
> > > > > > > > > issues
> > > > > > > > > > in
> > > > > > > > > > > > > detail
> > > > > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59
> PM,
> > > Jun
> > > > > Rao
> > > > > > <
> > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > 101. There may be a use case
> > where
> > > > you
> > > > > > > only
> > > > > > > > > want
> > > > > > > > > > > the
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > > > > >> > > manually by admins. Currently,
> > you
> > > > can
> > > > > > do
> > > > > > > > that
> > > > > > > > > > by
> > > > > > > > > > > > > > > disabling
> > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > creation and issue topic
> > creation
> > > > from
> > > > > > the
> > > > > > > > > > > > > TopicCommand.
> > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > > > > >> > > topic creation completely on
> the
> > > > > broker
> > > > > > > and
> > > > > > > > > > don't
> > > > > > > > > > > > > have a
> > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > >> > > between topic creation
> requests
> > > from
> > > > > the
> > > > > > > > > regular
> > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > > > > >> > > can't support manual topic
> > > creation
> > > > > any
> > > > > > > > more.
> > > > > > > > > I
> > > > > > > > > > > was
> > > > > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > > > > >> > > way of distinguishing the
> > clients
> > > > > making
> > > > > > > the
> > > > > > > > > > topic
> > > > > > > > > > > > > > > creation
> > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > >> > > using clientId. For example,
> the
> > > > admin
> > > > > > > tool
> > > > > > > > > can
> > > > > > > > > > > set
> > > > > > > > > > > > > it to
> > > > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > > > > >> > > admin and the broker can treat
> > > that
> > > > > > > clientId
> > > > > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > Also, there is a related
> > > discussion
> > > > in
> > > > > > > > > > KAFKA-2020.
> > > > > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > following in
> > > TopicMetadataResponse:
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > 1. If leader is not available,
> > we
> > > > set
> > > > > > the
> > > > > > > > > > > partition
> > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > > > > >> > > 2. If a non-leader replica is
> > not
> > > > > > > available,
> > > > > > > > > we
> > > > > > > > > > > take
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > >> > > the assigned replica list and
> > isr
> > > in
> > > > > the
> > > > > > > > > > response.
> > > > > > > > > > > > As
> > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > > > > >> > > doing that, we set the
> partition
> > > > level
> > > > > > > error
> > > > > > > > > > code
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > This has a few problems.
> First,
> > > > > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > > > > >> > > an error, at least for the
> > normal
> > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > > > > >> > > to find out the leader.
> Second,
> > it
> > > > can
> > > > > > > > happen
> > > > > > > > > > that
> > > > > > > > > > > > > both
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > another replica are not
> > available
> > > at
> > > > > the
> > > > > > > > same
> > > > > > > > > > > time.
> > > > > > > > > > > > > There
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > >> > > to indicate both. Third, even
> > if a
> > > > > > replica
> > > > > > > > is
> > > > > > > > > > not
> > > > > > > > > > > > > > > available,
> > > > > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > > > > >> > > useful to return its replica
> id
> > > > since
> > > > > > some
> > > > > > > > > > clients
> > > > > > > > > > > > > (e.g.
> > > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > One way to address this issue
> is
> > > to
> > > > > > always
> > > > > > > > > > return
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > > > > >> > > leader, assigned replicas, and
> > isr
> > > > > > > > regardless
> > > > > > > > > of
> > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > > > > >> > > broker is live or not. Since
> we
> > > also
> > > > > > > return
> > > > > > > > > the
> > > > > > > > > > > list
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > client can figure out whether
> a
> > > > leader
> > > > > > or
> > > > > > > a
> > > > > > > > > > > replica
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > > > > >> > > accordingly. This way, we
> don't
> > > need
> > > > > to
> > > > > > > set
> > > > > > > > > the
> > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > >> > > when the leader or a replica
> is
> > > not
> > > > > > > > available.
> > > > > > > > > > > This
> > > > > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > > > > >> > > protocol, but does change the
> > > > > semantics.
> > > > > > > > Since
> > > > > > > > > > we
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > > > > >> > > of TopicMetadataRequest here,
> we
> > > can
> > > > > > > > > potentially
> > > > > > > > > > > > > piggyback
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > 102.1 For those types of
> errors
> > > due
> > > > to
> > > > > > > > invalid
> > > > > > > > > > > > input,
> > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > >> > > guard it at parameter
> validation
> > > > time
> > > > > > and
> > > > > > > > > throw
> > > > > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > > > > >> > > without even sending the
> request
> > > to
> > > > > the
> > > > > > > > > broker?
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37
> > AM,
> > > > > Andrii
> > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > 101. If I understand you
> > > > correctly,
> > > > > > you
> > > > > > > > are
> > > > > > > > > > > saying
> > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > > > > >> > > > will be ported to TMR_V1)
> > won't
> > > be
> > > > > > able
> > > > > > > to
> > > > > > > > > > > > > automatically
> > > > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > unconditionally remove topic
> > > > > creation
> > > > > > > from
> > > > > > > > > > > there).
> > > > > > > > > > > > > But
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > > > > >> > > > Ok, about your proposal: I'm
> > > not a
> > > > > big
> > > > > > > fan
> > > > > > > > > > too,
> > > > > > > > > > > > > when it
> > > > > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > > > > >> > > > clients directly in protocol
> > > > schema.
> > > > > > And
> > > > > > > > > also
> > > > > > > > > > > I'm
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > > > > >> > > > auto.create.topics.enable
> is a
> > > > > server
> > > > > > > side
> > > > > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > > > > >> > > > in future versions, add this
> > > > setting
> > > > > > to
> > > > > > > > > > producer
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > based
> > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > > > > >> > > > UnknownTopic create topic
> > > > explicitly
> > > > > > by
> > > > > > > a
> > > > > > > > > > > separate
> > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because
> > we
> > > > want
> > > > > > to
> > > > > > > > > > support
> > > > > > > > > > > > > batching
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > > > > >> > > > want to give descriptive
> error
> > > > > > messages
> > > > > > > > for
> > > > > > > > > > > > clients.
> > > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > > > > >> > > > to construct such messages
> > (e.g.
> > > > > > > > AdminClient
> > > > > > > > > > > layer
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > > > > > >> > > > means two cases: either
> > invalid
> > > > > > number -
> > > > > > > > > e.g.
> > > > > > > > > > > -1;
> > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > > > > >> > > > partitions argument wasn't)
> -
> > I
> > > > > > wrapped
> > > > > > > > > > > responses
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > > > > >> > > > other ideas, this was just
> > > initial
> > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll
> > change
> > > > > that
> > > > > > to
> > > > > > > > > > > probably
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16
> > PM,
> > > > Jun
> > > > > > > Rao <
> > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > 101. That's what I was
> > > thinking
> > > > > too,
> > > > > > > but
> > > > > > > > > it
> > > > > > > > > > > may
> > > > > > > > > > > > > not be
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > > > > >> > > > > we can let it not trigger
> > auto
> > > > > topic
> > > > > > > > > > creation.
> > > > > > > > > > > > > Then,
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > > > > UnknownTopicException,
> > > > > > > it
> > > > > > > > > can
> > > > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > > > > >> > > > > createTopicRequest for
> auto
> > > > topic
> > > > > > > > > creation.
> > > > > > > > > > On
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > never issue
> > > createTopicRequest.
> > > > > This
> > > > > > > > works
> > > > > > > > > > > when
> > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > > > > >> > > > > enabled on the broker
> side.
> > > > > > However, I
> > > > > > > > am
> > > > > > > > > > not
> > > > > > > > > > > > > sure how
> > > > > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > > > > >> > > > > when auto topic creation
> is
> > > > > disabled
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > > > broker
> > > > > > > > > > > > > > > side.
> > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > want to have a way to
> > manually
> > > > > > create
> > > > > > > a
> > > > > > > > > > topic,
> > > > > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > >> > > > > commands. However, then we
> > > need
> > > > a
> > > > > > way
> > > > > > > to
> > > > > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > > > > >> > > > > issued from the producer
> > > clients
> > > > > and
> > > > > > > the
> > > > > > > > > > admin
> > > > > > > > > > > > > tools.
> > > > > > > > > > > > > > > > May
> > > > > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > >> > > > > new field in
> > > createTopicRequest
> > > > > and
> > > > > > > set
> > > > > > > > it
> > > > > > > > > > > > > differently
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > >> > > > > client and the admin
> client.
> > > > > > However,
> > > > > > > I
> > > > > > > > am
> > > > > > > > > > not
> > > > > > > > > > > > > sure if
> > > > > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring
> existing
> > > > > > requests
> > > > > > > > is a
> > > > > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > > > > >> > > > > posted some comments in
> > > > > KAFKA-1927.
> > > > > > We
> > > > > > > > > will
> > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > > > > >> > > > > first, before adding the
> new
> > > > logic
> > > > > > in
> > > > > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > 102. About the
> AdminClient:
> > > > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird
> that
> > > we
> > > > > > return
> > > > > > > > > > > exception
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > we should either return
> > error
> > > > code
> > > > > > or
> > > > > > > > > throw
> > > > > > > > > > an
> > > > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > > > > >> > > > > 102.2. We probably
> shouldn't
> > > > > > > explicitly
> > > > > > > > > use
> > > > > > > > > > > the
> > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > > > > >> > > > > Not every request
> evolution
> > > > > requires
> > > > > > > an
> > > > > > > > > api
> > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at
> 4:08
> > > AM,
> > > > > > > Andrii
> > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > >
> andrii.biletskyi@stealth.ly
> > >
> > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks for you comments.
> > > > Answers
> > > > > > > > inline:
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > 100. There are a few
> > fields
> > > > such
> > > > > > as
> > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> PartitionsSerialized
> > > > that
> > > > > > are
> > > > > > > > > > > > represented
> > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > >> > > > > > > composite structures
> in
> > > > json.
> > > > > > > Could
> > > > > > > > we
> > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, now with Admin
> Client
> > > > this
> > > > > > > looks
> > > > > > > > a
> > > > > > > > > > bit
> > > > > > > > > > > > > weird.
> > > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > > > > >> > > > > > ReassignPartitionCommand
> > > > accepts
> > > > > > > input
> > > > > > > > > in
> > > > > > > > > > > > json,
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > > > > >> > > > > > interfaces unchanged,
> > where
> > > > > > > possible.
> > > > > > > > > > > > > > > > > > > > >> > > > > > If we port it to
> > > deserialized
> > > > > > > format,
> > > > > > > > in
> > > > > > > > > > CLI
> > > > > > > > > > > > > (/tools
> > > > > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > > > > >> > > > > > json library since
> /tools
> > is
> > > > > > written
> > > > > > > > in
> > > > > > > > > > java
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > > > > >> > > > > > provided by a user. Can
> we
> > > > > quickly
> > > > > > > > agree
> > > > > > > > > > on
> > > > > > > > > > > > what
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON,
> whatever)?
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> > > TopicMetadataRequest
> > > > > v1
> > > > > > > > still
> > > > > > > > > > > > trigger
> > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird
> now
> > > that
> > > > > we
> > > > > > > > have a
> > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought about how
> > the
> > > > new
> > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > > > > producer/consumer
> > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> > > ideally,
> > > > > we
> > > > > > > > don't
> > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger
> auto
> > > > topic
> > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > I agree, this strange
> > logic
> > > > > should
> > > > > > > be
> > > > > > > > > > fixed.
> > > > > > > > > > > > > I'm not
> > > > > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > > > > >> > > > > > correct me if I'm wrong,
> > but
> > > > it
> > > > > > > > doesn't
> > > > > > > > > > look
> > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > > > > >> > > > > > leverage AdminClient for
> > > that
> > > > in
> > > > > > > > > Producer
> > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant
> > getting
> > > > rid
> > > > > > of
> > > > > > > > > scala
> > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > HeartbeatRequestAndHeader
> > > > > > and
> > > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing
> when
> > > > > adding
> > > > > > > the
> > > > > > > > > new
> > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the long term
> > > plan
> > > > is
> > > > > > to
> > > > > > > > get
> > > > > > > > > > rid
> > > > > > > > > > > of
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > java request/response
> in
> > > the
> > > > > > > client.
> > > > > > > > > > Since
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > significant number of
> > new
> > > > > > > requests,
> > > > > > > > > > > perhaps
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing
> > > scala
> > > > > > > requests
> > > > > > > > > > first
> > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks like I
> > > > misunderstood
> > > > > > the
> > > > > > > > > point
> > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > > rework that. The only
> > thing
> > > is
> > > > > > that
> > > > > > > I
> > > > > > > > > > don't
> > > > > > > > > > > > see
> > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > > > > >> > > > > > least one existing
> > protocol
> > > > > > message.
> > > > > > > > > Thus,
> > > > > > > > > > > as
> > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > > > > > > > >> > > > > > Re porting all existing
> > > RQ/RP
> > > > in
> > > > > > > this
> > > > > > > > > > patch.
> > > > > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > > > > > >> > > > > > requirement to have
> Admin
> > > KIP
> > > > > > done,
> > > > > > > > I'm
> > > > > > > > > > > afraid
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > > > > > >> > > > > > There are 13 protocol
> > > messages
> > > > > and
> > > > > > > all
> > > > > > > > > > that
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > > > > > >> > > > > > intensive manual
> testing,
> > > no?
> > > > > I'm
> > > > > > > > afraid
> > > > > > > > > > I'm
> > > > > > > > > > > > > not the
> > > > > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka core internals :).
> > Let
> > > > me
> > > > > > know
> > > > > > > > > your
> > > > > > > > > > > > > thoughts
> > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at
> > 6:40
> > > > AM,
> > > > > > Jun
> > > > > > > > > Rao <
> > > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > 100. There are a few
> > > fields
> > > > > such
> > > > > > > as
> > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> PartitionsSerialized
> > > > that
> > > > > > are
> > > > > > > > > > > > represented
> > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > >> > > > > > > composite structures
> in
> > > > json.
> > > > > > > Could
> > > > > > > > we
> > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> > > > TopicMetadataRequest
> > > > > > v1
> > > > > > > > > still
> > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird
> now
> > > that
> > > > > we
> > > > > > > > have a
> > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought about how
> > the
> > > > new
> > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > > > > producer/consumer
> > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> > > ideally,
> > > > > we
> > > > > > > > don't
> > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger
> auto
> > > > topic
> > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant
> > > getting
> > > > > rid
> > > > > > > of
> > > > > > > > > > scala
> > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > HeartbeatRequestAndHeader
> > > > > > and
> > > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing
> when
> > > > > adding
> > > > > > > the
> > > > > > > > > new
> > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the long term
> > > plan
> > > > is
> > > > > > to
> > > > > > > > get
> > > > > > > > > > rid
> > > > > > > > > > > of
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > java request/response
> in
> > > the
> > > > > > > client.
> > > > > > > > > > Since
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > significant number of
> > new
> > > > > > > requests,
> > > > > > > > > > > perhaps
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing
> > > scala
> > > > > > > requests
> > > > > > > > > > first
> > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015
> at
> > > 3:37
> > > > > PM,
> > > > > > > > > Andrii
> > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > andrii.biletskyi@stealth.ly
> > > > >
> > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > As said above - I
> list
> > > > again
> > > > > > all
> > > > > > > > > > > comments
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > can see what's left
> > and
> > > > > > finalize
> > > > > > > > all
> > > > > > > > > > > > pending
> > > > > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is much
> needed
> > > > > > > > > functionality,
> > > > > > > > > > > but
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > really think these
> > > > protocols
> > > > > > > > > through.
> > > > > > > > > > We
> > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > of well thought-out,
> > > > > > orthoganol
> > > > > > > > > apis.
> > > > > > > > > > > For
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > important to think
> > > through
> > > > > the
> > > > > > > end
> > > > > > > > > > state
> > > > > > > > > > > > > even if
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > won't implement in
> the
> > > > first
> > > > > > > > phase.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Definitely behind
> > > this.
> > > > > > Would
> > > > > > > > > > > > appreciate
> > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > how this can be
> > > improved.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's please
> please
> > > > > please
> > > > > > > wait
> > > > > > > > > > until
> > > > > > > > > > > > we
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > to the new java
> > protocol
> > > > > > > > > definitions.
> > > > > > > > > > If
> > > > > > > > > > > > we
> > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > objects that is just
> > > > > > generating
> > > > > > > > more
> > > > > > > > > > > work
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> latest
> > > > > patch -
> > > > > > > > > removed
> > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This proposal
> > > > introduces
> > > > > a
> > > > > > > new
> > > > > > > > > type
> > > > > > > > > > > of
> > > > > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > inconsistent with
> > > > everything
> > > > > > > else
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > other marker value.
> > You
> > > > > could
> > > > > > > > argue
> > > > > > > > > > > either
> > > > > > > > > > > > > way
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > for consistency. For
> > > > clients
> > > > > > > that
> > > > > > > > > > > > > implemented
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > than our scala code
> > > these
> > > > > > basic
> > > > > > > > > > > primitives
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> latest
> > > > > patch -
> > > > > > > > > removed
> > > > > > > > > > > > > MaybeOf
> > > > > > > > > > > > > > > > type
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata:
> > This
> > > > > seems
> > > > > > > to
> > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers, topics, and
> > > > > > > partitions. I
> > > > > > > > > > think
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> ClusterMetadataRequest
> > > (or
> > > > > > just
> > > > > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > controller. Or are
> > there
> > > > > other
> > > > > > > > > things
> > > > > > > > > > we
> > > > > > > > > > > > > could
> > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I agree. Updated
> > the
> > > > KIP.
> > > > > > > Let's
> > > > > > > > > > > extends
> > > > > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a
> tendency
> > to
> > > > try
> > > > > > to
> > > > > > > > > make a
> > > > > > > > > > > lot
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > particular nodes.
> This
> > > > adds
> > > > > a
> > > > > > > lot
> > > > > > > > of
> > > > > > > > > > > > burden
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > sounds easy but each
> > > > > discovery
> > > > > > > can
> > > > > > > > > > fail
> > > > > > > > > > > in
> > > > > > > > > > > > > many
> > > > > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > full state machine
> to
> > do
> > > > > > > right). I
> > > > > > > > > > think
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > commands and ideally
> > as
> > > > many
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > other
> > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers and just
> > > redirect
> > > > to
> > > > > > the
> > > > > > > > > > > > controller
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > there would be a
> > general
> > > > way
> > > > > > to
> > > > > > > > > > > > encapsulate
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a very
> > > interesting
> > > > > > idea,
> > > > > > > > but
> > > > > > > > > > > seems
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > feature (like
> > > performance
> > > > > > > > > > > considerations,
> > > > > > > > > > > > > how
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > I believe this
> > shouldn't
> > > > be
> > > > > a
> > > > > > > > > blocker.
> > > > > > > > > > > If
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > point it won't
> affect
> > > > Admin
> > > > > > > > changes
> > > > > > > > > -
> > > > > > > > > > at
> > > > > > > > > > > > > least
> > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We should
> probably
> > > > > > normalize
> > > > > > > > the
> > > > > > > > > > key
> > > > > > > > > > > > > value
> > > > > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > than embedding a new
> > > > > > formatting.
> > > > > > > > So
> > > > > > > > > > two
> > > > > > > > > > > > > strings
> > > > > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > internal equals
> sign.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> latest
> > > > > patch -
> > > > > > > > > > > normalized
> > > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the
> > postcondition
> > > of
> > > > > > these
> > > > > > > > > APIs
> > > > > > > > > > > that
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > the command has been
> > > > > > completed?
> > > > > > > It
> > > > > > > > > is
> > > > > > > > > > a
> > > > > > > > > > > > lot
> > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > been completed so
> you
> > > know
> > > > > > that
> > > > > > > if
> > > > > > > > > you
> > > > > > > > > > > > > create a
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > it you won't get an
> > > > > exception
> > > > > > > > about
> > > > > > > > > > > there
> > > > > > > > > > > > > being
> > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: For long running
> > > > requests
> > > > > > > (like
> > > > > > > > > > > > reassign
> > > > > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > command has begun -
> so
> > > we
> > > > > > don't
> > > > > > > > > block
> > > > > > > > > > > the
> > > > > > > > > > > > > > > client.
> > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic commands, this
> > > will
> > > > be
> > > > > > > > > > refactored
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > immediately, since
> the
> > > > > > > Controller
> > > > > > > > > will
> > > > > > > > > > > > serve
> > > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up ticket
> > > > > KAFKA-1777).
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe topic
> and
> > > list
> > > > > > > topics
> > > > > > > > > > > > duplicate
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > request. Is there a
> > > reason
> > > > > to
> > > > > > > give
> > > > > > > > > > back
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > like if we just make
> > the
> > > > > > > > > > post-condition
> > > > > > > > > > > of
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic is deleted
> that
> > > will
> > > > > get
> > > > > > > rid
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > be much more
> > intuitive.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> latest
> > > > > patch -
> > > > > > > > > removed
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we
> consider
> > > > > batching
> > > > > > > > these
> > > > > > > > > > > > > requests?
> > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > allow multiple
> > > operations
> > > > to
> > > > > > be
> > > > > > > > > > batched.
> > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > we will get a lot of
> > > code
> > > > > that
> > > > > > > > does
> > > > > > > > > > > > > something
> > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > this code will work
> > > great
> > > > > when
> > > > > > > you
> > > > > > > > > > test
> > > > > > > > > > > > on 5
> > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > > > please
> > > > > > > check
> > > > > > > > > > "Topic
> > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I think we
> should
> > > also
> > > > > > > discuss
> > > > > > > > > how
> > > > > > > > > > > we
> > > > > > > > > > > > > want
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > client api for these
> > > > > > operations.
> > > > > > > > > > > Currently
> > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > is totally sketchy.
> I
> > > > think
> > > > > we
> > > > > > > > > > probably
> > > > > > > > > > > > need
> > > > > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > that exposes
> > > > administrative
> > > > > > > > > > > functionality.
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > properly test the
> new
> > > > apis,
> > > > > I
> > > > > > > > > suspect.
> > > > > > > > > > > We
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > > > please
> > > > > > > check
> > > > > > > > > > "Admin
> > > > > > > > > > > > > Client"
> > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The other
> > > information
> > > > > that
> > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > information about
> > > > > > > partitions--how
> > > > > > > > > much
> > > > > > > > > > > > data
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > the segment offsets,
> > > what
> > > > is
> > > > > > the
> > > > > > > > > > log-end
> > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > the compaction
> point,
> > > > etc. I
> > > > > > > think
> > > > > > > > > > that
> > > > > > > > > > > > done
> > > > > > > > > > > > > > > right
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > successor to the
> very
> > > > > awkward
> > > > > > > > > > > > OffsetRequest
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > be resolved in a
> > > separate
> > > > > KIP
> > > > > > /
> > > > > > > > jira
> > > > > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 12. Generally we can
> > do
> > > > good
> > > > > > > error
> > > > > > > > > > > > handling
> > > > > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > messages. I.e.
> > generally
> > > > the
> > > > > > > > client
> > > > > > > > > > has
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > context
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > an error that the
> > topic
> > > > > > doesn't
> > > > > > > > > exist
> > > > > > > > > > to
> > > > > > > > > > > > say
> > > > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > than "error code 14"
> > (or
> > > > > > > > whatever).
> > > > > > > > > > > Maybe
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > this is hard? If we
> > want
> > > > to
> > > > > > add
> > > > > > > > > > > > server-side
> > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > need to do this in a
> > > > > > consistent
> > > > > > > > way
> > > > > > > > > > > across
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > > > please
> > > > > > > check
> > > > > > > > > > > > "Protocol
> > > > > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > comprehensive,
> > > > fine-grained
> > > > > > list
> > > > > > > > of
> > > > > > > > > > > error
> > > > > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from
> > Guozhang:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe topic
> > > > request:
> > > > > it
> > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > > great to
> > > > > > > > > > > > > > > > go
> > > > > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex for
> > > this
> > > > > > > request.
> > > > > > > > > For
> > > > > > > > > > > > > example,
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > the topic command is
> > to
> > > > list
> > > > > > all
> > > > > > > > > > topics
> > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex
> then
> > we
> > > > > have
> > > > > > to
> > > > > > > > > first
> > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > description info and
> > > then
> > > > > > filter
> > > > > > > > at
> > > > > > > > > > the
> > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in
> > > create
> > > > > > topic:
> > > > > > > > > this
> > > > > > > > > > is
> > > > > > > > > > > > > related
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > maybe we can add
> > another
> > > > > > > metadata
> > > > > > > > > K-V
> > > > > > > > > > or
> > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > with config K-V in
> > > create
> > > > > > topic
> > > > > > > > like
> > > > > > > > > > we
> > > > > > > > > > > > did
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > field can be quite
> > > useful
> > > > in
> > > > > > > > storing
> > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > who issue the create
> > > > > command,
> > > > > > > etc,
> > > > > > > > > > which
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant
> setting.
> > > Then
> > > > > in
> > > > > > > the
> > > > > > > > > > > describe
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > on regex of the
> > metadata
> > > > > > field.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: As discussed it
> is
> > > very
> > > > > > > > > interesting
> > > > > > > > > > > but
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > we have some basic
> > > > > > functionality
> > > > > > > > > > there.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today all the
> > admin
> > > > > > > operations
> > > > > > > > > are
> > > > > > > > > > > > > async in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > return once it is
> > > written
> > > > in
> > > > > > ZK,
> > > > > > > > and
> > > > > > > > > > > that
> > > > > > > > > > > > > is why
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > > > > testUtil.waitForTopicCreated()
> > > > > > > > > /
> > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > request, etc. With
> > admin
> > > > > > > requests
> > > > > > > > we
> > > > > > > > > > > could
> > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > synchronous
> requests;
> > > when
> > > > > it
> > > > > > is
> > > > > > > > > > turned
> > > > > > > > > > > > on,
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > until the request
> has
> > > been
> > > > > > > > > completed.
> > > > > > > > > > > And
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > "token" field in the
> > > > > response,
> > > > > > > and
> > > > > > > > > > then
> > > > > > > > > > > > only
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > verification
> request"
> > > with
> > > > > the
> > > > > > > > given
> > > > > > > > > > > token
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I see your point.
> > My
> > > > idea
> > > > > > was
> > > > > > > > to
> > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > long running
> request,
> > > > where
> > > > > > > > needed.
> > > > > > > > > We
> > > > > > > > > > > can
> > > > > > > > > > > > > do it
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > concern is that
> > > > introducing
> > > > > a
> > > > > > > > token
> > > > > > > > > we
> > > > > > > > > > > > again
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > to do similar thing
> > > > > > introducing
> > > > > > > > > single
> > > > > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > this idea because we
> > > > wanted
> > > > > to
> > > > > > > > have
> > > > > > > > > > > schema
> > > > > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > a) have fixed schema
> > but
> > > > > > > introduce
> > > > > > > > > > each
> > > > > > > > > > > > > time new
> > > > > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > long-running
> requests
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > b) use one request
> for
> > > > > > > > verification
> > > > > > > > > > but
> > > > > > > > > > > > > > > generalize
> > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine with
> whatever
> > > > > > decision
> > > > > > > > > > > community
> > > > > > > > > > > > > come
> > > > > > > > > > > > > > > to.
> > > > > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 16. Specifically for
> > > > > > ownership,
> > > > > > > I
> > > > > > > > > > think
> > > > > > > > > > > > the
> > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > like you are
> > describing
> > > > ACL)
> > > > > > via
> > > > > > > > an
> > > > > > > > > > > > external
> > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > I remember KIP-11
> > > > described
> > > > > > > this,
> > > > > > > > > but
> > > > > > > > > > I
> > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no problem.
> > Not
> > > > > sure
> > > > > > > > though
> > > > > > > > > > how
> > > > > > > > > > > > we
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be committed
> > first
> > > > and
> > > > > > > > include
> > > > > > > > > > > > changes
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I added this
> > > note
> > > > to
> > > > > > > "Open
> > > > > > > > > > > > > Questions"
> > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015
> > at
> > > > > 12:34
> > > > > > > AM,
> > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > andrii.biletskyi@stealth.ly
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Today I uploaded
> the
> > > > patch
> > > > > > > that
> > > > > > > > > > covers
> > > > > > > > > > > > > some of
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf
> > > > optional
> > > > > > > type
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - switched to java
> > > > > protocol
> > > > > > > > > > > definitions
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - simplified
> > messages
> > > > > > > > (normalized
> > > > > > > > > > > > configs,
> > > > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I also updated the
> > > KIP-4
> > > > > > with
> > > > > > > > > > > respective
> > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin
> > > Operations
> > > > > ->
> > > > > > > > > updated
> > > > > > > > > > > Wire
> > > > > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove
> > > ClusterMetadata
> > > > > ->
> > > > > > > > > changed
> > > > > > > > > > to
> > > > > > > > > > > > > extend
> > > > > > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin Client ->
> > > > updated
> > > > > my
> > > > > > > > > initial
> > > > > > > > > > > > > proposal
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Error codes ->
> > > > proposed
> > > > > > > > > > fine-grained
> > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I will also send a
> > > > > separate
> > > > > > > > email
> > > > > > > > > to
> > > > > > > > > > > > > cover all
> > > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12,
> 2015
> > > at
> > > > > 9:26
> > > > > > > PM,
> > > > > > > > > > Gwen
> > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> It actually
> > specifies
> > > > > > changes
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are
> > > > consistent
> > > > > in
> > > > > > > > this
> > > > > > > > > > > regard
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12,
> > 2015
> > > at
> > > > > > 12:21
> > > > > > > > PM,
> > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Specifically
> for
> > > > > > > ownership, I
> > > > > > > > > > think
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > like you are
> > > > describing
> > > > > > > ACL)
> > > > > > > > > via
> > > > > > > > > > an
> > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I remember
> KIP-11
> > > > > > described
> > > > > > > > > this,
> > > > > > > > > > > > but I
> > > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I
> > think
> > > > > KIP-4
> > > > > > > > > focuses
> > > > > > > > > > > on
> > > > > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > exists from
> Kafka
> > > > > > brokers,
> > > > > > > > not
> > > > > > > > > on
> > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > should exist
> but
> > > > > doesn't
> > > > > > > yet?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12,
> > > 2015
> > > > at
> > > > > > > 6:37
> > > > > > > > > AM,
> > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just want to
> > > > > elaborate a
> > > > > > > bit
> > > > > > > > > > more
> > > > > > > > > > > on
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> describe-topic
> > > based
> > > > > on
> > > > > > > > > config /
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694.
> > The
> > > > > main
> > > > > > > > > > motivation
> > > > > > > > > > > > is
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms,
> > which
> > > I
> > > > > > think
> > > > > > > is
> > > > > > > > > > quite
> > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> architecture:
> > > today
> > > > > > anyone
> > > > > > > > can
> > > > > > > > > > > > create
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> there is no
> > > concept
> > > > or
> > > > > > > > > > "ownership"
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> users. For
> > > example,
> > > > at
> > > > > > > > > LinkedIn
> > > > > > > > > > we
> > > > > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some casual
> > topic
> > > > name
> > > > > > > > prefix,
> > > > > > > > > > > which
> > > > > > > > > > > > > is a
> > > > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale our
> > > > > customers.
> > > > > > It
> > > > > > > > > would
> > > > > > > > > > > be
> > > > > > > > > > > > > great
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > > topics
> > > > > that
> > > > > > > is
> > > > > > > > > > > created
> > > > > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > > topics
> > > > > > whose
> > > > > > > > > > > retention
> > > > > > > > > > > > > time
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > > topics
> > > > > > whose
> > > > > > > > > > writable
> > > > > > > > > > > > > group
> > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> authorization),
> > > > etc..
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One possible
> way
> > > to
> > > > > > > achieve
> > > > > > > > > this
> > > > > > > > > > > is
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> create-topic
> > > > request,
> > > > > > > whose
> > > > > > > > > > value
> > > > > > > > > > > > will
> > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic; then
> > > > > > > describe-topics
> > > > > > > > > can
> > > > > > > > > > > > > choose to
> > > > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2)
> config
> > > K-V
> > > > > > > > matching,
> > > > > > > > > > 3)
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5,
> > > 2015
> > > > at
> > > > > > > 4:37
> > > > > > > > > PM,
> > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for
> the
> > > > > updated
> > > > > > > > wiki.
> > > > > > > > > A
> > > > > > > > > > > few
> > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error
> > > > description
> > > > > in
> > > > > > > > > > > response: I
> > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> several
> > different
> > > > > error
> > > > > > > > cases
> > > > > > > > > > > then
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In
> > general
> > > > the
> > > > > > > > > errorCode
> > > > > > > > > > > > > itself
> > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> describing
> the
> > > > server
> > > > > > > side
> > > > > > > > > > > errors.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe
> > topic
> > > > > > > request:
> > > > > > > > it
> > > > > > > > > > > would
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name
> > regex
> > > > for
> > > > > > this
> > > > > > > > > > > request.
> > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic
> > command
> > > > is
> > > > > to
> > > > > > > > list
> > > > > > > > > > all
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name
> > regex
> > > > then
> > > > > > we
> > > > > > > > have
> > > > > > > > > > to
> > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> description
> > info
> > > > and
> > > > > > then
> > > > > > > > > > filter
> > > > > > > > > > > at
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config
> K-Vs
> > in
> > > > > > create
> > > > > > > > > topic:
> > > > > > > > > > > > this
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can
> > add
> > > > > > another
> > > > > > > > > > metadata
> > > > > > > > > > > > > K-V or
> > > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with config
> K-V
> > > in
> > > > > > create
> > > > > > > > > topic
> > > > > > > > > > > > like
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field can be
> > > quite
> > > > > > useful
> > > > > > > > in
> > > > > > > > > > > > storing
> > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue the
> > > > create
> > > > > > > > command,
> > > > > > > > > > > etc,
> > > > > > > > > > > > > which
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant
> > > > setting.
> > > > > > > Then
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of
> the
> > > > > > metadata
> > > > > > > > > field.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all
> > the
> > > > > admin
> > > > > > > > > > operations
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> return once
> it
> > is
> > > > > > written
> > > > > > > > in
> > > > > > > > > > ZK,
> > > > > > > > > > > > and
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> request, etc.
> > > With
> > > > > > admin
> > > > > > > > > > requests
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous
> > > > requests;
> > > > > > > when
> > > > > > > > it
> > > > > > > > > > is
> > > > > > > > > > > > > turned
> > > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until the
> > request
> > > > has
> > > > > > > been
> > > > > > > > > > > > > completed. And
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token" field
> > in
> > > > the
> > > > > > > > > response,
> > > > > > > > > > > and
> > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> verification
> > > > request"
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > > > given
> > > > > > > > > > > > > > > token
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has been
> > > completed.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for
> > > extending
> > > > > > > > Metadata
> > > > > > > > > > > > request
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> information,
> > and
> > > > then
> > > > > > we
> > > > > > > > can
> > > > > > > > > > > remove
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar
> 3,
> > > 2015
> > > > > at
> > > > > > > > 10:23
> > > > > > > > > > AM,
> > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for
> > > sending
> > > > > > that
> > > > > > > > out
> > > > > > > > > > Joe
> > > > > > > > > > > -
> > > > > > > > > > > > I
> > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so
> > if
> > > > > notes
> > > > > > > can
> > > > > > > > be
> > > > > > > > > > > sent
> > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar
> > 02,
> > > > 2015
> > > > > > at
> > > > > > > > > > > 09:16:13AM
> > > > > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for
> > > > sending
> > > > > > > this
> > > > > > > > > out
> > > > > > > > > > > Joe.
> > > > > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon,
> Mar
> > 2,
> > > > > 2015
> > > > > > at
> > > > > > > > > 6:46
> > > > > > > > > > > AM,
> > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I
> > just
> > > > sent
> > > > > > > out a
> > > > > > > > > > > google
> > > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> everyone I
> > > > found
> > > > > > > > working
> > > > > > > > > > on
> > > > > > > > > > > a
> > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me
> > know
> > > > and
> > > > > > can
> > > > > > > > > update
> > > > > > > > > > > it,
> > > > > > > > > > > > > np.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We
> should
> > do
> > > > > this
> > > > > > > > every
> > > > > > > > > > > > Tuesday
> > > > > > > > > > > > > @
> > > > > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to
> > > make a
> > > > > > > google
> > > > > > > > > > > account
> > > > > > > > > > > > > so we
> > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To
> discuss
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in
> > progress
> > > > and
> > > > > > > > related
> > > > > > > > > > JIRA
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe
> > Stein
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue,
> > Feb
> > > > 24,
> > > > > > 2015
> > > > > > > > at
> > > > > > > > > > 2:59
> > > > > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com
> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's
> > stay
> > > on
> > > > > > > Google
> > > > > > > > > > > hangouts
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> available
> > > on
> > > > > > > youtube.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue,
> > Feb
> > > > 24,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > 11:49
> > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > jholoman@cloudera.com
> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay /
> > Joe
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're
> > > happy
> > > > > to
> > > > > > > send
> > > > > > > > > > out a
> > > > > > > > > > > > > Webex
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > sessions
> > > if
> > > > > > there
> > > > > > > > is
> > > > > > > > > > > > > interest and
> > > > > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> Thanks
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On
> Tue,
> > > Feb
> > > > > 24,
> > > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> Let's
> > > try
> > > > > to
> > > > > > > get
> > > > > > > > > the
> > > > > > > > > > > > > technical
> > > > > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> there
> > > is
> > > > > some
> > > > > > > > > benefit
> > > > > > > > > > > to
> > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we
> > post
> > > > > > > > > instructions
> > > > > > > > > > > and
> > > > > > > > > > > > > give
> > > > > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > working.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > Tuesday
> > > > at
> > > > > > that
> > > > > > > > > time
> > > > > > > > > > > > would
> > > > > > > > > > > > > work
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> -Jay
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On
> > Tue,
> > > > Feb
> > > > > > 24,
> > > > > > > > > 2015
> > > > > > > > > > at
> > > > > > > > > > > > > 8:18
> > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > Weekly
> > > > > > would
> > > > > > > be
> > > > > > > > > > great
> > > > > > > > > > > > > maybe
> > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I
> > > don't
> > > > > > mind
> > > > > > > > > google
> > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> whatever
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> we
> > > know
> > > > > the
> > > > > > > > > apache
> > > > > > > > > > > irc
> > > > > > > > > > > > > > > channel
> > > > > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > goes?
> > > > We
> > > > > > can
> > > > > > > > pull
> > > > > > > > > > > > > transcripts
> > > > > > > > > > > > > > > > too
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> makes
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> it
> > > > > helpful
> > > > > > > for
> > > > > > > > > > > things.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~
> > > > > Joestein
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> On
> > > Tue,
> > > > > Feb
> > > > > > > 24,
> > > > > > > > > > 2015
> > > > > > > > > > > at
> > > > > > > > > > > > > 11:10
> > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > jay.kreps@gmail.com
> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > We'd
> > > > > > talked
> > > > > > > > > about
> > > > > > > > > > > > > doing a
> > > > > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > generalizing
> > > > > > > > > > that a
> > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> for
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > everyone
> > > > > > > > > > spending a
> > > > > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > maybe
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > sync
> > > > up
> > > > > > > once
> > > > > > > > a
> > > > > > > > > > > week.
> > > > > > > > > > > > I
> > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > stuff,
> > > > > > make
> > > > > > > > > sure
> > > > > > > > > > we
> > > > > > > > > > > > > are on
> > > > > > > > > > > > > > > > top
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> tricky
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > issues,
> > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > We
> > > > can
> > > > > > make
> > > > > > > > it
> > > > > > > > > > > > publicly
> > > > > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > likes.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > Any
> > > > > > > interest
> > > > > > > > in
> > > > > > > > > > > doing
> > > > > > > > > > > > > this?
> > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > week.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > -Jay
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > On
> > > > Tue,
> > > > > > Feb
> > > > > > > > 24,
> > > > > > > > > > > 2015
> > > > > > > > > > > > at
> > > > > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > >
> > > Hi
> > > > > all,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > >
> > > > I've
> > > > > > > > updated
> > > > > > > > > > KIP
> > > > > > > > > > > > > page,
> > > > > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> added
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > >
> > > > some
> > > > > > > > > > >
> > > > > > > > > > ...
> > > > > > > > > >
> > > > > > > > > > [Message clipped]
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > --
> > > > > > > > -- Guozhang
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Guozhang Wang <wa...@gmail.com>.
Andrii,

I was not thinking about directly exporting them for users, but rather
re-implement their logic as command tool classes that get rid of the
dependencies like KafkaServer (we can use broker id list) and ZkClient
(since with the admin request we do not need it anymore), etc.

Admittedly it still requires users to have javac installed on their machine
to call these aded command line tools in their python / ruby / go programs,
and thinking about it for a second time, I am now not sure if it is a good
idea..

Guozhang

On Sat, Mar 21, 2015 at 2:03 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Guozhang,
>
> I'm not sure I understand how can we use those tools in CLI.
> First of all, those are *Test*Utils, e.g. waitUntilMetadataIsPropagated
> (which might be very useful for us) requires servers:Seq[KafkaServer]
> as an argument:
>
> def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String,
> partition: Int, timeout: Long = 5000L): Int
>
> how can we have access it at runtime in one of the brokers?
>
> Secondly, how can user directly call these tools if zookeeper might be
> not accessible (which is used in the tools, right?) at all - e.g. behind
> the VPC
> in AWS. Furthermore, I think with KIP-4 we are trying to create an
> abstraction and
> a single point of zookeeper interactions, to eliminate direct calls to ZK
> either to get
> some cluster information or to change something.
>
> Thanks,
> Andrii Biletskyi
>
> On Sat, Mar 21, 2015 at 3:37 AM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > Andrii,
> >
> > Actually the checking logic Jun mentioned is already implemented as
> > TestUtils.waitUntilXXX (LeaderIsElected, MetadataIsPropagated, etc...) I
> > think we can extend these functions as CLI tools like TopicCommand so
> that
> > users re-implementing such endpoint can directly call something like
> > java.tools.WaitUntilXXX (of course this requires them to have javac
> > installed, which should be a reasonable requirement?)
> >
> > Guozhang
> >
> >
> >
> > On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Jun,
> > >
> > > Not that I was saying we need to make requests blocking on server,
> > > it was just to emphasize that with async requests a client
> > implementations
> > > may be a little bit more than just issue request - get the response.
> > > Thanks for the explanation, I understand now that we can go with agreed
> > > solution though it may not be perfect.
> > > I believe this was one of the last controversial questions from the
> list.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Andrii,
> > > >
> > > > A few points.
> > > >
> > > > 1. Create/Alter can typically complete quickly. So, it's possible to
> > make
> > > > the request block until it's completed. However, currently, doing
> this
> > at
> > > > the broker is a bit involved. To make Create block, we will need to
> add
> > > > some callbacks in KafkaController. This is possible. However, the
> > > > controller logic currently is pretty completed. It would probably be
> > > better
> > > > if we clean it up first before adding more complexity to it. Alter is
> > > even
> > > > trickier. Adding partition is currently handled through
> > KafkaController.
> > > So
> > > > it can be dealt with in a similar way. However, Alter config is done
> > > > completely differently. It doesn't go through the controller.
> Instead,
> > > each
> > > > broker listens to ZooKeeper directly. So, it's not clear if there is
> an
> > > > easy way on the broker to figure out whether a config is applied on
> > every
> > > > broker.
> > > >
> > > > 2. Delete can potentially take long if a replica to be deleted is
> > > offline.
> > > > PreferredLeader/PartitionReassign can also take long. So, we can't
> > really
> > > > make those requests block on the broker.
> > > >
> > > > As you can see, at this moment it's not easy to make all admin
> requests
> > > > block on the broker. So, if we want the blocking feature in the admin
> > > > utility in the short term, doing the completion check at the admin
> > client
> > > > is probably an easier route, even though it may not be ideal.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Jun,
> > > > >
> > > > > I see your point. But wouldn't that lead to a "fat" client
> > > > implementations?
> > > > > Suppose someone would like to implement client for Admin Wire
> > protocol.
> > > > > Not only people will have to code quite complicated logic like
> "send
> > > > > describe
> > > > > request to each broker" (again state machin?) but it will also mean
> > > > people
> > > > > must understand internal kafka logic related to topic storage and
> how
> > > > > information is propageted from the controller to brokers.
> > > > > I see this like a dilemma between having a concise Wire Protocol
> and
> > > > > self-sufficient API to make client implementations simple.
> > > > > I don't have a win-win solution though.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > >
> > > > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > >
> > > > > > For 1), 2) and 3), blocking would probably mean that the new
> > metadata
> > > > is
> > > > > > propagated to every broker. To achieve that, the client can keep
> > > > issuing
> > > > > > the describe topic request to every broker until it sees the new
> > > > metadata
> > > > > > in the response.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Hm, actually the ticket you linked, Guozhang, brings as back
> > > > > > > to the problem what should be considered a post-condition for
> > > > > > > each of the admin commands.
> > > > > > > In my understanding:
> > > > > > >
> > > > > > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > > > > > (Not the controller picked up changes from zk and broadcasted
> > > > > > > LeaderAndIsr and UpdateMetadata)
> > > > > > >
> > > > > > > 2) AlterTopic - same as 1) - broker changed assignment data
> > > > > > > in zookeeper or created admin path for topic config change
> > > > > > >
> > > > > > > 3) DeleteTopic - admin path /admin/delete_topics is created
> > > > > > >
> > > > > > > 4) ReassignPartitions and PreferredReplica - corresponding
> admin
> > > > > > > path is created
> > > > > > >
> > > > > > > Now what can be considered a completed operation from the
> > client's
> > > > > > > perspective?
> > > > > > > 1) Topic is created once corresponding data is in zk
> > > > > > > (I remember there were some thoughts that it'd be good to
> > consider
> > > > > > > topic created once all replicas receive information about it
> and
> > > thus
> > > > > > > clients can produce/consume from it, but as was discussed this
> > > seems
> > > > > > > to be a hard thing to do)
> > > > > > >
> > > > > > > 2) Probably same as 1), so right after AlterTopic is issued
> > > > > > >
> > > > > > > 3) The topic has been removed from /brokers/topics
> > > > > > >
> > > > > > > 4) ReassignPartitions and PrefferedReplica were discussed
> > earlier -
> > > > > > > in short the former is completed once partition state info in
> zk
> > > > > matches
> > > > > > > reassignment request and admin path is empty, the latter - once
> > > data
> > > > > > > in zk shows that head of assignned replicas of the partition
> and
> > > > leader
> > > > > > > is the same replica
> > > > > > >
> > > > > > > Thoughts?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <
> > wangguoz@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > I think while loop is fine for supporting blocking, just that
> > we
> > > > need
> > > > > > to
> > > > > > > > add back off to avoid bombarding brokers with DescribeTopic
> > > > requests.
> > > > > > > >
> > > > > > > > Also I have linked KAFKA-1125
> > > > > > > > <https://issues.apache.org/jira/browse/KAFKA-1125> to your
> > > > proposal,
> > > > > > and
> > > > > > > > when KAFKA-1694 is done this ticket can also be closed.
> > > > > > > >
> > > > > > > > Guozhang
> > > > > > > >
> > > > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >
> > > > > > > > > Great.
> > > > > > > > > I want to elaborate this a bit more, to see we are on the
> > same
> > > > page
> > > > > > > > > concerning the client code.
> > > > > > > > >
> > > > > > > > > So with all topic commands being async a client
> (AdminClient
> > in
> > > > our
> > > > > > > > > case or any other other client people would like to
> > implement)
> > > to
> > > > > > > support
> > > > > > > > > a blocking operation (which seems to be a natural use-case
> > e.g.
> > > > for
> > > > > > > topic
> > > > > > > > > creation): would have to do:
> > > > > > > > > 1. issue CreateTopicRequest
> > > > > > > > > 2. if successful, in a "while" loop send
> DescribeTopicRequest
> > > and
> > > > > > > > > break the loop once all topics are returned in response (or
> > > upon
> > > > > > > > timeout).
> > > > > > > > > 3. if unsuccessful throw exception
> > > > > > > > > Would it be okay?
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Andrii Biletskyi
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Andrii,
> > > > > > > > > >
> > > > > > > > > > I think you are right. It seems that only
> > ReassignPartitions
> > > > > needs
> > > > > > a
> > > > > > > > > > separate verification request.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > >
> > > > > > > > > > > Guys,
> > > > > > > > > > > I like this idea too. Let's stick with that. I'll
> update
> > > KIP
> > > > > > > > > accordingly.
> > > > > > > > > > >
> > > > > > > > > > > I was also thinking we can avoid adding dedicated
> status
> > > > check
> > > > > > > > > > > requests for topic commands. - We have everything in
> > > > > > DescribeTopic
> > > > > > > > > > > for that! E.g.:
> > > > > > > > > > > User issued CreateTopic - to check the status client
> > sends
> > > > > > > > > DescribeTopic
> > > > > > > > > > > and checks whether is something returned for that
> topic.
> > > The
> > > > > same
> > > > > > > for
> > > > > > > > > > > alteration, deletion.
> > > > > > > > > > > Btw, PreferredReplica status can be also checked with
> > > > > > > > > > DescribeTopicRequest
> > > > > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > > > > For ReassignPartitions as discussed we'll need to have
> a
> > > > > separate
> > > > > > > > > > Verify...
> > > > > > > > > > > request.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> > > > > > wangguoz@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > +1 on broker writing to ZK for async handling. I was
> > > > thinking
> > > > > > > that
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > end state the admin requests would be eventually sent
> > to
> > > > > > > controller
> > > > > > > > > > > either
> > > > > > > > > > > > through re-routing or clients discovering them,
> instead
> > > of
> > > > > > > letting
> > > > > > > > > > > > controller listen on ZK admin path. But thinking
> about
> > > it a
> > > > > > > second
> > > > > > > > > > time,
> > > > > > > > > > > I
> > > > > > > > > > > > think it is actually simpler to let controller manage
> > > > > > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > > > > > >
> > > > > > > > > > > > Guozhang
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > > > > > jjkoshy.w@gmail.com
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > +1 as well. I think it helps to keep the rerouting
> > > > approach
> > > > > > > > > > orthogonal
> > > > > > > > > > > > > to this KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps
> > > > wrote:
> > > > > > > > > > > > > > I'm +1 on Jun's suggestion as long as it can work
> > for
> > > > all
> > > > > > the
> > > > > > > > > > > requests.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <
> > > > > jun@confluent.io
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > (a) Admin requests can be sent to and handled
> by
> > > any
> > > > > > > broker.
> > > > > > > > > > > > > > > (b) Admin requests are processed
> asynchronously,
> > at
> > > > > least
> > > > > > > for
> > > > > > > > > > now.
> > > > > > > > > > > > > That is,
> > > > > > > > > > > > > > > when the client gets a response, it just means
> > that
> > > > the
> > > > > > > > request
> > > > > > > > > > is
> > > > > > > > > > > > > > > initiated, but not necessarily completed. Then,
> > > it's
> > > > up
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > client
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > issue another request to check the status for
> > > > > completion.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > To support (a), we were thinking of doing
> request
> > > > > > > forwarding
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > > > controller (utilizing KAFKA-1912). I am making
> an
> > > > > > > alternative
> > > > > > > > > > > > proposal.
> > > > > > > > > > > > > > > Basically, the broker can just write to
> ZooKeeper
> > > to
> > > > > > inform
> > > > > > > > the
> > > > > > > > > > > > > controller
> > > > > > > > > > > > > > > about the request. For example, to handle
> > > > > > > > > partitionReassignment,
> > > > > > > > > > > the
> > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > will just write the requested partitions to
> > > > > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > > > > (like what AdminUtils currently does) and then
> > > send a
> > > > > > > > response
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > > client. This shouldn't take long and the
> > > > implementation
> > > > > > > will
> > > > > > > > be
> > > > > > > > > > > > simpler
> > > > > > > > > > > > > > > than forwarding the requests to the controller
> > > > through
> > > > > > RPC.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii
> > Biletskyi <
> > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jun,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I might be wrong but didn't we agree we will
> > let
> > > > any
> > > > > > > broker
> > > > > > > > > > from
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > cluster handle *long-running* admin requests
> > (at
> > > > this
> > > > > > > time
> > > > > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > reassignPartitions), via zk admin path. Thus
> > > > > > CreateTopics
> > > > > > > > etc
> > > > > > > > > > > > should
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> > > > > > > > jun@confluent.io>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I think we agreed that those admin requests
> > can
> > > > be
> > > > > > > issued
> > > > > > > > > to
> > > > > > > > > > > any
> > > > > > > > > > > > > > > broker.
> > > > > > > > > > > > > > > > > Because of that, there doesn't seem to be a
> > > > strong
> > > > > > need
> > > > > > > > to
> > > > > > > > > > know
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > controller. So, perhaps we can proceed by
> not
> > > > > making
> > > > > > > any
> > > > > > > > > > change
> > > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > format of TMR right now. When we start
> using
> > > > create
> > > > > > > topic
> > > > > > > > > > > request
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > producer, we will need a new version of TMR
> > > that
> > > > > > > doesn't
> > > > > > > > > > > trigger
> > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > As a first cut implementation, I think the
> > > broker
> > > > > can
> > > > > > > > just
> > > > > > > > > > > write
> > > > > > > > > > > > > to ZK
> > > > > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > >
> > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > > > > requests, instead of forwarding them to the
> > > > > > controller.
> > > > > > > > > This
> > > > > > > > > > > will
> > > > > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel
> Koshy
> > <
> > > > > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > For (1) yes we will circle back on that
> > > shortly
> > > > > > after
> > > > > > > > > > syncing
> > > > > > > > > > > > up
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > person. I think it is close to getting
> > > > committed
> > > > > > > > although
> > > > > > > > > > > > > development
> > > > > > > > > > > > > > > > > > for KAFKA-1927 can probably begin without
> > it.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > There is one more item we covered at the
> > > > hangout.
> > > > > > > i.e.,
> > > > > > > > > > > whether
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > want to add the coordinator to the topic
> > > > metadata
> > > > > > > > > response
> > > > > > > > > > or
> > > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > There are two reasons I think we should
> try
> > > and
> > > > > > avoid
> > > > > > > > > > adding
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > > > > > > > - If we finally do request rerouting in
> > Kafka
> > > > > then
> > > > > > > the
> > > > > > > > > > field
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > >   little to no value. (It still helps to
> > > have a
> > > > > > > > separate
> > > > > > > > > > > > > > > > > >   ClusterMetadataRequest to query for
> > > > > cluster-wide
> > > > > > > > > > > information
> > > > > > > > > > > > > such
> > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > >   'which broker is the controller?' as
> Joe
> > > > > > > mentioned.)
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I think it would be cleaner to have an
> > > explicit
> > > > > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > > > > that you can send to any broker in order
> to
> > > > > obtain
> > > > > > > the
> > > > > > > > > > > > controller
> > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > in the future possibly other cluster-wide
> > > > > > > > information). I
> > > > > > > > > > > think
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > main argument against doing this and
> > instead
> > > > > adding
> > > > > > > it
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > metadata response was convenience - i.e.,
> > you
> > > > > don't
> > > > > > > > have
> > > > > > > > > to
> > > > > > > > > > > > > discover
> > > > > > > > > > > > > > > > > > the controller in advance. However, I
> don't
> > > see
> > > > > > much
> > > > > > > > > actual
> > > > > > > > > > > > > > > > > > benefit/convenience in this and in fact
> > think
> > > > it
> > > > > > is a
> > > > > > > > > > > > non-issue.
> > > > > > > > > > > > > Let
> > > > > > > > > > > > > > > > > > me know if I'm overlooking something
> here.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > As an example, say we need to initiate
> > > > partition
> > > > > > > > > > reassignment
> > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > issuing the new ReassignPartitionsRequest
> > to
> > > > the
> > > > > > > > > controller
> > > > > > > > > > > > > (assume
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > already have the desired manual partition
> > > > > > > assignment).
> > > > > > > > > If
> > > > > > > > > > we
> > > > > > > > > > > > > are to
> > > > > > > > > > > > > > > > > > augment topic metadata response then the
> > flow
> > > > be
> > > > > > > > > something
> > > > > > > > > > > like
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > :
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > - Issue topic metadata request to any
> > broker
> > > > (and
> > > > > > > > > discover
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > > > > - Connect to controller if required
> (i.e.,
> > if
> > > > the
> > > > > > > > broker
> > > > > > > > > > > above
> > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > - Issue the partition reassignment
> request
> > to
> > > > the
> > > > > > > > > > controller.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > With an explicit cluster metadata request
> > it
> > > > > would
> > > > > > > be:
> > > > > > > > > > > > > > > > > > - Issue cluster metadata request to any
> > > broker
> > > > > > > > > > > > > > > > > > - Connect to controller if required
> (i.e.,
> > if
> > > > the
> > > > > > > > broker
> > > > > > > > > > > above
> > > > > > > > > > > > !=
> > > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > > - Issue the partition reassignment
> request
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > So it seems to add little practical value
> > and
> > > > > > bloats
> > > > > > > > > topic
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > The other angle to this is the following
> -
> > is
> > > > it
> > > > > a
> > > > > > > > matter
> > > > > > > > > > of
> > > > > > > > > > > > > naming?
> > > > > > > > > > > > > > > > > > Should we just rename topic metadata
> > > > > > request/response
> > > > > > > > to
> > > > > > > > > > just
> > > > > > > > > > > > > > > > > > MetadataRequest/Response and add cluster
> > > > metadata
> > > > > > to
> > > > > > > > it?
> > > > > > > > > By
> > > > > > > > > > > > that
> > > > > > > > > > > > > same
> > > > > > > > > > > > > > > > > > token should we also allow querying for
> the
> > > > > > consumer
> > > > > > > > > > > > coordinator
> > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > in future transaction coordinator) as
> well?
> > > > This
> > > > > > > leads
> > > > > > > > > to a
> > > > > > > > > > > > > bloated
> > > > > > > > > > > > > > > > > > request which isn't very appealing and
> > > > altogether
> > > > > > > > > > confusing.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700,
> > Jun
> > > > Rao
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 1. I just realized that in order to
> start
> > > > > working
> > > > > > > on
> > > > > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > need to merge the changes to
> > > > > OffsetCommitRequest
> > > > > > > > (from
> > > > > > > > > > > 0.8.2)
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > > > > This is planned to be done as part of
> > > > > KAFKA-1634.
> > > > > > > So,
> > > > > > > > > we
> > > > > > > > > > > will
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 2. Thinking about this a bit more, if
> the
> > > > > > semantic
> > > > > > > of
> > > > > > > > > > those
> > > > > > > > > > > > > "write"
> > > > > > > > > > > > > > > > > > > requests is async (i.e., after the
> client
> > > > gets
> > > > > a
> > > > > > > > > > response,
> > > > > > > > > > > it
> > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > > > > that the operation is initiated, but
> not
> > > > > > > necessarily
> > > > > > > > > > > > > completed), we
> > > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > > really need to forward the requests to
> > the
> > > > > > > > controller.
> > > > > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > > > > receiving broker can just write the
> > > operation
> > > > > to
> > > > > > ZK
> > > > > > > > as
> > > > > > > > > > the
> > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > > > > line tool previously does. This will
> > > simplify
> > > > > the
> > > > > > > > > > > > > implementation.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 8. There is another implementation
> detail
> > > for
> > > > > > > > describe
> > > > > > > > > > > topic.
> > > > > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > want to read the topic config from the
> > > broker
> > > > > > > cache,
> > > > > > > > > > > instead
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > > > > Currently, every broker reads the
> > > topic-level
> > > > > > > config
> > > > > > > > > for
> > > > > > > > > > > all
> > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > > However, it ignores those for topics
> not
> > > > hosted
> > > > > > on
> > > > > > > > > > itself.
> > > > > > > > > > > > So,
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > to change TopicConfigManager a bit so
> > that
> > > it
> > > > > > > caches
> > > > > > > > > the
> > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii
> > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala requests
> > > > objects,
> > > > > > use
> > > > > > > > java
> > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > > > > >     A: Gwen kindly took that
> > > (KAFKA-1927).
> > > > > It's
> > > > > > > > > > important
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > speed
> > > > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > > > > >          there since this ticket
> blocks
> > > > other
> > > > > > > > > important
> > > > > > > > > > > > > changes.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute facility vs
> > > client
> > > > > > > > > maintaining
> > > > > > > > > > > > > cluster
> > > > > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > > > > >     A: Jay has added pseudo code to
> > > > > KAFKA-1912
> > > > > > -
> > > > > > > > need
> > > > > > > > > > to
> > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > > > > >         easy to implement as a
> > > server-side
> > > > > > > feature
> > > > > > > > > > > > (comments
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 3. Q: Controller field in wire
> > protocol.
> > > > > > > > > > > > > > > > > > > >     A: This might be useful for
> > clients,
> > > > add
> > > > > > this
> > > > > > > > to
> > > > > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic creation from
> > TMR.
> > > > > > > > > > > > > > > > > > > >     A: I will add proposed by Jun
> > > solution
> > > > > > (using
> > > > > > > > > > > clientId
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs
> > > > grabbing
> > > > > > all
> > > > > > > > > > > protocol
> > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > > >     A: It was decided to try to
> gather
> > > all
> > > > > > > changes
> > > > > > > > to
> > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > > > > >         In case of TMR it worth
> > checking:
> > > > > > > > KAFKA-2020
> > > > > > > > > > and
> > > > > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to
> deserialize
> > > > > user's
> > > > > > > > input
> > > > > > > > > in
> > > > > > > > > > > CLI
> > > > > > > > > > > > > tool.
> > > > > > > > > > > > > > > > > > > >     A: Use jackson for that, /tools
> > > project
> > > > > is
> > > > > > a
> > > > > > > > > > separate
> > > > > > > > > > > > > jar so
> > > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs
> > > generic
> > > > > > status
> > > > > > > > > check
> > > > > > > > > > > > > command.
> > > > > > > > > > > > > > > > > > > >      A: For long-running requests
> like
> > > > > reassign
> > > > > > > > > > > partitions
> > > > > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > > > > >          it makes sense to introduce
> > it.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >  Please add, correct me if I missed
> > > > > something.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM,
> Andrii
> > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > You are right, I removed
> > > ClusterMetadata
> > > > > > > because
> > > > > > > > we
> > > > > > > > > > > have
> > > > > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > > > > what we need in TopicMetadata.
> Also,
> > as
> > > > Jay
> > > > > > > > pointed
> > > > > > > > > > out
> > > > > > > > > > > > > > > earlier,
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > would like to have "orthogonal"
> API,
> > > but
> > > > at
> > > > > > the
> > > > > > > > > same
> > > > > > > > > > > time
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > But I like your idea and even have
> > some
> > > > > other
> > > > > > > > > > arguments
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > > > > There is also DescribeTopicRequest
> > > which
> > > > > was
> > > > > > > > > proposed
> > > > > > > > > > > in
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > > > > it returns topic configs,
> partitions,
> > > > > > > replication
> > > > > > > > > > > factor
> > > > > > > > > > > > > plus
> > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > > > > leader replica. The later part is
> > > really
> > > > > > > already
> > > > > > > > > > there
> > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > > > > So again we'll have to add stuff to
> > > TMR,
> > > > > not
> > > > > > to
> > > > > > > > > > > duplicate
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > > newly added requests. However, this
> > way
> > > > > we'll
> > > > > > > end
> > > > > > > > > up
> > > > > > > > > > > with
> > > > > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > > > > request which returns cluster
> > metadata,
> > > > > topic
> > > > > > > > > > > replication
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > > plus partition replication data.
> > Seems
> > > > > > logical
> > > > > > > to
> > > > > > > > > > split
> > > > > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > > > > - ClusterMetadata (brokers +
> > > controller,
> > > > > > maybe
> > > > > > > > smth
> > > > > > > > > > > else)
> > > > > > > > > > > > > > > > > > > > > - TopicMetadata (topic info +
> > partition
> > > > > > > details)
> > > > > > > > > > > > > > > > > > > > > But since current TMR is involved
> in
> > > lots
> > > > > of
> > > > > > > > places
> > > > > > > > > > > > > (including
> > > > > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > > > as I understand) this might be very
> > > > serious
> > > > > > > > change
> > > > > > > > > > and
> > > > > > > > > > > it
> > > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > > > > sense to stick with current
> approach.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM,
> Joel
> > > > > Koshy <
> > > > > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> I may be missing some context but
> > > > > hopefully
> > > > > > > this
> > > > > > > > > > will
> > > > > > > > > > > > > also be
> > > > > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > > > > >> today: I thought the earlier
> > proposal
> > > > > where
> > > > > > > > there
> > > > > > > > > > was
> > > > > > > > > > > an
> > > > > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > > > > >> ClusterMetadata request was
> clearer
> > > and
> > > > > > > > explicit.
> > > > > > > > > > > During
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > >> this thread I think the conclusion
> > was
> > > > > that
> > > > > > > the
> > > > > > > > > main
> > > > > > > > > > > > need
> > > > > > > > > > > > > was
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> controller information and that
> can
> > be
> > > > > > rolled
> > > > > > > > into
> > > > > > > > > > the
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > >> response but that seems a bit
> > > irrelevant
> > > > > to
> > > > > > > > topic
> > > > > > > > > > > > > metadata.
> > > > > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > > > > >> think the full broker-list is also
> > > > > > irrelevant
> > > > > > > to
> > > > > > > > > > topic
> > > > > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > >> it is already there and in use. I
> > > think
> > > > > > there
> > > > > > > is
> > > > > > > > > > still
> > > > > > > > > > > > > room
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > > >> explicit ClusterMetadata request
> > since
> > > > > there
> > > > > > > may
> > > > > > > > > be
> > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > >> cluster-level information that we
> > may
> > > > want
> > > > > > to
> > > > > > > > add
> > > > > > > > > > over
> > > > > > > > > > > > > time
> > > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> have nothing to do with topic
> > > metadata).
> > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM
> > > > +0200,
> > > > > > > Andrii
> > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > 101. Okay, if you say that such
> > use
> > > > case
> > > > > > is
> > > > > > > > > > > > important. I
> > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > >> > using clientId for these
> purposes
> > is
> > > > > fine
> > > > > > -
> > > > > > > if
> > > > > > > > > we
> > > > > > > > > > > > > already
> > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > > > > >> > as part of all Wire protocol
> > > messages,
> > > > > why
> > > > > > > not
> > > > > > > > > use
> > > > > > > > > > > > that.
> > > > > > > > > > > > > > > > > > > > >> > I will update KIP-4 page if
> nobody
> > > has
> > > > > > other
> > > > > > > > > ideas
> > > > > > > > > > > > > (which
> > > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP
> > > > > > > accordingly.
> > > > > > > > I
> > > > > > > > > > > think
> > > > > > > > > > > > > we can
> > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > > > > >> > fine-grained error codes if some
> > > error
> > > > > > code
> > > > > > > > > > received
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > >> > won't give enough context to
> > return
> > > a
> > > > > > > > > descriptive
> > > > > > > > > > > > error
> > > > > > > > > > > > > > > > message
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > Look forward to discussing all
> > > > > outstanding
> > > > > > > > > issues
> > > > > > > > > > in
> > > > > > > > > > > > > detail
> > > > > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59
> PM,
> > > Jun
> > > > > Rao
> > > > > > <
> > > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > 101. There may be a use case
> > where
> > > > you
> > > > > > > only
> > > > > > > > > want
> > > > > > > > > > > the
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > > > > >> > > manually by admins. Currently,
> > you
> > > > can
> > > > > > do
> > > > > > > > that
> > > > > > > > > > by
> > > > > > > > > > > > > > > disabling
> > > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > creation and issue topic
> > creation
> > > > from
> > > > > > the
> > > > > > > > > > > > > TopicCommand.
> > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > > > > >> > > topic creation completely on
> the
> > > > > broker
> > > > > > > and
> > > > > > > > > > don't
> > > > > > > > > > > > > have a
> > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > >> > > between topic creation
> requests
> > > from
> > > > > the
> > > > > > > > > regular
> > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > > > > >> > > can't support manual topic
> > > creation
> > > > > any
> > > > > > > > more.
> > > > > > > > > I
> > > > > > > > > > > was
> > > > > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > > > > >> > > way of distinguishing the
> > clients
> > > > > making
> > > > > > > the
> > > > > > > > > > topic
> > > > > > > > > > > > > > > creation
> > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > >> > > using clientId. For example,
> the
> > > > admin
> > > > > > > tool
> > > > > > > > > can
> > > > > > > > > > > set
> > > > > > > > > > > > > it to
> > > > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > > > > >> > > admin and the broker can treat
> > > that
> > > > > > > clientId
> > > > > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > Also, there is a related
> > > discussion
> > > > in
> > > > > > > > > > KAFKA-2020.
> > > > > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > following in
> > > TopicMetadataResponse:
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > 1. If leader is not available,
> > we
> > > > set
> > > > > > the
> > > > > > > > > > > partition
> > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > > > > >> > > 2. If a non-leader replica is
> > not
> > > > > > > available,
> > > > > > > > > we
> > > > > > > > > > > take
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > >> > > the assigned replica list and
> > isr
> > > in
> > > > > the
> > > > > > > > > > response.
> > > > > > > > > > > > As
> > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > > > > >> > > doing that, we set the
> partition
> > > > level
> > > > > > > error
> > > > > > > > > > code
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > This has a few problems.
> First,
> > > > > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > > > > >> > > an error, at least for the
> > normal
> > > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > > > > >> > > to find out the leader.
> Second,
> > it
> > > > can
> > > > > > > > happen
> > > > > > > > > > that
> > > > > > > > > > > > > both
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > another replica are not
> > available
> > > at
> > > > > the
> > > > > > > > same
> > > > > > > > > > > time.
> > > > > > > > > > > > > There
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > >> > > to indicate both. Third, even
> > if a
> > > > > > replica
> > > > > > > > is
> > > > > > > > > > not
> > > > > > > > > > > > > > > available,
> > > > > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > > > > >> > > useful to return its replica
> id
> > > > since
> > > > > > some
> > > > > > > > > > clients
> > > > > > > > > > > > > (e.g.
> > > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > One way to address this issue
> is
> > > to
> > > > > > always
> > > > > > > > > > return
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > > > > >> > > leader, assigned replicas, and
> > isr
> > > > > > > > regardless
> > > > > > > > > of
> > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > > > > >> > > broker is live or not. Since
> we
> > > also
> > > > > > > return
> > > > > > > > > the
> > > > > > > > > > > list
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > client can figure out whether
> a
> > > > leader
> > > > > > or
> > > > > > > a
> > > > > > > > > > > replica
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > > > > >> > > accordingly. This way, we
> don't
> > > need
> > > > > to
> > > > > > > set
> > > > > > > > > the
> > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > > >> > > when the leader or a replica
> is
> > > not
> > > > > > > > available.
> > > > > > > > > > > This
> > > > > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > > > > >> > > protocol, but does change the
> > > > > semantics.
> > > > > > > > Since
> > > > > > > > > > we
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > > > > >> > > of TopicMetadataRequest here,
> we
> > > can
> > > > > > > > > potentially
> > > > > > > > > > > > > piggyback
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > 102.1 For those types of
> errors
> > > due
> > > > to
> > > > > > > > invalid
> > > > > > > > > > > > input,
> > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > >> > > guard it at parameter
> validation
> > > > time
> > > > > > and
> > > > > > > > > throw
> > > > > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > > > > >> > > without even sending the
> request
> > > to
> > > > > the
> > > > > > > > > broker?
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37
> > AM,
> > > > > Andrii
> > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > 101. If I understand you
> > > > correctly,
> > > > > > you
> > > > > > > > are
> > > > > > > > > > > saying
> > > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > > > > >> > > > will be ported to TMR_V1)
> > won't
> > > be
> > > > > > able
> > > > > > > to
> > > > > > > > > > > > > automatically
> > > > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > unconditionally remove topic
> > > > > creation
> > > > > > > from
> > > > > > > > > > > there).
> > > > > > > > > > > > > But
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > > > > >> > > > Ok, about your proposal: I'm
> > > not a
> > > > > big
> > > > > > > fan
> > > > > > > > > > too,
> > > > > > > > > > > > > when it
> > > > > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > > > > >> > > > clients directly in protocol
> > > > schema.
> > > > > > And
> > > > > > > > > also
> > > > > > > > > > > I'm
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > > > > >> > > > auto.create.topics.enable
> is a
> > > > > server
> > > > > > > side
> > > > > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > > > > >> > > > in future versions, add this
> > > > setting
> > > > > > to
> > > > > > > > > > producer
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > based
> > > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > > > > >> > > > UnknownTopic create topic
> > > > explicitly
> > > > > > by
> > > > > > > a
> > > > > > > > > > > separate
> > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because
> > we
> > > > want
> > > > > > to
> > > > > > > > > > support
> > > > > > > > > > > > > batching
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > > > > >> > > > want to give descriptive
> error
> > > > > > messages
> > > > > > > > for
> > > > > > > > > > > > clients.
> > > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > > > > >> > > > to construct such messages
> > (e.g.
> > > > > > > > AdminClient
> > > > > > > > > > > layer
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > > > > > >> > > > means two cases: either
> > invalid
> > > > > > number -
> > > > > > > > > e.g.
> > > > > > > > > > > -1;
> > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > > > > >> > > > partitions argument wasn't)
> -
> > I
> > > > > > wrapped
> > > > > > > > > > > responses
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > > > > >> > > > other ideas, this was just
> > > initial
> > > > > > > > version.
> > > > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll
> > change
> > > > > that
> > > > > > to
> > > > > > > > > > > probably
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16
> > PM,
> > > > Jun
> > > > > > > Rao <
> > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > 101. That's what I was
> > > thinking
> > > > > too,
> > > > > > > but
> > > > > > > > > it
> > > > > > > > > > > may
> > > > > > > > > > > > > not be
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > > > > >> > > > > we can let it not trigger
> > auto
> > > > > topic
> > > > > > > > > > creation.
> > > > > > > > > > > > > Then,
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > > > > UnknownTopicException,
> > > > > > > it
> > > > > > > > > can
> > > > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > > > > >> > > > > createTopicRequest for
> auto
> > > > topic
> > > > > > > > > creation.
> > > > > > > > > > On
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > never issue
> > > createTopicRequest.
> > > > > This
> > > > > > > > works
> > > > > > > > > > > when
> > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > > > > >> > > > > enabled on the broker
> side.
> > > > > > However, I
> > > > > > > > am
> > > > > > > > > > not
> > > > > > > > > > > > > sure how
> > > > > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > > > > >> > > > > when auto topic creation
> is
> > > > > disabled
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > > > broker
> > > > > > > > > > > > > > > side.
> > > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > want to have a way to
> > manually
> > > > > > create
> > > > > > > a
> > > > > > > > > > topic,
> > > > > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > >> > > > > commands. However, then we
> > > need
> > > > a
> > > > > > way
> > > > > > > to
> > > > > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > > > > >> > > > > issued from the producer
> > > clients
> > > > > and
> > > > > > > the
> > > > > > > > > > admin
> > > > > > > > > > > > > tools.
> > > > > > > > > > > > > > > > May
> > > > > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > >> > > > > new field in
> > > createTopicRequest
> > > > > and
> > > > > > > set
> > > > > > > > it
> > > > > > > > > > > > > differently
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > > >> > > > > client and the admin
> client.
> > > > > > However,
> > > > > > > I
> > > > > > > > am
> > > > > > > > > > not
> > > > > > > > > > > > > sure if
> > > > > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring
> existing
> > > > > > requests
> > > > > > > > is a
> > > > > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > > > > >> > > > > posted some comments in
> > > > > KAFKA-1927.
> > > > > > We
> > > > > > > > > will
> > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > > > > >> > > > > first, before adding the
> new
> > > > logic
> > > > > > in
> > > > > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > 102. About the
> AdminClient:
> > > > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird
> that
> > > we
> > > > > > return
> > > > > > > > > > > exception
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > we should either return
> > error
> > > > code
> > > > > > or
> > > > > > > > > throw
> > > > > > > > > > an
> > > > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > > > > >> > > > > 102.2. We probably
> shouldn't
> > > > > > > explicitly
> > > > > > > > > use
> > > > > > > > > > > the
> > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > > > > >> > > > > Not every request
> evolution
> > > > > requires
> > > > > > > an
> > > > > > > > > api
> > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at
> 4:08
> > > AM,
> > > > > > > Andrii
> > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > >
> andrii.biletskyi@stealth.ly
> > >
> > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks for you comments.
> > > > Answers
> > > > > > > > inline:
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > 100. There are a few
> > fields
> > > > such
> > > > > > as
> > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> PartitionsSerialized
> > > > that
> > > > > > are
> > > > > > > > > > > > represented
> > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > >> > > > > > > composite structures
> in
> > > > json.
> > > > > > > Could
> > > > > > > > we
> > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, now with Admin
> Client
> > > > this
> > > > > > > looks
> > > > > > > > a
> > > > > > > > > > bit
> > > > > > > > > > > > > weird.
> > > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > > > > >> > > > > > ReassignPartitionCommand
> > > > accepts
> > > > > > > input
> > > > > > > > > in
> > > > > > > > > > > > json,
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > > > > >> > > > > > interfaces unchanged,
> > where
> > > > > > > possible.
> > > > > > > > > > > > > > > > > > > > >> > > > > > If we port it to
> > > deserialized
> > > > > > > format,
> > > > > > > > in
> > > > > > > > > > CLI
> > > > > > > > > > > > > (/tools
> > > > > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > > > > >> > > > > > json library since
> /tools
> > is
> > > > > > written
> > > > > > > > in
> > > > > > > > > > java
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > > > > >> > > > > > provided by a user. Can
> we
> > > > > quickly
> > > > > > > > agree
> > > > > > > > > > on
> > > > > > > > > > > > what
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON,
> whatever)?
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> > > TopicMetadataRequest
> > > > > v1
> > > > > > > > still
> > > > > > > > > > > > trigger
> > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird
> now
> > > that
> > > > > we
> > > > > > > > have a
> > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought about how
> > the
> > > > new
> > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > > > > producer/consumer
> > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> > > ideally,
> > > > > we
> > > > > > > > don't
> > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger
> auto
> > > > topic
> > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > I agree, this strange
> > logic
> > > > > should
> > > > > > > be
> > > > > > > > > > fixed.
> > > > > > > > > > > > > I'm not
> > > > > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > > > > >> > > > > > correct me if I'm wrong,
> > but
> > > > it
> > > > > > > > doesn't
> > > > > > > > > > look
> > > > > > > > > > > > > like a
> > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > > > > >> > > > > > leverage AdminClient for
> > > that
> > > > in
> > > > > > > > > Producer
> > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant
> > getting
> > > > rid
> > > > > > of
> > > > > > > > > scala
> > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > HeartbeatRequestAndHeader
> > > > > > and
> > > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing
> when
> > > > > adding
> > > > > > > the
> > > > > > > > > new
> > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the long term
> > > plan
> > > > is
> > > > > > to
> > > > > > > > get
> > > > > > > > > > rid
> > > > > > > > > > > of
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > java request/response
> in
> > > the
> > > > > > > client.
> > > > > > > > > > Since
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > significant number of
> > new
> > > > > > > requests,
> > > > > > > > > > > perhaps
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing
> > > scala
> > > > > > > requests
> > > > > > > > > > first
> > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks like I
> > > > misunderstood
> > > > > > the
> > > > > > > > > point
> > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > > rework that. The only
> > thing
> > > is
> > > > > > that
> > > > > > > I
> > > > > > > > > > don't
> > > > > > > > > > > > see
> > > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > > > > >> > > > > > least one existing
> > protocol
> > > > > > message.
> > > > > > > > > Thus,
> > > > > > > > > > > as
> > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > > > > > > > >> > > > > > Re porting all existing
> > > RQ/RP
> > > > in
> > > > > > > this
> > > > > > > > > > patch.
> > > > > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > > > > > >> > > > > > requirement to have
> Admin
> > > KIP
> > > > > > done,
> > > > > > > > I'm
> > > > > > > > > > > afraid
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > > > > > >> > > > > > There are 13 protocol
> > > messages
> > > > > and
> > > > > > > all
> > > > > > > > > > that
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > > > > > >> > > > > > intensive manual
> testing,
> > > no?
> > > > > I'm
> > > > > > > > afraid
> > > > > > > > > > I'm
> > > > > > > > > > > > > not the
> > > > > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > > > > > >> > > > > > Kafka core internals :).
> > Let
> > > > me
> > > > > > know
> > > > > > > > > your
> > > > > > > > > > > > > thoughts
> > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at
> > 6:40
> > > > AM,
> > > > > > Jun
> > > > > > > > > Rao <
> > > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > 100. There are a few
> > > fields
> > > > > such
> > > > > > > as
> > > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > and
> PartitionsSerialized
> > > > that
> > > > > > are
> > > > > > > > > > > > represented
> > > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > > >> > > > > > > composite structures
> in
> > > > json.
> > > > > > > Could
> > > > > > > > we
> > > > > > > > > > > > flatten
> > > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> > > > TopicMetadataRequest
> > > > > > v1
> > > > > > > > > still
> > > > > > > > > > > > > trigger
> > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird
> now
> > > that
> > > > > we
> > > > > > > > have a
> > > > > > > > > > > > > separate
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > > >> > > > > > > you thought about how
> > the
> > > > new
> > > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > > > > producer/consumer
> > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> > > ideally,
> > > > > we
> > > > > > > > don't
> > > > > > > > > > want
> > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger
> auto
> > > > topic
> > > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant
> > > getting
> > > > > rid
> > > > > > > of
> > > > > > > > > > scala
> > > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > > HeartbeatRequestAndHeader
> > > > > > and
> > > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing
> when
> > > > > adding
> > > > > > > the
> > > > > > > > > new
> > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > However, the long term
> > > plan
> > > > is
> > > > > > to
> > > > > > > > get
> > > > > > > > > > rid
> > > > > > > > > > > of
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > java request/response
> in
> > > the
> > > > > > > client.
> > > > > > > > > > Since
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > significant number of
> > new
> > > > > > > requests,
> > > > > > > > > > > perhaps
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing
> > > scala
> > > > > > > requests
> > > > > > > > > > first
> > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015
> at
> > > 3:37
> > > > > PM,
> > > > > > > > > Andrii
> > > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > andrii.biletskyi@stealth.ly
> > > > >
> > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > As said above - I
> list
> > > > again
> > > > > > all
> > > > > > > > > > > comments
> > > > > > > > > > > > > from
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > can see what's left
> > and
> > > > > > finalize
> > > > > > > > all
> > > > > > > > > > > > pending
> > > > > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is much
> needed
> > > > > > > > > functionality,
> > > > > > > > > > > but
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > really think these
> > > > protocols
> > > > > > > > > through.
> > > > > > > > > > We
> > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > of well thought-out,
> > > > > > orthoganol
> > > > > > > > > apis.
> > > > > > > > > > > For
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > important to think
> > > through
> > > > > the
> > > > > > > end
> > > > > > > > > > state
> > > > > > > > > > > > > even if
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > won't implement in
> the
> > > > first
> > > > > > > > phase.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Definitely behind
> > > this.
> > > > > > Would
> > > > > > > > > > > > appreciate
> > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > how this can be
> > > improved.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's please
> please
> > > > > please
> > > > > > > wait
> > > > > > > > > > until
> > > > > > > > > > > > we
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > to the new java
> > protocol
> > > > > > > > > definitions.
> > > > > > > > > > If
> > > > > > > > > > > > we
> > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > objects that is just
> > > > > > generating
> > > > > > > > more
> > > > > > > > > > > work
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> latest
> > > > > patch -
> > > > > > > > > removed
> > > > > > > > > > > > scala
> > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This proposal
> > > > introduces
> > > > > a
> > > > > > > new
> > > > > > > > > type
> > > > > > > > > > > of
> > > > > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > inconsistent with
> > > > everything
> > > > > > > else
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > other marker value.
> > You
> > > > > could
> > > > > > > > argue
> > > > > > > > > > > either
> > > > > > > > > > > > > way
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > for consistency. For
> > > > clients
> > > > > > > that
> > > > > > > > > > > > > implemented
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > than our scala code
> > > these
> > > > > > basic
> > > > > > > > > > > primitives
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> latest
> > > > > patch -
> > > > > > > > > removed
> > > > > > > > > > > > > MaybeOf
> > > > > > > > > > > > > > > > type
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata:
> > This
> > > > > seems
> > > > > > > to
> > > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers, topics, and
> > > > > > > partitions. I
> > > > > > > > > > think
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> ClusterMetadataRequest
> > > (or
> > > > > > just
> > > > > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > controller. Or are
> > there
> > > > > other
> > > > > > > > > things
> > > > > > > > > > we
> > > > > > > > > > > > > could
> > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I agree. Updated
> > the
> > > > KIP.
> > > > > > > Let's
> > > > > > > > > > > extends
> > > > > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a
> tendency
> > to
> > > > try
> > > > > > to
> > > > > > > > > make a
> > > > > > > > > > > lot
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > particular nodes.
> This
> > > > adds
> > > > > a
> > > > > > > lot
> > > > > > > > of
> > > > > > > > > > > > burden
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > sounds easy but each
> > > > > discovery
> > > > > > > can
> > > > > > > > > > fail
> > > > > > > > > > > in
> > > > > > > > > > > > > many
> > > > > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > full state machine
> to
> > do
> > > > > > > right). I
> > > > > > > > > > think
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > commands and ideally
> > as
> > > > many
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > other
> > > > > > > > > > > > > apis
> > > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers and just
> > > redirect
> > > > to
> > > > > > the
> > > > > > > > > > > > controller
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > there would be a
> > general
> > > > way
> > > > > > to
> > > > > > > > > > > > encapsulate
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a very
> > > interesting
> > > > > > idea,
> > > > > > > > but
> > > > > > > > > > > seems
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > feature (like
> > > performance
> > > > > > > > > > > considerations,
> > > > > > > > > > > > > how
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > I believe this
> > shouldn't
> > > > be
> > > > > a
> > > > > > > > > blocker.
> > > > > > > > > > > If
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > point it won't
> affect
> > > > Admin
> > > > > > > > changes
> > > > > > > > > -
> > > > > > > > > > at
> > > > > > > > > > > > > least
> > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We should
> probably
> > > > > > normalize
> > > > > > > > the
> > > > > > > > > > key
> > > > > > > > > > > > > value
> > > > > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > than embedding a new
> > > > > > formatting.
> > > > > > > > So
> > > > > > > > > > two
> > > > > > > > > > > > > strings
> > > > > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > internal equals
> sign.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> latest
> > > > > patch -
> > > > > > > > > > > normalized
> > > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the
> > postcondition
> > > of
> > > > > > these
> > > > > > > > > APIs
> > > > > > > > > > > that
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > the command has been
> > > > > > completed?
> > > > > > > It
> > > > > > > > > is
> > > > > > > > > > a
> > > > > > > > > > > > lot
> > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > been completed so
> you
> > > know
> > > > > > that
> > > > > > > if
> > > > > > > > > you
> > > > > > > > > > > > > create a
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > it you won't get an
> > > > > exception
> > > > > > > > about
> > > > > > > > > > > there
> > > > > > > > > > > > > being
> > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: For long running
> > > > requests
> > > > > > > (like
> > > > > > > > > > > > reassign
> > > > > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > command has begun -
> so
> > > we
> > > > > > don't
> > > > > > > > > block
> > > > > > > > > > > the
> > > > > > > > > > > > > > > client.
> > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic commands, this
> > > will
> > > > be
> > > > > > > > > > refactored
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > immediately, since
> the
> > > > > > > Controller
> > > > > > > > > will
> > > > > > > > > > > > serve
> > > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up ticket
> > > > > KAFKA-1777).
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe topic
> and
> > > list
> > > > > > > topics
> > > > > > > > > > > > duplicate
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > request. Is there a
> > > reason
> > > > > to
> > > > > > > give
> > > > > > > > > > back
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > like if we just make
> > the
> > > > > > > > > > post-condition
> > > > > > > > > > > of
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic is deleted
> that
> > > will
> > > > > get
> > > > > > > rid
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > be much more
> > intuitive.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the
> latest
> > > > > patch -
> > > > > > > > > removed
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we
> consider
> > > > > batching
> > > > > > > > these
> > > > > > > > > > > > > requests?
> > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > allow multiple
> > > operations
> > > > to
> > > > > > be
> > > > > > > > > > batched.
> > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > we will get a lot of
> > > code
> > > > > that
> > > > > > > > does
> > > > > > > > > > > > > something
> > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > this code will work
> > > great
> > > > > when
> > > > > > > you
> > > > > > > > > > test
> > > > > > > > > > > > on 5
> > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > > > please
> > > > > > > check
> > > > > > > > > > "Topic
> > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I think we
> should
> > > also
> > > > > > > discuss
> > > > > > > > > how
> > > > > > > > > > > we
> > > > > > > > > > > > > want
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > client api for these
> > > > > > operations.
> > > > > > > > > > > Currently
> > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > is totally sketchy.
> I
> > > > think
> > > > > we
> > > > > > > > > > probably
> > > > > > > > > > > > need
> > > > > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > that exposes
> > > > administrative
> > > > > > > > > > > functionality.
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > properly test the
> new
> > > > apis,
> > > > > I
> > > > > > > > > suspect.
> > > > > > > > > > > We
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > > > please
> > > > > > > check
> > > > > > > > > > "Admin
> > > > > > > > > > > > > Client"
> > > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The other
> > > information
> > > > > that
> > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > information about
> > > > > > > partitions--how
> > > > > > > > > much
> > > > > > > > > > > > data
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > the segment offsets,
> > > what
> > > > is
> > > > > > the
> > > > > > > > > > log-end
> > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > the compaction
> point,
> > > > etc. I
> > > > > > > think
> > > > > > > > > > that
> > > > > > > > > > > > done
> > > > > > > > > > > > > > > right
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > successor to the
> very
> > > > > awkward
> > > > > > > > > > > > OffsetRequest
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > be resolved in a
> > > separate
> > > > > KIP
> > > > > > /
> > > > > > > > jira
> > > > > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 12. Generally we can
> > do
> > > > good
> > > > > > > error
> > > > > > > > > > > > handling
> > > > > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > messages. I.e.
> > generally
> > > > the
> > > > > > > > client
> > > > > > > > > > has
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > context
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > an error that the
> > topic
> > > > > > doesn't
> > > > > > > > > exist
> > > > > > > > > > to
> > > > > > > > > > > > say
> > > > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > than "error code 14"
> > (or
> > > > > > > > whatever).
> > > > > > > > > > > Maybe
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > this is hard? If we
> > want
> > > > to
> > > > > > add
> > > > > > > > > > > > server-side
> > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > need to do this in a
> > > > > > consistent
> > > > > > > > way
> > > > > > > > > > > across
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > > > please
> > > > > > > check
> > > > > > > > > > > > "Protocol
> > > > > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > comprehensive,
> > > > fine-grained
> > > > > > list
> > > > > > > > of
> > > > > > > > > > > error
> > > > > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from
> > Guozhang:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe topic
> > > > request:
> > > > > it
> > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > > great to
> > > > > > > > > > > > > > > > go
> > > > > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex for
> > > this
> > > > > > > request.
> > > > > > > > > For
> > > > > > > > > > > > > example,
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > the topic command is
> > to
> > > > list
> > > > > > all
> > > > > > > > > > topics
> > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex
> then
> > we
> > > > > have
> > > > > > to
> > > > > > > > > first
> > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > description info and
> > > then
> > > > > > filter
> > > > > > > > at
> > > > > > > > > > the
> > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in
> > > create
> > > > > > topic:
> > > > > > > > > this
> > > > > > > > > > is
> > > > > > > > > > > > > related
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > maybe we can add
> > another
> > > > > > > metadata
> > > > > > > > > K-V
> > > > > > > > > > or
> > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > with config K-V in
> > > create
> > > > > > topic
> > > > > > > > like
> > > > > > > > > > we
> > > > > > > > > > > > did
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > field can be quite
> > > useful
> > > > in
> > > > > > > > storing
> > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > who issue the create
> > > > > command,
> > > > > > > etc,
> > > > > > > > > > which
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant
> setting.
> > > Then
> > > > > in
> > > > > > > the
> > > > > > > > > > > describe
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > on regex of the
> > metadata
> > > > > > field.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: As discussed it
> is
> > > very
> > > > > > > > > interesting
> > > > > > > > > > > but
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > we have some basic
> > > > > > functionality
> > > > > > > > > > there.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today all the
> > admin
> > > > > > > operations
> > > > > > > > > are
> > > > > > > > > > > > > async in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > return once it is
> > > written
> > > > in
> > > > > > ZK,
> > > > > > > > and
> > > > > > > > > > > that
> > > > > > > > > > > > > is why
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > > > > testUtil.waitForTopicCreated()
> > > > > > > > > /
> > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > request, etc. With
> > admin
> > > > > > > requests
> > > > > > > > we
> > > > > > > > > > > could
> > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > synchronous
> requests;
> > > when
> > > > > it
> > > > > > is
> > > > > > > > > > turned
> > > > > > > > > > > > on,
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > until the request
> has
> > > been
> > > > > > > > > completed.
> > > > > > > > > > > And
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > "token" field in the
> > > > > response,
> > > > > > > and
> > > > > > > > > > then
> > > > > > > > > > > > only
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > verification
> request"
> > > with
> > > > > the
> > > > > > > > given
> > > > > > > > > > > token
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I see your point.
> > My
> > > > idea
> > > > > > was
> > > > > > > > to
> > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > long running
> request,
> > > > where
> > > > > > > > needed.
> > > > > > > > > We
> > > > > > > > > > > can
> > > > > > > > > > > > > do it
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > concern is that
> > > > introducing
> > > > > a
> > > > > > > > token
> > > > > > > > > we
> > > > > > > > > > > > again
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > to do similar thing
> > > > > > introducing
> > > > > > > > > single
> > > > > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > this idea because we
> > > > wanted
> > > > > to
> > > > > > > > have
> > > > > > > > > > > schema
> > > > > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > a) have fixed schema
> > but
> > > > > > > introduce
> > > > > > > > > > each
> > > > > > > > > > > > > time new
> > > > > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > long-running
> requests
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > b) use one request
> for
> > > > > > > > verification
> > > > > > > > > > but
> > > > > > > > > > > > > > > generalize
> > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine with
> whatever
> > > > > > decision
> > > > > > > > > > > community
> > > > > > > > > > > > > come
> > > > > > > > > > > > > > > to.
> > > > > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > 16. Specifically for
> > > > > > ownership,
> > > > > > > I
> > > > > > > > > > think
> > > > > > > > > > > > the
> > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > like you are
> > describing
> > > > ACL)
> > > > > > via
> > > > > > > > an
> > > > > > > > > > > > external
> > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > I remember KIP-11
> > > > described
> > > > > > > this,
> > > > > > > > > but
> > > > > > > > > > I
> > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no problem.
> > Not
> > > > > sure
> > > > > > > > though
> > > > > > > > > > how
> > > > > > > > > > > > we
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > will be committed
> > first
> > > > and
> > > > > > > > include
> > > > > > > > > > > > changes
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I added this
> > > note
> > > > to
> > > > > > > "Open
> > > > > > > > > > > > > Questions"
> > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015
> > at
> > > > > 12:34
> > > > > > > AM,
> > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > andrii.biletskyi@stealth.ly
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Today I uploaded
> the
> > > > patch
> > > > > > > that
> > > > > > > > > > covers
> > > > > > > > > > > > > some of
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf
> > > > optional
> > > > > > > type
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - switched to java
> > > > > protocol
> > > > > > > > > > > definitions
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - simplified
> > messages
> > > > > > > > (normalized
> > > > > > > > > > > > configs,
> > > > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I also updated the
> > > KIP-4
> > > > > > with
> > > > > > > > > > > respective
> > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin
> > > Operations
> > > > > ->
> > > > > > > > > updated
> > > > > > > > > > > Wire
> > > > > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove
> > > ClusterMetadata
> > > > > ->
> > > > > > > > > changed
> > > > > > > > > > to
> > > > > > > > > > > > > extend
> > > > > > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin Client ->
> > > > updated
> > > > > my
> > > > > > > > > initial
> > > > > > > > > > > > > proposal
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Error codes ->
> > > > proposed
> > > > > > > > > > fine-grained
> > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > I will also send a
> > > > > separate
> > > > > > > > email
> > > > > > > > > to
> > > > > > > > > > > > > cover all
> > > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12,
> 2015
> > > at
> > > > > 9:26
> > > > > > > PM,
> > > > > > > > > > Gwen
> > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> It actually
> > specifies
> > > > > > changes
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are
> > > > consistent
> > > > > in
> > > > > > > > this
> > > > > > > > > > > regard
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12,
> > 2015
> > > at
> > > > > > 12:21
> > > > > > > > PM,
> > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Specifically
> for
> > > > > > > ownership, I
> > > > > > > > > > think
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > like you are
> > > > describing
> > > > > > > ACL)
> > > > > > > > > via
> > > > > > > > > > an
> > > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I remember
> KIP-11
> > > > > > described
> > > > > > > > > this,
> > > > > > > > > > > > but I
> > > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I
> > think
> > > > > KIP-4
> > > > > > > > > focuses
> > > > > > > > > > > on
> > > > > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > exists from
> Kafka
> > > > > > brokers,
> > > > > > > > not
> > > > > > > > > on
> > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > should exist
> but
> > > > > doesn't
> > > > > > > yet?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12,
> > > 2015
> > > > at
> > > > > > > 6:37
> > > > > > > > > AM,
> > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just want to
> > > > > elaborate a
> > > > > > > bit
> > > > > > > > > > more
> > > > > > > > > > > on
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> describe-topic
> > > based
> > > > > on
> > > > > > > > > config /
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694.
> > The
> > > > > main
> > > > > > > > > > motivation
> > > > > > > > > > > > is
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms,
> > which
> > > I
> > > > > > think
> > > > > > > is
> > > > > > > > > > quite
> > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> architecture:
> > > today
> > > > > > anyone
> > > > > > > > can
> > > > > > > > > > > > create
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> there is no
> > > concept
> > > > or
> > > > > > > > > > "ownership"
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> users. For
> > > example,
> > > > at
> > > > > > > > > LinkedIn
> > > > > > > > > > we
> > > > > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some casual
> > topic
> > > > name
> > > > > > > > prefix,
> > > > > > > > > > > which
> > > > > > > > > > > > > is a
> > > > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale our
> > > > > customers.
> > > > > > It
> > > > > > > > > would
> > > > > > > > > > > be
> > > > > > > > > > > > > great
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > > topics
> > > > > that
> > > > > > > is
> > > > > > > > > > > created
> > > > > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > > topics
> > > > > > whose
> > > > > > > > > > > retention
> > > > > > > > > > > > > time
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > > topics
> > > > > > whose
> > > > > > > > > > writable
> > > > > > > > > > > > > group
> > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> authorization),
> > > > etc..
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One possible
> way
> > > to
> > > > > > > achieve
> > > > > > > > > this
> > > > > > > > > > > is
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> create-topic
> > > > request,
> > > > > > > whose
> > > > > > > > > > value
> > > > > > > > > > > > will
> > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic; then
> > > > > > > describe-topics
> > > > > > > > > can
> > > > > > > > > > > > > choose to
> > > > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2)
> config
> > > K-V
> > > > > > > > matching,
> > > > > > > > > > 3)
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5,
> > > 2015
> > > > at
> > > > > > > 4:37
> > > > > > > > > PM,
> > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for
> the
> > > > > updated
> > > > > > > > wiki.
> > > > > > > > > A
> > > > > > > > > > > few
> > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error
> > > > description
> > > > > in
> > > > > > > > > > > response: I
> > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> several
> > different
> > > > > error
> > > > > > > > cases
> > > > > > > > > > > then
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In
> > general
> > > > the
> > > > > > > > > errorCode
> > > > > > > > > > > > > itself
> > > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> describing
> the
> > > > server
> > > > > > > side
> > > > > > > > > > > errors.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe
> > topic
> > > > > > > request:
> > > > > > > > it
> > > > > > > > > > > would
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name
> > regex
> > > > for
> > > > > > this
> > > > > > > > > > > request.
> > > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic
> > command
> > > > is
> > > > > to
> > > > > > > > list
> > > > > > > > > > all
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name
> > regex
> > > > then
> > > > > > we
> > > > > > > > have
> > > > > > > > > > to
> > > > > > > > > > > > > first
> > > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> description
> > info
> > > > and
> > > > > > then
> > > > > > > > > > filter
> > > > > > > > > > > at
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config
> K-Vs
> > in
> > > > > > create
> > > > > > > > > topic:
> > > > > > > > > > > > this
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can
> > add
> > > > > > another
> > > > > > > > > > metadata
> > > > > > > > > > > > > K-V or
> > > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with config
> K-V
> > > in
> > > > > > create
> > > > > > > > > topic
> > > > > > > > > > > > like
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field can be
> > > quite
> > > > > > useful
> > > > > > > > in
> > > > > > > > > > > > storing
> > > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue the
> > > > create
> > > > > > > > command,
> > > > > > > > > > > etc,
> > > > > > > > > > > > > which
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant
> > > > setting.
> > > > > > > Then
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of
> the
> > > > > > metadata
> > > > > > > > > field.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all
> > the
> > > > > admin
> > > > > > > > > > operations
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> return once
> it
> > is
> > > > > > written
> > > > > > > > in
> > > > > > > > > > ZK,
> > > > > > > > > > > > and
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> request, etc.
> > > With
> > > > > > admin
> > > > > > > > > > requests
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous
> > > > requests;
> > > > > > > when
> > > > > > > > it
> > > > > > > > > > is
> > > > > > > > > > > > > turned
> > > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until the
> > request
> > > > has
> > > > > > > been
> > > > > > > > > > > > > completed. And
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token" field
> > in
> > > > the
> > > > > > > > > response,
> > > > > > > > > > > and
> > > > > > > > > > > > > then
> > > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> verification
> > > > request"
> > > > > > > with
> > > > > > > > > the
> > > > > > > > > > > > given
> > > > > > > > > > > > > > > token
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has been
> > > completed.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for
> > > extending
> > > > > > > > Metadata
> > > > > > > > > > > > request
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> information,
> > and
> > > > then
> > > > > > we
> > > > > > > > can
> > > > > > > > > > > remove
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar
> 3,
> > > 2015
> > > > > at
> > > > > > > > 10:23
> > > > > > > > > > AM,
> > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for
> > > sending
> > > > > > that
> > > > > > > > out
> > > > > > > > > > Joe
> > > > > > > > > > > -
> > > > > > > > > > > > I
> > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so
> > if
> > > > > notes
> > > > > > > can
> > > > > > > > be
> > > > > > > > > > > sent
> > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar
> > 02,
> > > > 2015
> > > > > > at
> > > > > > > > > > > 09:16:13AM
> > > > > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for
> > > > sending
> > > > > > > this
> > > > > > > > > out
> > > > > > > > > > > Joe.
> > > > > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon,
> Mar
> > 2,
> > > > > 2015
> > > > > > at
> > > > > > > > > 6:46
> > > > > > > > > > > AM,
> > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I
> > just
> > > > sent
> > > > > > > out a
> > > > > > > > > > > google
> > > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> everyone I
> > > > found
> > > > > > > > working
> > > > > > > > > > on
> > > > > > > > > > > a
> > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me
> > know
> > > > and
> > > > > > can
> > > > > > > > > update
> > > > > > > > > > > it,
> > > > > > > > > > > > > np.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We
> should
> > do
> > > > > this
> > > > > > > > every
> > > > > > > > > > > > Tuesday
> > > > > > > > > > > > > @
> > > > > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to
> > > make a
> > > > > > > google
> > > > > > > > > > > account
> > > > > > > > > > > > > so we
> > > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To
> discuss
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in
> > progress
> > > > and
> > > > > > > > related
> > > > > > > > > > JIRA
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe
> > Stein
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue,
> > Feb
> > > > 24,
> > > > > > 2015
> > > > > > > > at
> > > > > > > > > > 2:59
> > > > > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com
> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's
> > stay
> > > on
> > > > > > > Google
> > > > > > > > > > > hangouts
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> available
> > > on
> > > > > > > youtube.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue,
> > Feb
> > > > 24,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > 11:49
> > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > jholoman@cloudera.com
> > > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay /
> > Joe
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're
> > > happy
> > > > > to
> > > > > > > send
> > > > > > > > > > out a
> > > > > > > > > > > > > Webex
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > sessions
> > > if
> > > > > > there
> > > > > > > > is
> > > > > > > > > > > > > interest and
> > > > > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> Thanks
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On
> Tue,
> > > Feb
> > > > > 24,
> > > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> Let's
> > > try
> > > > > to
> > > > > > > get
> > > > > > > > > the
> > > > > > > > > > > > > technical
> > > > > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> there
> > > is
> > > > > some
> > > > > > > > > benefit
> > > > > > > > > > > to
> > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we
> > post
> > > > > > > > > instructions
> > > > > > > > > > > and
> > > > > > > > > > > > > give
> > > > > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > working.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > Tuesday
> > > > at
> > > > > > that
> > > > > > > > > time
> > > > > > > > > > > > would
> > > > > > > > > > > > > work
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> -Jay
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On
> > Tue,
> > > > Feb
> > > > > > 24,
> > > > > > > > > 2015
> > > > > > > > > > at
> > > > > > > > > > > > > 8:18
> > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > Weekly
> > > > > > would
> > > > > > > be
> > > > > > > > > > great
> > > > > > > > > > > > > maybe
> > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I
> > > don't
> > > > > > mind
> > > > > > > > > google
> > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> whatever
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> we
> > > know
> > > > > the
> > > > > > > > > apache
> > > > > > > > > > > irc
> > > > > > > > > > > > > > > channel
> > > > > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > goes?
> > > > We
> > > > > > can
> > > > > > > > pull
> > > > > > > > > > > > > transcripts
> > > > > > > > > > > > > > > > too
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> makes
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> it
> > > > > helpful
> > > > > > > for
> > > > > > > > > > > things.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~
> > > > > Joestein
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> On
> > > Tue,
> > > > > Feb
> > > > > > > 24,
> > > > > > > > > > 2015
> > > > > > > > > > > at
> > > > > > > > > > > > > 11:10
> > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > jay.kreps@gmail.com
> > > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > We'd
> > > > > > talked
> > > > > > > > > about
> > > > > > > > > > > > > doing a
> > > > > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > generalizing
> > > > > > > > > > that a
> > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> for
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > everyone
> > > > > > > > > > spending a
> > > > > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > maybe
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > sync
> > > > up
> > > > > > > once
> > > > > > > > a
> > > > > > > > > > > week.
> > > > > > > > > > > > I
> > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > stuff,
> > > > > > make
> > > > > > > > > sure
> > > > > > > > > > we
> > > > > > > > > > > > > are on
> > > > > > > > > > > > > > > > top
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> tricky
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > issues,
> > > > > > > etc.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > We
> > > > can
> > > > > > make
> > > > > > > > it
> > > > > > > > > > > > publicly
> > > > > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > likes.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > Any
> > > > > > > interest
> > > > > > > > in
> > > > > > > > > > > doing
> > > > > > > > > > > > > this?
> > > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > week.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > -Jay
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > On
> > > > Tue,
> > > > > > Feb
> > > > > > > > 24,
> > > > > > > > > > > 2015
> > > > > > > > > > > > at
> > > > > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > >
> > > Hi
> > > > > all,
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > >
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > >
> > > > I've
> > > > > > > > updated
> > > > > > > > > > KIP
> > > > > > > > > > > > > page,
> > > > > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> added
> > > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > >
> > > > some
> > > > > > > > > > >
> > > > > > > > > > ...
> > > > > > > > > >
> > > > > > > > > > [Message clipped]
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > --
> > > > > > > > -- Guozhang
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Guozhang,

I'm not sure I understand how can we use those tools in CLI.
First of all, those are *Test*Utils, e.g. waitUntilMetadataIsPropagated
(which might be very useful for us) requires servers:Seq[KafkaServer]
as an argument:

def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String,
partition: Int, timeout: Long = 5000L): Int

how can we have access it at runtime in one of the brokers?

Secondly, how can user directly call these tools if zookeeper might be
not accessible (which is used in the tools, right?) at all - e.g. behind
the VPC
in AWS. Furthermore, I think with KIP-4 we are trying to create an
abstraction and
a single point of zookeeper interactions, to eliminate direct calls to ZK
either to get
some cluster information or to change something.

Thanks,
Andrii Biletskyi

On Sat, Mar 21, 2015 at 3:37 AM, Guozhang Wang <wa...@gmail.com> wrote:

> Andrii,
>
> Actually the checking logic Jun mentioned is already implemented as
> TestUtils.waitUntilXXX (LeaderIsElected, MetadataIsPropagated, etc...) I
> think we can extend these functions as CLI tools like TopicCommand so that
> users re-implementing such endpoint can directly call something like
> java.tools.WaitUntilXXX (of course this requires them to have javac
> installed, which should be a reasonable requirement?)
>
> Guozhang
>
>
>
> On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Jun,
> >
> > Not that I was saying we need to make requests blocking on server,
> > it was just to emphasize that with async requests a client
> implementations
> > may be a little bit more than just issue request - get the response.
> > Thanks for the explanation, I understand now that we can go with agreed
> > solution though it may not be perfect.
> > I believe this was one of the last controversial questions from the list.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> > On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Andrii,
> > >
> > > A few points.
> > >
> > > 1. Create/Alter can typically complete quickly. So, it's possible to
> make
> > > the request block until it's completed. However, currently, doing this
> at
> > > the broker is a bit involved. To make Create block, we will need to add
> > > some callbacks in KafkaController. This is possible. However, the
> > > controller logic currently is pretty completed. It would probably be
> > better
> > > if we clean it up first before adding more complexity to it. Alter is
> > even
> > > trickier. Adding partition is currently handled through
> KafkaController.
> > So
> > > it can be dealt with in a similar way. However, Alter config is done
> > > completely differently. It doesn't go through the controller. Instead,
> > each
> > > broker listens to ZooKeeper directly. So, it's not clear if there is an
> > > easy way on the broker to figure out whether a config is applied on
> every
> > > broker.
> > >
> > > 2. Delete can potentially take long if a replica to be deleted is
> > offline.
> > > PreferredLeader/PartitionReassign can also take long. So, we can't
> really
> > > make those requests block on the broker.
> > >
> > > As you can see, at this moment it's not easy to make all admin requests
> > > block on the broker. So, if we want the blocking feature in the admin
> > > utility in the short term, doing the completion check at the admin
> client
> > > is probably an easier route, even though it may not be ideal.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Jun,
> > > >
> > > > I see your point. But wouldn't that lead to a "fat" client
> > > implementations?
> > > > Suppose someone would like to implement client for Admin Wire
> protocol.
> > > > Not only people will have to code quite complicated logic like "send
> > > > describe
> > > > request to each broker" (again state machin?) but it will also mean
> > > people
> > > > must understand internal kafka logic related to topic storage and how
> > > > information is propageted from the controller to brokers.
> > > > I see this like a dilemma between having a concise Wire Protocol and
> > > > self-sufficient API to make client implementations simple.
> > > > I don't have a win-win solution though.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > >
> > > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > For 1), 2) and 3), blocking would probably mean that the new
> metadata
> > > is
> > > > > propagated to every broker. To achieve that, the client can keep
> > > issuing
> > > > > the describe topic request to every broker until it sees the new
> > > metadata
> > > > > in the response.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Hm, actually the ticket you linked, Guozhang, brings as back
> > > > > > to the problem what should be considered a post-condition for
> > > > > > each of the admin commands.
> > > > > > In my understanding:
> > > > > >
> > > > > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > > > > (Not the controller picked up changes from zk and broadcasted
> > > > > > LeaderAndIsr and UpdateMetadata)
> > > > > >
> > > > > > 2) AlterTopic - same as 1) - broker changed assignment data
> > > > > > in zookeeper or created admin path for topic config change
> > > > > >
> > > > > > 3) DeleteTopic - admin path /admin/delete_topics is created
> > > > > >
> > > > > > 4) ReassignPartitions and PreferredReplica - corresponding admin
> > > > > > path is created
> > > > > >
> > > > > > Now what can be considered a completed operation from the
> client's
> > > > > > perspective?
> > > > > > 1) Topic is created once corresponding data is in zk
> > > > > > (I remember there were some thoughts that it'd be good to
> consider
> > > > > > topic created once all replicas receive information about it and
> > thus
> > > > > > clients can produce/consume from it, but as was discussed this
> > seems
> > > > > > to be a hard thing to do)
> > > > > >
> > > > > > 2) Probably same as 1), so right after AlterTopic is issued
> > > > > >
> > > > > > 3) The topic has been removed from /brokers/topics
> > > > > >
> > > > > > 4) ReassignPartitions and PrefferedReplica were discussed
> earlier -
> > > > > > in short the former is completed once partition state info in zk
> > > > matches
> > > > > > reassignment request and admin path is empty, the latter - once
> > data
> > > > > > in zk shows that head of assignned replicas of the partition and
> > > leader
> > > > > > is the same replica
> > > > > >
> > > > > > Thoughts?
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <
> wangguoz@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > I think while loop is fine for supporting blocking, just that
> we
> > > need
> > > > > to
> > > > > > > add back off to avoid bombarding brokers with DescribeTopic
> > > requests.
> > > > > > >
> > > > > > > Also I have linked KAFKA-1125
> > > > > > > <https://issues.apache.org/jira/browse/KAFKA-1125> to your
> > > proposal,
> > > > > and
> > > > > > > when KAFKA-1694 is done this ticket can also be closed.
> > > > > > >
> > > > > > > Guozhang
> > > > > > >
> > > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >
> > > > > > > > Great.
> > > > > > > > I want to elaborate this a bit more, to see we are on the
> same
> > > page
> > > > > > > > concerning the client code.
> > > > > > > >
> > > > > > > > So with all topic commands being async a client (AdminClient
> in
> > > our
> > > > > > > > case or any other other client people would like to
> implement)
> > to
> > > > > > support
> > > > > > > > a blocking operation (which seems to be a natural use-case
> e.g.
> > > for
> > > > > > topic
> > > > > > > > creation): would have to do:
> > > > > > > > 1. issue CreateTopicRequest
> > > > > > > > 2. if successful, in a "while" loop send DescribeTopicRequest
> > and
> > > > > > > > break the loop once all topics are returned in response (or
> > upon
> > > > > > > timeout).
> > > > > > > > 3. if unsuccessful throw exception
> > > > > > > > Would it be okay?
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Andrii Biletskyi
> > > > > > > >
> > > > > > > >
> > > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > >
> > > > > > > > > Andrii,
> > > > > > > > >
> > > > > > > > > I think you are right. It seems that only
> ReassignPartitions
> > > > needs
> > > > > a
> > > > > > > > > separate verification request.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >
> > > > > > > > > > Guys,
> > > > > > > > > > I like this idea too. Let's stick with that. I'll update
> > KIP
> > > > > > > > accordingly.
> > > > > > > > > >
> > > > > > > > > > I was also thinking we can avoid adding dedicated status
> > > check
> > > > > > > > > > requests for topic commands. - We have everything in
> > > > > DescribeTopic
> > > > > > > > > > for that! E.g.:
> > > > > > > > > > User issued CreateTopic - to check the status client
> sends
> > > > > > > > DescribeTopic
> > > > > > > > > > and checks whether is something returned for that topic.
> > The
> > > > same
> > > > > > for
> > > > > > > > > > alteration, deletion.
> > > > > > > > > > Btw, PreferredReplica status can be also checked with
> > > > > > > > > DescribeTopicRequest
> > > > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > > > For ReassignPartitions as discussed we'll need to have a
> > > > separate
> > > > > > > > > Verify...
> > > > > > > > > > request.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Andrii Biletskyi
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> > > > > wangguoz@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > +1 on broker writing to ZK for async handling. I was
> > > thinking
> > > > > > that
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > end state the admin requests would be eventually sent
> to
> > > > > > controller
> > > > > > > > > > either
> > > > > > > > > > > through re-routing or clients discovering them, instead
> > of
> > > > > > letting
> > > > > > > > > > > controller listen on ZK admin path. But thinking about
> > it a
> > > > > > second
> > > > > > > > > time,
> > > > > > > > > > I
> > > > > > > > > > > think it is actually simpler to let controller manage
> > > > > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > > > > >
> > > > > > > > > > > Guozhang
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > > > > jjkoshy.w@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > +1 as well. I think it helps to keep the rerouting
> > > approach
> > > > > > > > > orthogonal
> > > > > > > > > > > > to this KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps
> > > wrote:
> > > > > > > > > > > > > I'm +1 on Jun's suggestion as long as it can work
> for
> > > all
> > > > > the
> > > > > > > > > > requests.
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <
> > > > jun@confluent.io
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > (a) Admin requests can be sent to and handled by
> > any
> > > > > > broker.
> > > > > > > > > > > > > > (b) Admin requests are processed asynchronously,
> at
> > > > least
> > > > > > for
> > > > > > > > > now.
> > > > > > > > > > > > That is,
> > > > > > > > > > > > > > when the client gets a response, it just means
> that
> > > the
> > > > > > > request
> > > > > > > > > is
> > > > > > > > > > > > > > initiated, but not necessarily completed. Then,
> > it's
> > > up
> > > > > to
> > > > > > > the
> > > > > > > > > > client
> > > > > > > > > > > > to
> > > > > > > > > > > > > > issue another request to check the status for
> > > > completion.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > To support (a), we were thinking of doing request
> > > > > > forwarding
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > > > controller (utilizing KAFKA-1912). I am making an
> > > > > > alternative
> > > > > > > > > > > proposal.
> > > > > > > > > > > > > > Basically, the broker can just write to ZooKeeper
> > to
> > > > > inform
> > > > > > > the
> > > > > > > > > > > > controller
> > > > > > > > > > > > > > about the request. For example, to handle
> > > > > > > > partitionReassignment,
> > > > > > > > > > the
> > > > > > > > > > > > broker
> > > > > > > > > > > > > > will just write the requested partitions to
> > > > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > > > (like what AdminUtils currently does) and then
> > send a
> > > > > > > response
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > > client. This shouldn't take long and the
> > > implementation
> > > > > > will
> > > > > > > be
> > > > > > > > > > > simpler
> > > > > > > > > > > > > > than forwarding the requests to the controller
> > > through
> > > > > RPC.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii
> Biletskyi <
> > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I might be wrong but didn't we agree we will
> let
> > > any
> > > > > > broker
> > > > > > > > > from
> > > > > > > > > > > the
> > > > > > > > > > > > > > > cluster handle *long-running* admin requests
> (at
> > > this
> > > > > > time
> > > > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > reassignPartitions), via zk admin path. Thus
> > > > > CreateTopics
> > > > > > > etc
> > > > > > > > > > > should
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> > > > > > > jun@confluent.io>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I think we agreed that those admin requests
> can
> > > be
> > > > > > issued
> > > > > > > > to
> > > > > > > > > > any
> > > > > > > > > > > > > > broker.
> > > > > > > > > > > > > > > > Because of that, there doesn't seem to be a
> > > strong
> > > > > need
> > > > > > > to
> > > > > > > > > know
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > controller. So, perhaps we can proceed by not
> > > > making
> > > > > > any
> > > > > > > > > change
> > > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > format of TMR right now. When we start using
> > > create
> > > > > > topic
> > > > > > > > > > request
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > producer, we will need a new version of TMR
> > that
> > > > > > doesn't
> > > > > > > > > > trigger
> > > > > > > > > > > > auto
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > As a first cut implementation, I think the
> > broker
> > > > can
> > > > > > > just
> > > > > > > > > > write
> > > > > > > > > > > > to ZK
> > > > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > >
> createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > > > requests, instead of forwarding them to the
> > > > > controller.
> > > > > > > > This
> > > > > > > > > > will
> > > > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy
> <
> > > > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > For (1) yes we will circle back on that
> > shortly
> > > > > after
> > > > > > > > > syncing
> > > > > > > > > > > up
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > person. I think it is close to getting
> > > committed
> > > > > > > although
> > > > > > > > > > > > development
> > > > > > > > > > > > > > > > > for KAFKA-1927 can probably begin without
> it.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > There is one more item we covered at the
> > > hangout.
> > > > > > i.e.,
> > > > > > > > > > whether
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > want to add the coordinator to the topic
> > > metadata
> > > > > > > > response
> > > > > > > > > or
> > > > > > > > > > > > provide
> > > > > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > There are two reasons I think we should try
> > and
> > > > > avoid
> > > > > > > > > adding
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > > > > > > - If we finally do request rerouting in
> Kafka
> > > > then
> > > > > > the
> > > > > > > > > field
> > > > > > > > > > > > would
> > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > >   little to no value. (It still helps to
> > have a
> > > > > > > separate
> > > > > > > > > > > > > > > > >   ClusterMetadataRequest to query for
> > > > cluster-wide
> > > > > > > > > > information
> > > > > > > > > > > > such
> > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > >   'which broker is the controller?' as Joe
> > > > > > mentioned.)
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I think it would be cleaner to have an
> > explicit
> > > > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > > > that you can send to any broker in order to
> > > > obtain
> > > > > > the
> > > > > > > > > > > controller
> > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > in the future possibly other cluster-wide
> > > > > > > information). I
> > > > > > > > > > think
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > main argument against doing this and
> instead
> > > > adding
> > > > > > it
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > metadata response was convenience - i.e.,
> you
> > > > don't
> > > > > > > have
> > > > > > > > to
> > > > > > > > > > > > discover
> > > > > > > > > > > > > > > > > the controller in advance. However, I don't
> > see
> > > > > much
> > > > > > > > actual
> > > > > > > > > > > > > > > > > benefit/convenience in this and in fact
> think
> > > it
> > > > > is a
> > > > > > > > > > > non-issue.
> > > > > > > > > > > > Let
> > > > > > > > > > > > > > > > > me know if I'm overlooking something here.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > As an example, say we need to initiate
> > > partition
> > > > > > > > > reassignment
> > > > > > > > > > > by
> > > > > > > > > > > > > > > > > issuing the new ReassignPartitionsRequest
> to
> > > the
> > > > > > > > controller
> > > > > > > > > > > > (assume
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > already have the desired manual partition
> > > > > > assignment).
> > > > > > > > If
> > > > > > > > > we
> > > > > > > > > > > > are to
> > > > > > > > > > > > > > > > > augment topic metadata response then the
> flow
> > > be
> > > > > > > > something
> > > > > > > > > > like
> > > > > > > > > > > > this
> > > > > > > > > > > > > > :
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > - Issue topic metadata request to any
> broker
> > > (and
> > > > > > > > discover
> > > > > > > > > > the
> > > > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > > > - Connect to controller if required (i.e.,
> if
> > > the
> > > > > > > broker
> > > > > > > > > > above
> > > > > > > > > > > !=
> > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > - Issue the partition reassignment request
> to
> > > the
> > > > > > > > > controller.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > With an explicit cluster metadata request
> it
> > > > would
> > > > > > be:
> > > > > > > > > > > > > > > > > - Issue cluster metadata request to any
> > broker
> > > > > > > > > > > > > > > > > - Connect to controller if required (i.e.,
> if
> > > the
> > > > > > > broker
> > > > > > > > > > above
> > > > > > > > > > > !=
> > > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > > - Issue the partition reassignment request
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > So it seems to add little practical value
> and
> > > > > bloats
> > > > > > > > topic
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > The other angle to this is the following -
> is
> > > it
> > > > a
> > > > > > > matter
> > > > > > > > > of
> > > > > > > > > > > > naming?
> > > > > > > > > > > > > > > > > Should we just rename topic metadata
> > > > > request/response
> > > > > > > to
> > > > > > > > > just
> > > > > > > > > > > > > > > > > MetadataRequest/Response and add cluster
> > > metadata
> > > > > to
> > > > > > > it?
> > > > > > > > By
> > > > > > > > > > > that
> > > > > > > > > > > > same
> > > > > > > > > > > > > > > > > token should we also allow querying for the
> > > > > consumer
> > > > > > > > > > > coordinator
> > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > in future transaction coordinator) as well?
> > > This
> > > > > > leads
> > > > > > > > to a
> > > > > > > > > > > > bloated
> > > > > > > > > > > > > > > > > request which isn't very appealing and
> > > altogether
> > > > > > > > > confusing.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700,
> Jun
> > > Rao
> > > > > > > wrote:
> > > > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 1. I just realized that in order to start
> > > > working
> > > > > > on
> > > > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > need to merge the changes to
> > > > OffsetCommitRequest
> > > > > > > (from
> > > > > > > > > > 0.8.2)
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > > > This is planned to be done as part of
> > > > KAFKA-1634.
> > > > > > So,
> > > > > > > > we
> > > > > > > > > > will
> > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 2. Thinking about this a bit more, if the
> > > > > semantic
> > > > > > of
> > > > > > > > > those
> > > > > > > > > > > > "write"
> > > > > > > > > > > > > > > > > > requests is async (i.e., after the client
> > > gets
> > > > a
> > > > > > > > > response,
> > > > > > > > > > it
> > > > > > > > > > > > just
> > > > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > > > that the operation is initiated, but not
> > > > > > necessarily
> > > > > > > > > > > > completed), we
> > > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > really need to forward the requests to
> the
> > > > > > > controller.
> > > > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > > > receiving broker can just write the
> > operation
> > > > to
> > > > > ZK
> > > > > > > as
> > > > > > > > > the
> > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > > > line tool previously does. This will
> > simplify
> > > > the
> > > > > > > > > > > > implementation.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 8. There is another implementation detail
> > for
> > > > > > > describe
> > > > > > > > > > topic.
> > > > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > want to read the topic config from the
> > broker
> > > > > > cache,
> > > > > > > > > > instead
> > > > > > > > > > > of
> > > > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > > > Currently, every broker reads the
> > topic-level
> > > > > > config
> > > > > > > > for
> > > > > > > > > > all
> > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > > However, it ignores those for topics not
> > > hosted
> > > > > on
> > > > > > > > > itself.
> > > > > > > > > > > So,
> > > > > > > > > > > > we
> > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > to change TopicConfigManager a bit so
> that
> > it
> > > > > > caches
> > > > > > > > the
> > > > > > > > > > > > configs
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii
> > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala requests
> > > objects,
> > > > > use
> > > > > > > java
> > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > > > >     A: Gwen kindly took that
> > (KAFKA-1927).
> > > > It's
> > > > > > > > > important
> > > > > > > > > > > to
> > > > > > > > > > > > > > speed
> > > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > > > >          there since this ticket blocks
> > > other
> > > > > > > > important
> > > > > > > > > > > > changes.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute facility vs
> > client
> > > > > > > > maintaining
> > > > > > > > > > > > cluster
> > > > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > > > >     A: Jay has added pseudo code to
> > > > KAFKA-1912
> > > > > -
> > > > > > > need
> > > > > > > > > to
> > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > > > >         easy to implement as a
> > server-side
> > > > > > feature
> > > > > > > > > > > (comments
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 3. Q: Controller field in wire
> protocol.
> > > > > > > > > > > > > > > > > > >     A: This might be useful for
> clients,
> > > add
> > > > > this
> > > > > > > to
> > > > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic creation from
> TMR.
> > > > > > > > > > > > > > > > > > >     A: I will add proposed by Jun
> > solution
> > > > > (using
> > > > > > > > > > clientId
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs
> > > grabbing
> > > > > all
> > > > > > > > > > protocol
> > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > > >     A: It was decided to try to gather
> > all
> > > > > > changes
> > > > > > > to
> > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > > > >         In case of TMR it worth
> checking:
> > > > > > > KAFKA-2020
> > > > > > > > > and
> > > > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to deserialize
> > > > user's
> > > > > > > input
> > > > > > > > in
> > > > > > > > > > CLI
> > > > > > > > > > > > tool.
> > > > > > > > > > > > > > > > > > >     A: Use jackson for that, /tools
> > project
> > > > is
> > > > > a
> > > > > > > > > separate
> > > > > > > > > > > > jar so
> > > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs
> > generic
> > > > > status
> > > > > > > > check
> > > > > > > > > > > > command.
> > > > > > > > > > > > > > > > > > >      A: For long-running requests like
> > > > reassign
> > > > > > > > > > partitions
> > > > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > > > >          it makes sense to introduce
> it.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >  Please add, correct me if I missed
> > > > something.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii
> > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > You are right, I removed
> > ClusterMetadata
> > > > > > because
> > > > > > > we
> > > > > > > > > > have
> > > > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > > > what we need in TopicMetadata. Also,
> as
> > > Jay
> > > > > > > pointed
> > > > > > > > > out
> > > > > > > > > > > > > > earlier,
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > would like to have "orthogonal" API,
> > but
> > > at
> > > > > the
> > > > > > > > same
> > > > > > > > > > time
> > > > > > > > > > > > we
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > But I like your idea and even have
> some
> > > > other
> > > > > > > > > arguments
> > > > > > > > > > > for
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > > > There is also DescribeTopicRequest
> > which
> > > > was
> > > > > > > > proposed
> > > > > > > > > > in
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > > > it returns topic configs, partitions,
> > > > > > replication
> > > > > > > > > > factor
> > > > > > > > > > > > plus
> > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > > > leader replica. The later part is
> > really
> > > > > > already
> > > > > > > > > there
> > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > > > So again we'll have to add stuff to
> > TMR,
> > > > not
> > > > > to
> > > > > > > > > > duplicate
> > > > > > > > > > > > some
> > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > newly added requests. However, this
> way
> > > > we'll
> > > > > > end
> > > > > > > > up
> > > > > > > > > > with
> > > > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > > > request which returns cluster
> metadata,
> > > > topic
> > > > > > > > > > replication
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > > plus partition replication data.
> Seems
> > > > > logical
> > > > > > to
> > > > > > > > > split
> > > > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > > > - ClusterMetadata (brokers +
> > controller,
> > > > > maybe
> > > > > > > smth
> > > > > > > > > > else)
> > > > > > > > > > > > > > > > > > > > - TopicMetadata (topic info +
> partition
> > > > > > details)
> > > > > > > > > > > > > > > > > > > > But since current TMR is involved in
> > lots
> > > > of
> > > > > > > places
> > > > > > > > > > > > (including
> > > > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > > as I understand) this might be very
> > > serious
> > > > > > > change
> > > > > > > > > and
> > > > > > > > > > it
> > > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel
> > > > Koshy <
> > > > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >> I may be missing some context but
> > > > hopefully
> > > > > > this
> > > > > > > > > will
> > > > > > > > > > > > also be
> > > > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > > > >> today: I thought the earlier
> proposal
> > > > where
> > > > > > > there
> > > > > > > > > was
> > > > > > > > > > an
> > > > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > > > >> ClusterMetadata request was clearer
> > and
> > > > > > > explicit.
> > > > > > > > > > During
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > >> this thread I think the conclusion
> was
> > > > that
> > > > > > the
> > > > > > > > main
> > > > > > > > > > > need
> > > > > > > > > > > > was
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > >> controller information and that can
> be
> > > > > rolled
> > > > > > > into
> > > > > > > > > the
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > >> response but that seems a bit
> > irrelevant
> > > > to
> > > > > > > topic
> > > > > > > > > > > > metadata.
> > > > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > > > >> think the full broker-list is also
> > > > > irrelevant
> > > > > > to
> > > > > > > > > topic
> > > > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > >> it is already there and in use. I
> > think
> > > > > there
> > > > > > is
> > > > > > > > > still
> > > > > > > > > > > > room
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > > >> explicit ClusterMetadata request
> since
> > > > there
> > > > > > may
> > > > > > > > be
> > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > >> cluster-level information that we
> may
> > > want
> > > > > to
> > > > > > > add
> > > > > > > > > over
> > > > > > > > > > > > time
> > > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > >> have nothing to do with topic
> > metadata).
> > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM
> > > +0200,
> > > > > > Andrii
> > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > 101. Okay, if you say that such
> use
> > > case
> > > > > is
> > > > > > > > > > > important. I
> > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > >> > using clientId for these purposes
> is
> > > > fine
> > > > > -
> > > > > > if
> > > > > > > > we
> > > > > > > > > > > > already
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > > > >> > as part of all Wire protocol
> > messages,
> > > > why
> > > > > > not
> > > > > > > > use
> > > > > > > > > > > that.
> > > > > > > > > > > > > > > > > > > >> > I will update KIP-4 page if nobody
> > has
> > > > > other
> > > > > > > > ideas
> > > > > > > > > > > > (which
> > > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP
> > > > > > accordingly.
> > > > > > > I
> > > > > > > > > > think
> > > > > > > > > > > > we can
> > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > > > >> > fine-grained error codes if some
> > error
> > > > > code
> > > > > > > > > received
> > > > > > > > > > > in
> > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > >> > won't give enough context to
> return
> > a
> > > > > > > > descriptive
> > > > > > > > > > > error
> > > > > > > > > > > > > > > message
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > Look forward to discussing all
> > > > outstanding
> > > > > > > > issues
> > > > > > > > > in
> > > > > > > > > > > > detail
> > > > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM,
> > Jun
> > > > Rao
> > > > > <
> > > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > > 101. There may be a use case
> where
> > > you
> > > > > > only
> > > > > > > > want
> > > > > > > > > > the
> > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > > > >> > > manually by admins. Currently,
> you
> > > can
> > > > > do
> > > > > > > that
> > > > > > > > > by
> > > > > > > > > > > > > > disabling
> > > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > >> > > creation and issue topic
> creation
> > > from
> > > > > the
> > > > > > > > > > > > TopicCommand.
> > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > > > >> > > topic creation completely on the
> > > > broker
> > > > > > and
> > > > > > > > > don't
> > > > > > > > > > > > have a
> > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > >> > > between topic creation requests
> > from
> > > > the
> > > > > > > > regular
> > > > > > > > > > > > clients
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > > > >> > > can't support manual topic
> > creation
> > > > any
> > > > > > > more.
> > > > > > > > I
> > > > > > > > > > was
> > > > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > > > >> > > way of distinguishing the
> clients
> > > > making
> > > > > > the
> > > > > > > > > topic
> > > > > > > > > > > > > > creation
> > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > >> > > using clientId. For example, the
> > > admin
> > > > > > tool
> > > > > > > > can
> > > > > > > > > > set
> > > > > > > > > > > > it to
> > > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > > > >> > > admin and the broker can treat
> > that
> > > > > > clientId
> > > > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >> > > Also, there is a related
> > discussion
> > > in
> > > > > > > > > KAFKA-2020.
> > > > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > >> > > following in
> > TopicMetadataResponse:
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >> > > 1. If leader is not available,
> we
> > > set
> > > > > the
> > > > > > > > > > partition
> > > > > > > > > > > > level
> > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > > > >> > > 2. If a non-leader replica is
> not
> > > > > > available,
> > > > > > > > we
> > > > > > > > > > take
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > >> > > the assigned replica list and
> isr
> > in
> > > > the
> > > > > > > > > response.
> > > > > > > > > > > As
> > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > > > >> > > doing that, we set the partition
> > > level
> > > > > > error
> > > > > > > > > code
> > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >> > > This has a few problems. First,
> > > > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > > > >> > > an error, at least for the
> normal
> > > > > > > > > > producer/consumer
> > > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > > > >> > > to find out the leader. Second,
> it
> > > can
> > > > > > > happen
> > > > > > > > > that
> > > > > > > > > > > > both
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >> > > another replica are not
> available
> > at
> > > > the
> > > > > > > same
> > > > > > > > > > time.
> > > > > > > > > > > > There
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > >> > > to indicate both. Third, even
> if a
> > > > > replica
> > > > > > > is
> > > > > > > > > not
> > > > > > > > > > > > > > available,
> > > > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > > > >> > > useful to return its replica id
> > > since
> > > > > some
> > > > > > > > > clients
> > > > > > > > > > > > (e.g.
> > > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >> > > One way to address this issue is
> > to
> > > > > always
> > > > > > > > > return
> > > > > > > > > > > the
> > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > > > >> > > leader, assigned replicas, and
> isr
> > > > > > > regardless
> > > > > > > > of
> > > > > > > > > > > > whether
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > > > >> > > broker is live or not. Since we
> > also
> > > > > > return
> > > > > > > > the
> > > > > > > > > > list
> > > > > > > > > > > > of
> > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > >> > > client can figure out whether a
> > > leader
> > > > > or
> > > > > > a
> > > > > > > > > > replica
> > > > > > > > > > > is
> > > > > > > > > > > > > > live
> > > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > > > >> > > accordingly. This way, we don't
> > need
> > > > to
> > > > > > set
> > > > > > > > the
> > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > > >> > > when the leader or a replica is
> > not
> > > > > > > available.
> > > > > > > > > > This
> > > > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > > > >> > > protocol, but does change the
> > > > semantics.
> > > > > > > Since
> > > > > > > > > we
> > > > > > > > > > > are
> > > > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > > > >> > > of TopicMetadataRequest here, we
> > can
> > > > > > > > potentially
> > > > > > > > > > > > piggyback
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >> > > 102.1 For those types of errors
> > due
> > > to
> > > > > > > invalid
> > > > > > > > > > > input,
> > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > >> > > guard it at parameter validation
> > > time
> > > > > and
> > > > > > > > throw
> > > > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > > > >> > > without even sending the request
> > to
> > > > the
> > > > > > > > broker?
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37
> AM,
> > > > Andrii
> > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly>
> > wrote:
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > >> > > > 101. If I understand you
> > > correctly,
> > > > > you
> > > > > > > are
> > > > > > > > > > saying
> > > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > > > >> > > > will be ported to TMR_V1)
> won't
> > be
> > > > > able
> > > > > > to
> > > > > > > > > > > > automatically
> > > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > >> > > > unconditionally remove topic
> > > > creation
> > > > > > from
> > > > > > > > > > there).
> > > > > > > > > > > > But
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > > > >> > > > Ok, about your proposal: I'm
> > not a
> > > > big
> > > > > > fan
> > > > > > > > > too,
> > > > > > > > > > > > when it
> > > > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > > > >> > > > clients directly in protocol
> > > schema.
> > > > > And
> > > > > > > > also
> > > > > > > > > > I'm
> > > > > > > > > > > > not
> > > > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > > > >> > > > auto.create.topics.enable is a
> > > > server
> > > > > > side
> > > > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > > > >> > > > in future versions, add this
> > > setting
> > > > > to
> > > > > > > > > producer
> > > > > > > > > > > and
> > > > > > > > > > > > > > based
> > > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > > > >> > > > UnknownTopic create topic
> > > explicitly
> > > > > by
> > > > > > a
> > > > > > > > > > separate
> > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because
> we
> > > want
> > > > > to
> > > > > > > > > support
> > > > > > > > > > > > batching
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > > > >> > > > want to give descriptive error
> > > > > messages
> > > > > > > for
> > > > > > > > > > > clients.
> > > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > > > >> > > > to construct such messages
> (e.g.
> > > > > > > AdminClient
> > > > > > > > > > layer
> > > > > > > > > > > > can
> > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > > > > >> > > > means two cases: either
> invalid
> > > > > number -
> > > > > > > > e.g.
> > > > > > > > > > -1;
> > > > > > > > > > > or
> > > > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > > > >> > > > partitions argument wasn't) -
> I
> > > > > wrapped
> > > > > > > > > > responses
> > > > > > > > > > > in
> > > > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > > > >> > > > other ideas, this was just
> > initial
> > > > > > > version.
> > > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll
> change
> > > > that
> > > > > to
> > > > > > > > > > probably
> > > > > > > > > > > > some
> > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16
> PM,
> > > Jun
> > > > > > Rao <
> > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > 101. That's what I was
> > thinking
> > > > too,
> > > > > > but
> > > > > > > > it
> > > > > > > > > > may
> > > > > > > > > > > > not be
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > > > >> > > > > we can let it not trigger
> auto
> > > > topic
> > > > > > > > > creation.
> > > > > > > > > > > > Then,
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > > > UnknownTopicException,
> > > > > > it
> > > > > > > > can
> > > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > > > >> > > > > createTopicRequest for auto
> > > topic
> > > > > > > > creation.
> > > > > > > > > On
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > > > >> > > > > never issue
> > createTopicRequest.
> > > > This
> > > > > > > works
> > > > > > > > > > when
> > > > > > > > > > > > auto
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > > > >> > > > > enabled on the broker side.
> > > > > However, I
> > > > > > > am
> > > > > > > > > not
> > > > > > > > > > > > sure how
> > > > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > > > >> > > > > when auto topic creation is
> > > > disabled
> > > > > > on
> > > > > > > > the
> > > > > > > > > > > broker
> > > > > > > > > > > > > > side.
> > > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > >> > > > > want to have a way to
> manually
> > > > > create
> > > > > > a
> > > > > > > > > topic,
> > > > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > >> > > > > commands. However, then we
> > need
> > > a
> > > > > way
> > > > > > to
> > > > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > > > >> > > > > issued from the producer
> > clients
> > > > and
> > > > > > the
> > > > > > > > > admin
> > > > > > > > > > > > tools.
> > > > > > > > > > > > > > > May
> > > > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > >> > > > > new field in
> > createTopicRequest
> > > > and
> > > > > > set
> > > > > > > it
> > > > > > > > > > > > differently
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > > >> > > > > client and the admin client.
> > > > > However,
> > > > > > I
> > > > > > > am
> > > > > > > > > not
> > > > > > > > > > > > sure if
> > > > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring existing
> > > > > requests
> > > > > > > is a
> > > > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > > > >> > > > > posted some comments in
> > > > KAFKA-1927.
> > > > > We
> > > > > > > > will
> > > > > > > > > > > > probably
> > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > > > >> > > > > first, before adding the new
> > > logic
> > > > > in
> > > > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird that
> > we
> > > > > return
> > > > > > > > > > exception
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > >> > > > > we should either return
> error
> > > code
> > > > > or
> > > > > > > > throw
> > > > > > > > > an
> > > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > > > >> > > > > 102.2. We probably shouldn't
> > > > > > explicitly
> > > > > > > > use
> > > > > > > > > > the
> > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > > > >> > > > > Not every request evolution
> > > > requires
> > > > > > an
> > > > > > > > api
> > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08
> > AM,
> > > > > > Andrii
> > > > > > > > > > > Biletskyi
> > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly
> >
> > > > wrote:
> > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > Thanks for you comments.
> > > Answers
> > > > > > > inline:
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > 100. There are a few
> fields
> > > such
> > > > > as
> > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized
> > > that
> > > > > are
> > > > > > > > > > > represented
> > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > >> > > > > > > composite structures in
> > > json.
> > > > > > Could
> > > > > > > we
> > > > > > > > > > > flatten
> > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > Yes, now with Admin Client
> > > this
> > > > > > looks
> > > > > > > a
> > > > > > > > > bit
> > > > > > > > > > > > weird.
> > > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > > > >> > > > > > ReassignPartitionCommand
> > > accepts
> > > > > > input
> > > > > > > > in
> > > > > > > > > > > json,
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > > > >> > > > > > interfaces unchanged,
> where
> > > > > > possible.
> > > > > > > > > > > > > > > > > > > >> > > > > > If we port it to
> > deserialized
> > > > > > format,
> > > > > > > in
> > > > > > > > > CLI
> > > > > > > > > > > > (/tools
> > > > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > > > >> > > > > > json library since /tools
> is
> > > > > written
> > > > > > > in
> > > > > > > > > java
> > > > > > > > > > > and
> > > > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > > > >> > > > > > provided by a user. Can we
> > > > quickly
> > > > > > > agree
> > > > > > > > > on
> > > > > > > > > > > what
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> > TopicMetadataRequest
> > > > v1
> > > > > > > still
> > > > > > > > > > > trigger
> > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now
> > that
> > > > we
> > > > > > > have a
> > > > > > > > > > > > separate
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > >> > > > > > > you thought about how
> the
> > > new
> > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > > > producer/consumer
> > > > > > > > > > > > client,
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> > ideally,
> > > > we
> > > > > > > don't
> > > > > > > > > want
> > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto
> > > topic
> > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > I agree, this strange
> logic
> > > > should
> > > > > > be
> > > > > > > > > fixed.
> > > > > > > > > > > > I'm not
> > > > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > > > >> > > > > > correct me if I'm wrong,
> but
> > > it
> > > > > > > doesn't
> > > > > > > > > look
> > > > > > > > > > > > like a
> > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > > > >> > > > > > leverage AdminClient for
> > that
> > > in
> > > > > > > > Producer
> > > > > > > > > > and
> > > > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant
> getting
> > > rid
> > > > > of
> > > > > > > > scala
> > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > HeartbeatRequestAndHeader
> > > > > and
> > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> > > > adding
> > > > > > the
> > > > > > > > new
> > > > > > > > > > > > requests
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > >> > > > > > > However, the long term
> > plan
> > > is
> > > > > to
> > > > > > > get
> > > > > > > > > rid
> > > > > > > > > > of
> > > > > > > > > > > > all
> > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > java request/response in
> > the
> > > > > > client.
> > > > > > > > > Since
> > > > > > > > > > > > this
> > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > >> > > > > > > significant number of
> new
> > > > > > requests,
> > > > > > > > > > perhaps
> > > > > > > > > > > we
> > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing
> > scala
> > > > > > requests
> > > > > > > > > first
> > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks like I
> > > misunderstood
> > > > > the
> > > > > > > > point
> > > > > > > > > of
> > > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > > > >> > > > > > rework that. The only
> thing
> > is
> > > > > that
> > > > > > I
> > > > > > > > > don't
> > > > > > > > > > > see
> > > > > > > > > > > > any
> > > > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > > > >> > > > > > least one existing
> protocol
> > > > > message.
> > > > > > > > Thus,
> > > > > > > > > > as
> > > > > > > > > > > I
> > > > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > > > > > > >> > > > > > Re porting all existing
> > RQ/RP
> > > in
> > > > > > this
> > > > > > > > > patch.
> > > > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > > > > >> > > > > > requirement to have Admin
> > KIP
> > > > > done,
> > > > > > > I'm
> > > > > > > > > > afraid
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > > > > >> > > > > > There are 13 protocol
> > messages
> > > > and
> > > > > > all
> > > > > > > > > that
> > > > > > > > > > > > would
> > > > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > > > > >> > > > > > intensive manual testing,
> > no?
> > > > I'm
> > > > > > > afraid
> > > > > > > > > I'm
> > > > > > > > > > > > not the
> > > > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > > > > >> > > > > > Kafka core internals :).
> Let
> > > me
> > > > > know
> > > > > > > > your
> > > > > > > > > > > > thoughts
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at
> 6:40
> > > AM,
> > > > > Jun
> > > > > > > > Rao <
> > > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > 100. There are a few
> > fields
> > > > such
> > > > > > as
> > > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized
> > > that
> > > > > are
> > > > > > > > > > > represented
> > > > > > > > > > > > as a
> > > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > > >> > > > > > > composite structures in
> > > json.
> > > > > > Could
> > > > > > > we
> > > > > > > > > > > flatten
> > > > > > > > > > > > > > them
> > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> > > TopicMetadataRequest
> > > > > v1
> > > > > > > > still
> > > > > > > > > > > > trigger
> > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now
> > that
> > > > we
> > > > > > > have a
> > > > > > > > > > > > separate
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > > >> > > > > > > you thought about how
> the
> > > new
> > > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > > > producer/consumer
> > > > > > > > > > > > client,
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> > ideally,
> > > > we
> > > > > > > don't
> > > > > > > > > want
> > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto
> > > topic
> > > > > > > > creation.
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant
> > getting
> > > > rid
> > > > > > of
> > > > > > > > > scala
> > > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > > >> > > > > > > like
> > > HeartbeatRequestAndHeader
> > > > > and
> > > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> > > > adding
> > > > > > the
> > > > > > > > new
> > > > > > > > > > > > requests
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > > >> > > > > > > However, the long term
> > plan
> > > is
> > > > > to
> > > > > > > get
> > > > > > > > > rid
> > > > > > > > > > of
> > > > > > > > > > > > all
> > > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > java request/response in
> > the
> > > > > > client.
> > > > > > > > > Since
> > > > > > > > > > > > this
> > > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > > >> > > > > > > significant number of
> new
> > > > > > requests,
> > > > > > > > > > perhaps
> > > > > > > > > > > we
> > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing
> > scala
> > > > > > requests
> > > > > > > > > first
> > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at
> > 3:37
> > > > PM,
> > > > > > > > Andrii
> > > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > andrii.biletskyi@stealth.ly
> > > >
> > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > As said above - I list
> > > again
> > > > > all
> > > > > > > > > > comments
> > > > > > > > > > > > from
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > > > >> > > > > > > > can see what's left
> and
> > > > > finalize
> > > > > > > all
> > > > > > > > > > > pending
> > > > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is much needed
> > > > > > > > functionality,
> > > > > > > > > > but
> > > > > > > > > > > > there
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > > > >> > > > > > > > really think these
> > > protocols
> > > > > > > > through.
> > > > > > > > > We
> > > > > > > > > > > > really
> > > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > > > >> > > > > > > > of well thought-out,
> > > > > orthoganol
> > > > > > > > apis.
> > > > > > > > > > For
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > > > >> > > > > > > > important to think
> > through
> > > > the
> > > > > > end
> > > > > > > > > state
> > > > > > > > > > > > even if
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > >> > > > > > > > won't implement in the
> > > first
> > > > > > > phase.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Definitely behind
> > this.
> > > > > Would
> > > > > > > > > > > appreciate
> > > > > > > > > > > > if
> > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > > > >> > > > > > > > how this can be
> > improved.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's please please
> > > > please
> > > > > > wait
> > > > > > > > > until
> > > > > > > > > > > we
> > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > > > >> > > > > > > > to the new java
> protocol
> > > > > > > > definitions.
> > > > > > > > > If
> > > > > > > > > > > we
> > > > > > > > > > > > add
> > > > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > > > >> > > > > > > > objects that is just
> > > > > generating
> > > > > > > more
> > > > > > > > > > work
> > > > > > > > > > > > for
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > > patch -
> > > > > > > > removed
> > > > > > > > > > > scala
> > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This proposal
> > > introduces
> > > > a
> > > > > > new
> > > > > > > > type
> > > > > > > > > > of
> > > > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > > > >> > > > > > > > inconsistent with
> > > everything
> > > > > > else
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > > > >> > > > > > > > other marker value.
> You
> > > > could
> > > > > > > argue
> > > > > > > > > > either
> > > > > > > > > > > > way
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > >> > > > > > > > for consistency. For
> > > clients
> > > > > > that
> > > > > > > > > > > > implemented
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > > > >> > > > > > > > than our scala code
> > these
> > > > > basic
> > > > > > > > > > primitives
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > > patch -
> > > > > > > > removed
> > > > > > > > > > > > MaybeOf
> > > > > > > > > > > > > > > type
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata:
> This
> > > > seems
> > > > > > to
> > > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers, topics, and
> > > > > > partitions. I
> > > > > > > > > think
> > > > > > > > > > > we
> > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > > > >> > > > > > > > ClusterMetadataRequest
> > (or
> > > > > just
> > > > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > > controller. Or are
> there
> > > > other
> > > > > > > > things
> > > > > > > > > we
> > > > > > > > > > > > could
> > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I agree. Updated
> the
> > > KIP.
> > > > > > Let's
> > > > > > > > > > extends
> > > > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a tendency
> to
> > > try
> > > > > to
> > > > > > > > make a
> > > > > > > > > > lot
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > > > >> > > > > > > > particular nodes. This
> > > adds
> > > > a
> > > > > > lot
> > > > > > > of
> > > > > > > > > > > burden
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > > > >> > > > > > > > sounds easy but each
> > > > discovery
> > > > > > can
> > > > > > > > > fail
> > > > > > > > > > in
> > > > > > > > > > > > many
> > > > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > > > >> > > > > > > > full state machine to
> do
> > > > > > right). I
> > > > > > > > > think
> > > > > > > > > > > we
> > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > > > >> > > > > > > > commands and ideally
> as
> > > many
> > > > > of
> > > > > > > the
> > > > > > > > > > other
> > > > > > > > > > > > apis
> > > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > > > >> > > > > > > > brokers and just
> > redirect
> > > to
> > > > > the
> > > > > > > > > > > controller
> > > > > > > > > > > > on
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > > > >> > > > > > > > there would be a
> general
> > > way
> > > > > to
> > > > > > > > > > > encapsulate
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a very
> > interesting
> > > > > idea,
> > > > > > > but
> > > > > > > > > > seems
> > > > > > > > > > > > there
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > > > >> > > > > > > > feature (like
> > performance
> > > > > > > > > > considerations,
> > > > > > > > > > > > how
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > > > >> > > > > > > > I believe this
> shouldn't
> > > be
> > > > a
> > > > > > > > blocker.
> > > > > > > > > > If
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > > > >> > > > > > > > point it won't affect
> > > Admin
> > > > > > > changes
> > > > > > > > -
> > > > > > > > > at
> > > > > > > > > > > > least
> > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We should probably
> > > > > normalize
> > > > > > > the
> > > > > > > > > key
> > > > > > > > > > > > value
> > > > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > >> > > > > > > > than embedding a new
> > > > > formatting.
> > > > > > > So
> > > > > > > > > two
> > > > > > > > > > > > strings
> > > > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > > patch -
> > > > > > > > > > normalized
> > > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the
> postcondition
> > of
> > > > > these
> > > > > > > > APIs
> > > > > > > > > > that
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > > >> > > > > > > > the command has been
> > > > > completed?
> > > > > > It
> > > > > > > > is
> > > > > > > > > a
> > > > > > > > > > > lot
> > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > > >> > > > > > > > been completed so you
> > know
> > > > > that
> > > > > > if
> > > > > > > > you
> > > > > > > > > > > > create a
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > >> > > > > > > > it you won't get an
> > > > exception
> > > > > > > about
> > > > > > > > > > there
> > > > > > > > > > > > being
> > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: For long running
> > > requests
> > > > > > (like
> > > > > > > > > > > reassign
> > > > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > > > > >> > > > > > > > command has begun - so
> > we
> > > > > don't
> > > > > > > > block
> > > > > > > > > > the
> > > > > > > > > > > > > > client.
> > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > > > >> > > > > > > > topic commands, this
> > will
> > > be
> > > > > > > > > refactored
> > > > > > > > > > > and
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > > > >> > > > > > > > immediately, since the
> > > > > > Controller
> > > > > > > > will
> > > > > > > > > > > serve
> > > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up ticket
> > > > KAFKA-1777).
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe topic and
> > list
> > > > > > topics
> > > > > > > > > > > duplicate
> > > > > > > > > > > > a
> > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > > > >> > > > > > > > request. Is there a
> > reason
> > > > to
> > > > > > give
> > > > > > > > > back
> > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > > > >> > > > > > > > like if we just make
> the
> > > > > > > > > post-condition
> > > > > > > > > > of
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > > topic is deleted that
> > will
> > > > get
> > > > > > rid
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > > need
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > >> > > > > > > > be much more
> intuitive.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > > patch -
> > > > > > > > removed
> > > > > > > > > > > > topics
> > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we consider
> > > > batching
> > > > > > > these
> > > > > > > > > > > > requests?
> > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > >> > > > > > > > allow multiple
> > operations
> > > to
> > > > > be
> > > > > > > > > batched.
> > > > > > > > > > > My
> > > > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > >> > > > > > > > we will get a lot of
> > code
> > > > that
> > > > > > > does
> > > > > > > > > > > > something
> > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > > > >> > > > > > > > this code will work
> > great
> > > > when
> > > > > > you
> > > > > > > > > test
> > > > > > > > > > > on 5
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > > please
> > > > > > check
> > > > > > > > > "Topic
> > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I think we should
> > also
> > > > > > discuss
> > > > > > > > how
> > > > > > > > > > we
> > > > > > > > > > > > want
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > > > >> > > > > > > > client api for these
> > > > > operations.
> > > > > > > > > > Currently
> > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > > > >> > > > > > > > is totally sketchy. I
> > > think
> > > > we
> > > > > > > > > probably
> > > > > > > > > > > need
> > > > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > > > >> > > > > > > > that exposes
> > > administrative
> > > > > > > > > > functionality.
> > > > > > > > > > > > We
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > >> > > > > > > > properly test the new
> > > apis,
> > > > I
> > > > > > > > suspect.
> > > > > > > > > > We
> > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > > please
> > > > > > check
> > > > > > > > > "Admin
> > > > > > > > > > > > Client"
> > > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The other
> > information
> > > > that
> > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > > really
> > > > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > >> > > > > > > > information about
> > > > > > partitions--how
> > > > > > > > much
> > > > > > > > > > > data
> > > > > > > > > > > > is
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > > > >> > > > > > > > the segment offsets,
> > what
> > > is
> > > > > the
> > > > > > > > > log-end
> > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > > > >> > > > > > > > the compaction point,
> > > etc. I
> > > > > > think
> > > > > > > > > that
> > > > > > > > > > > done
> > > > > > > > > > > > > > right
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > > successor to the very
> > > > awkward
> > > > > > > > > > > OffsetRequest
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > > > > >> > > > > > > > be resolved in a
> > separate
> > > > KIP
> > > > > /
> > > > > > > jira
> > > > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 12. Generally we can
> do
> > > good
> > > > > > error
> > > > > > > > > > > handling
> > > > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > > > > >> > > > > > > > messages. I.e.
> generally
> > > the
> > > > > > > client
> > > > > > > > > has
> > > > > > > > > > > the
> > > > > > > > > > > > > > > context
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > > > >> > > > > > > > an error that the
> topic
> > > > > doesn't
> > > > > > > > exist
> > > > > > > > > to
> > > > > > > > > > > say
> > > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > > >> > > > > > > > than "error code 14"
> (or
> > > > > > > whatever).
> > > > > > > > > > Maybe
> > > > > > > > > > > > there
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > > > >> > > > > > > > this is hard? If we
> want
> > > to
> > > > > add
> > > > > > > > > > > server-side
> > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > > > >> > > > > > > > need to do this in a
> > > > > consistent
> > > > > > > way
> > > > > > > > > > across
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > > please
> > > > > > check
> > > > > > > > > > > "Protocol
> > > > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > > comprehensive,
> > > fine-grained
> > > > > list
> > > > > > > of
> > > > > > > > > > error
> > > > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from
> Guozhang:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe topic
> > > request:
> > > > it
> > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > > great to
> > > > > > > > > > > > > > > go
> > > > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex for
> > this
> > > > > > request.
> > > > > > > > For
> > > > > > > > > > > > example,
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > > > >> > > > > > > > the topic command is
> to
> > > list
> > > > > all
> > > > > > > > > topics
> > > > > > > > > > > > whose
> > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex then
> we
> > > > have
> > > > > to
> > > > > > > > first
> > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > > > >> > > > > > > > description info and
> > then
> > > > > filter
> > > > > > > at
> > > > > > > > > the
> > > > > > > > > > > > client
> > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in
> > create
> > > > > topic:
> > > > > > > > this
> > > > > > > > > is
> > > > > > > > > > > > related
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > > > >> > > > > > > > maybe we can add
> another
> > > > > > metadata
> > > > > > > > K-V
> > > > > > > > > or
> > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > > > >> > > > > > > > with config K-V in
> > create
> > > > > topic
> > > > > > > like
> > > > > > > > > we
> > > > > > > > > > > did
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > > > >> > > > > > > > field can be quite
> > useful
> > > in
> > > > > > > storing
> > > > > > > > > > > > information
> > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > > > >> > > > > > > > who issue the create
> > > > command,
> > > > > > etc,
> > > > > > > > > which
> > > > > > > > > > > is
> > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant setting.
> > Then
> > > > in
> > > > > > the
> > > > > > > > > > describe
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > > > >> > > > > > > > on regex of the
> metadata
> > > > > field.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: As discussed it is
> > very
> > > > > > > > interesting
> > > > > > > > > > but
> > > > > > > > > > > > can
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > > > >> > > > > > > > we have some basic
> > > > > functionality
> > > > > > > > > there.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today all the
> admin
> > > > > > operations
> > > > > > > > are
> > > > > > > > > > > > async in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > > >> > > > > > > > return once it is
> > written
> > > in
> > > > > ZK,
> > > > > > > and
> > > > > > > > > > that
> > > > > > > > > > > > is why
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > > > testUtil.waitForTopicCreated()
> > > > > > > > /
> > > > > > > > > > > verify
> > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > > > >> > > > > > > > request, etc. With
> admin
> > > > > > requests
> > > > > > > we
> > > > > > > > > > could
> > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > > > >> > > > > > > > synchronous requests;
> > when
> > > > it
> > > > > is
> > > > > > > > > turned
> > > > > > > > > > > on,
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > > > >> > > > > > > > until the request has
> > been
> > > > > > > > completed.
> > > > > > > > > > And
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > > > >> > > > > > > > "token" field in the
> > > > response,
> > > > > > and
> > > > > > > > > then
> > > > > > > > > > > only
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > > > >> > > > > > > > verification request"
> > with
> > > > the
> > > > > > > given
> > > > > > > > > > token
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: I see your point.
> My
> > > idea
> > > > > was
> > > > > > > to
> > > > > > > > > > > provide
> > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > > > >> > > > > > > > long running request,
> > > where
> > > > > > > needed.
> > > > > > > > We
> > > > > > > > > > can
> > > > > > > > > > > > do it
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > > > >> > > > > > > > concern is that
> > > introducing
> > > > a
> > > > > > > token
> > > > > > > > we
> > > > > > > > > > > again
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > > > >> > > > > > > > to do similar thing
> > > > > introducing
> > > > > > > > single
> > > > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > > > > >> > > > > > > > this idea because we
> > > wanted
> > > > to
> > > > > > > have
> > > > > > > > > > schema
> > > > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > a) have fixed schema
> but
> > > > > > introduce
> > > > > > > > > each
> > > > > > > > > > > > time new
> > > > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > > > > > > > > > > >> > > > > > > > b) use one request for
> > > > > > > verification
> > > > > > > > > but
> > > > > > > > > > > > > > generalize
> > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine with whatever
> > > > > decision
> > > > > > > > > > community
> > > > > > > > > > > > come
> > > > > > > > > > > > > > to.
> > > > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > 16. Specifically for
> > > > > ownership,
> > > > > > I
> > > > > > > > > think
> > > > > > > > > > > the
> > > > > > > > > > > > plan
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > > > >> > > > > > > > like you are
> describing
> > > ACL)
> > > > > via
> > > > > > > an
> > > > > > > > > > > external
> > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > > > >> > > > > > > > I remember KIP-11
> > > described
> > > > > > this,
> > > > > > > > but
> > > > > > > > > I
> > > > > > > > > > > > can't
> > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no problem.
> Not
> > > > sure
> > > > > > > though
> > > > > > > > > how
> > > > > > > > > > > we
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > > > >> > > > > > > > will be committed
> first
> > > and
> > > > > > > include
> > > > > > > > > > > changes
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I added this
> > note
> > > to
> > > > > > "Open
> > > > > > > > > > > > Questions"
> > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015
> at
> > > > 12:34
> > > > > > AM,
> > > > > > > > > Andrii
> > > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > andrii.biletskyi@stealth.ly
> > > > >
> > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > Today I uploaded the
> > > patch
> > > > > > that
> > > > > > > > > covers
> > > > > > > > > > > > some of
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf
> > > optional
> > > > > > type
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > - switched to java
> > > > protocol
> > > > > > > > > > definitions
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > - simplified
> messages
> > > > > > > (normalized
> > > > > > > > > > > configs,
> > > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > I also updated the
> > KIP-4
> > > > > with
> > > > > > > > > > respective
> > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin
> > Operations
> > > > ->
> > > > > > > > updated
> > > > > > > > > > Wire
> > > > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove
> > ClusterMetadata
> > > > ->
> > > > > > > > changed
> > > > > > > > > to
> > > > > > > > > > > > extend
> > > > > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin Client ->
> > > updated
> > > > my
> > > > > > > > initial
> > > > > > > > > > > > proposal
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > - Error codes ->
> > > proposed
> > > > > > > > > fine-grained
> > > > > > > > > > > > error
> > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > I will also send a
> > > > separate
> > > > > > > email
> > > > > > > > to
> > > > > > > > > > > > cover all
> > > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015
> > at
> > > > 9:26
> > > > > > PM,
> > > > > > > > > Gwen
> > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> It actually
> specifies
> > > > > changes
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are
> > > consistent
> > > > in
> > > > > > > this
> > > > > > > > > > regard
> > > > > > > > > > > > will
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12,
> 2015
> > at
> > > > > 12:21
> > > > > > > PM,
> > > > > > > > > > Gwen
> > > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Specifically for
> > > > > > ownership, I
> > > > > > > > > think
> > > > > > > > > > > the
> > > > > > > > > > > > > > plan
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > like you are
> > > describing
> > > > > > ACL)
> > > > > > > > via
> > > > > > > > > an
> > > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I remember KIP-11
> > > > > described
> > > > > > > > this,
> > > > > > > > > > > but I
> > > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I
> think
> > > > KIP-4
> > > > > > > > focuses
> > > > > > > > > > on
> > > > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > exists from Kafka
> > > > > brokers,
> > > > > > > not
> > > > > > > > on
> > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > should exist but
> > > > doesn't
> > > > > > yet?
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12,
> > 2015
> > > at
> > > > > > 6:37
> > > > > > > > AM,
> > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just want to
> > > > elaborate a
> > > > > > bit
> > > > > > > > > more
> > > > > > > > > > on
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> describe-topic
> > based
> > > > on
> > > > > > > > config /
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694.
> The
> > > > main
> > > > > > > > > motivation
> > > > > > > > > > > is
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms,
> which
> > I
> > > > > think
> > > > > > is
> > > > > > > > > quite
> > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> architecture:
> > today
> > > > > anyone
> > > > > > > can
> > > > > > > > > > > create
> > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> there is no
> > concept
> > > or
> > > > > > > > > "ownership"
> > > > > > > > > > > of
> > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> users. For
> > example,
> > > at
> > > > > > > > LinkedIn
> > > > > > > > > we
> > > > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some casual
> topic
> > > name
> > > > > > > prefix,
> > > > > > > > > > which
> > > > > > > > > > > > is a
> > > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale our
> > > > customers.
> > > > > It
> > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > great
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > topics
> > > > that
> > > > > > is
> > > > > > > > > > created
> > > > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > topics
> > > > > whose
> > > > > > > > > > retention
> > > > > > > > > > > > time
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> > topics
> > > > > whose
> > > > > > > > > writable
> > > > > > > > > > > > group
> > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> authorization),
> > > etc..
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One possible way
> > to
> > > > > > achieve
> > > > > > > > this
> > > > > > > > > > is
> > > > > > > > > > > to
> > > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> create-topic
> > > request,
> > > > > > whose
> > > > > > > > > value
> > > > > > > > > > > will
> > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic; then
> > > > > > describe-topics
> > > > > > > > can
> > > > > > > > > > > > choose to
> > > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2) config
> > K-V
> > > > > > > matching,
> > > > > > > > > 3)
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5,
> > 2015
> > > at
> > > > > > 4:37
> > > > > > > > PM,
> > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for the
> > > > updated
> > > > > > > wiki.
> > > > > > > > A
> > > > > > > > > > few
> > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error
> > > description
> > > > in
> > > > > > > > > > response: I
> > > > > > > > > > > > think
> > > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> several
> different
> > > > error
> > > > > > > cases
> > > > > > > > > > then
> > > > > > > > > > > we
> > > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In
> general
> > > the
> > > > > > > > errorCode
> > > > > > > > > > > > itself
> > > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> describing the
> > > server
> > > > > > side
> > > > > > > > > > errors.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe
> topic
> > > > > > request:
> > > > > > > it
> > > > > > > > > > would
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name
> regex
> > > for
> > > > > this
> > > > > > > > > > request.
> > > > > > > > > > > > For
> > > > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic
> command
> > > is
> > > > to
> > > > > > > list
> > > > > > > > > all
> > > > > > > > > > > > topics
> > > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name
> regex
> > > then
> > > > > we
> > > > > > > have
> > > > > > > > > to
> > > > > > > > > > > > first
> > > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> description
> info
> > > and
> > > > > then
> > > > > > > > > filter
> > > > > > > > > > at
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs
> in
> > > > > create
> > > > > > > > topic:
> > > > > > > > > > > this
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can
> add
> > > > > another
> > > > > > > > > metadata
> > > > > > > > > > > > K-V or
> > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with config K-V
> > in
> > > > > create
> > > > > > > > topic
> > > > > > > > > > > like
> > > > > > > > > > > > we
> > > > > > > > > > > > > > did
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field can be
> > quite
> > > > > useful
> > > > > > > in
> > > > > > > > > > > storing
> > > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue the
> > > create
> > > > > > > command,
> > > > > > > > > > etc,
> > > > > > > > > > > > which
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant
> > > setting.
> > > > > > Then
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of the
> > > > > metadata
> > > > > > > > field.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all
> the
> > > > admin
> > > > > > > > > operations
> > > > > > > > > > > are
> > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> return once it
> is
> > > > > written
> > > > > > > in
> > > > > > > > > ZK,
> > > > > > > > > > > and
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> request, etc.
> > With
> > > > > admin
> > > > > > > > > requests
> > > > > > > > > > > we
> > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous
> > > requests;
> > > > > > when
> > > > > > > it
> > > > > > > > > is
> > > > > > > > > > > > turned
> > > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until the
> request
> > > has
> > > > > > been
> > > > > > > > > > > > completed. And
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token" field
> in
> > > the
> > > > > > > > response,
> > > > > > > > > > and
> > > > > > > > > > > > then
> > > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> verification
> > > request"
> > > > > > with
> > > > > > > > the
> > > > > > > > > > > given
> > > > > > > > > > > > > > token
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has been
> > completed.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for
> > extending
> > > > > > > Metadata
> > > > > > > > > > > request
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> information,
> and
> > > then
> > > > > we
> > > > > > > can
> > > > > > > > > > remove
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3,
> > 2015
> > > > at
> > > > > > > 10:23
> > > > > > > > > AM,
> > > > > > > > > > > Joel
> > > > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for
> > sending
> > > > > that
> > > > > > > out
> > > > > > > > > Joe
> > > > > > > > > > -
> > > > > > > > > > > I
> > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so
> if
> > > > notes
> > > > > > can
> > > > > > > be
> > > > > > > > > > sent
> > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar
> 02,
> > > 2015
> > > > > at
> > > > > > > > > > 09:16:13AM
> > > > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for
> > > sending
> > > > > > this
> > > > > > > > out
> > > > > > > > > > Joe.
> > > > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar
> 2,
> > > > 2015
> > > > > at
> > > > > > > > 6:46
> > > > > > > > > > AM,
> > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I
> just
> > > sent
> > > > > > out a
> > > > > > > > > > google
> > > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I
> > > found
> > > > > > > working
> > > > > > > > > on
> > > > > > > > > > a
> > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me
> know
> > > and
> > > > > can
> > > > > > > > update
> > > > > > > > > > it,
> > > > > > > > > > > > np.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We should
> do
> > > > this
> > > > > > > every
> > > > > > > > > > > Tuesday
> > > > > > > > > > > > @
> > > > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to
> > make a
> > > > > > google
> > > > > > > > > > account
> > > > > > > > > > > > so we
> > > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in
> progress
> > > and
> > > > > > > related
> > > > > > > > > JIRA
> > > > > > > > > > > > that
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe
> Stein
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue,
> Feb
> > > 24,
> > > > > 2015
> > > > > > > at
> > > > > > > > > 2:59
> > > > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's
> stay
> > on
> > > > > > Google
> > > > > > > > > > hangouts
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> available
> > on
> > > > > > youtube.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue,
> Feb
> > > 24,
> > > > > > 2015
> > > > > > > at
> > > > > > > > > > 11:49
> > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > jholoman@cloudera.com
> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay /
> Joe
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're
> > happy
> > > > to
> > > > > > send
> > > > > > > > > out a
> > > > > > > > > > > > Webex
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> sessions
> > if
> > > > > there
> > > > > > > is
> > > > > > > > > > > > interest and
> > > > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue,
> > Feb
> > > > 24,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's
> > try
> > > > to
> > > > > > get
> > > > > > > > the
> > > > > > > > > > > > technical
> > > > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there
> > is
> > > > some
> > > > > > > > benefit
> > > > > > > > > > to
> > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we
> post
> > > > > > > > instructions
> > > > > > > > > > and
> > > > > > > > > > > > give
> > > > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > working.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> Tuesday
> > > at
> > > > > that
> > > > > > > > time
> > > > > > > > > > > would
> > > > > > > > > > > > work
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On
> Tue,
> > > Feb
> > > > > 24,
> > > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > > 8:18
> > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > Weekly
> > > > > would
> > > > > > be
> > > > > > > > > great
> > > > > > > > > > > > maybe
> > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I
> > don't
> > > > > mind
> > > > > > > > google
> > > > > > > > > > > > hangout
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we
> > know
> > > > the
> > > > > > > > apache
> > > > > > > > > > irc
> > > > > > > > > > > > > > channel
> > > > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> goes?
> > > We
> > > > > can
> > > > > > > pull
> > > > > > > > > > > > transcripts
> > > > > > > > > > > > > > > too
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it
> > > > helpful
> > > > > > for
> > > > > > > > > > things.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~
> > > > Joestein
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On
> > Tue,
> > > > Feb
> > > > > > 24,
> > > > > > > > > 2015
> > > > > > > > > > at
> > > > > > > > > > > > 11:10
> > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > jay.kreps@gmail.com
> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > We'd
> > > > > talked
> > > > > > > > about
> > > > > > > > > > > > doing a
> > > > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > generalizing
> > > > > > > > > that a
> > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > everyone
> > > > > > > > > spending a
> > > > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> maybe
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > sync
> > > up
> > > > > > once
> > > > > > > a
> > > > > > > > > > week.
> > > > > > > > > > > I
> > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > stuff,
> > > > > make
> > > > > > > > sure
> > > > > > > > > we
> > > > > > > > > > > > are on
> > > > > > > > > > > > > > > top
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > issues,
> > > > > > etc.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> We
> > > can
> > > > > make
> > > > > > > it
> > > > > > > > > > > publicly
> > > > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > likes.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> Any
> > > > > > interest
> > > > > > > in
> > > > > > > > > > doing
> > > > > > > > > > > > this?
> > > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> week.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > -Jay
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> On
> > > Tue,
> > > > > Feb
> > > > > > > 24,
> > > > > > > > > > 2015
> > > > > > > > > > > at
> > > > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > Hi
> > > > all,
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > I've
> > > > > > > updated
> > > > > > > > > KIP
> > > > > > > > > > > > page,
> > > > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > some
> > > > > > > > > >
> > > > > > > > > ...
> > > > > > > > >
> > > > > > > > > [Message clipped]
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > -- Guozhang
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Guozhang Wang <wa...@gmail.com>.
Andrii,

Actually the checking logic Jun mentioned is already implemented as
TestUtils.waitUntilXXX (LeaderIsElected, MetadataIsPropagated, etc...) I
think we can extend these functions as CLI tools like TopicCommand so that
users re-implementing such endpoint can directly call something like
java.tools.WaitUntilXXX (of course this requires them to have javac
installed, which should be a reasonable requirement?)

Guozhang



On Fri, Mar 20, 2015 at 3:40 PM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Jun,
>
> Not that I was saying we need to make requests blocking on server,
> it was just to emphasize that with async requests a client implementations
> may be a little bit more than just issue request - get the response.
> Thanks for the explanation, I understand now that we can go with agreed
> solution though it may not be perfect.
> I believe this was one of the last controversial questions from the list.
>
> Thanks,
> Andrii Biletskyi
>
> On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <ju...@confluent.io> wrote:
>
> > Andrii,
> >
> > A few points.
> >
> > 1. Create/Alter can typically complete quickly. So, it's possible to make
> > the request block until it's completed. However, currently, doing this at
> > the broker is a bit involved. To make Create block, we will need to add
> > some callbacks in KafkaController. This is possible. However, the
> > controller logic currently is pretty completed. It would probably be
> better
> > if we clean it up first before adding more complexity to it. Alter is
> even
> > trickier. Adding partition is currently handled through KafkaController.
> So
> > it can be dealt with in a similar way. However, Alter config is done
> > completely differently. It doesn't go through the controller. Instead,
> each
> > broker listens to ZooKeeper directly. So, it's not clear if there is an
> > easy way on the broker to figure out whether a config is applied on every
> > broker.
> >
> > 2. Delete can potentially take long if a replica to be deleted is
> offline.
> > PreferredLeader/PartitionReassign can also take long. So, we can't really
> > make those requests block on the broker.
> >
> > As you can see, at this moment it's not easy to make all admin requests
> > block on the broker. So, if we want the blocking feature in the admin
> > utility in the short term, doing the completion check at the admin client
> > is probably an easier route, even though it may not be ideal.
> >
> > Thanks,
> >
> > Jun
> >
> > On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Jun,
> > >
> > > I see your point. But wouldn't that lead to a "fat" client
> > implementations?
> > > Suppose someone would like to implement client for Admin Wire protocol.
> > > Not only people will have to code quite complicated logic like "send
> > > describe
> > > request to each broker" (again state machin?) but it will also mean
> > people
> > > must understand internal kafka logic related to topic storage and how
> > > information is propageted from the controller to brokers.
> > > I see this like a dilemma between having a concise Wire Protocol and
> > > self-sufficient API to make client implementations simple.
> > > I don't have a win-win solution though.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > >
> > > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > For 1), 2) and 3), blocking would probably mean that the new metadata
> > is
> > > > propagated to every broker. To achieve that, the client can keep
> > issuing
> > > > the describe topic request to every broker until it sees the new
> > metadata
> > > > in the response.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Hm, actually the ticket you linked, Guozhang, brings as back
> > > > > to the problem what should be considered a post-condition for
> > > > > each of the admin commands.
> > > > > In my understanding:
> > > > >
> > > > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > > > (Not the controller picked up changes from zk and broadcasted
> > > > > LeaderAndIsr and UpdateMetadata)
> > > > >
> > > > > 2) AlterTopic - same as 1) - broker changed assignment data
> > > > > in zookeeper or created admin path for topic config change
> > > > >
> > > > > 3) DeleteTopic - admin path /admin/delete_topics is created
> > > > >
> > > > > 4) ReassignPartitions and PreferredReplica - corresponding admin
> > > > > path is created
> > > > >
> > > > > Now what can be considered a completed operation from the client's
> > > > > perspective?
> > > > > 1) Topic is created once corresponding data is in zk
> > > > > (I remember there were some thoughts that it'd be good to consider
> > > > > topic created once all replicas receive information about it and
> thus
> > > > > clients can produce/consume from it, but as was discussed this
> seems
> > > > > to be a hard thing to do)
> > > > >
> > > > > 2) Probably same as 1), so right after AlterTopic is issued
> > > > >
> > > > > 3) The topic has been removed from /brokers/topics
> > > > >
> > > > > 4) ReassignPartitions and PrefferedReplica were discussed earlier -
> > > > > in short the former is completed once partition state info in zk
> > > matches
> > > > > reassignment request and admin path is empty, the latter - once
> data
> > > > > in zk shows that head of assignned replicas of the partition and
> > leader
> > > > > is the same replica
> > > > >
> > > > > Thoughts?
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <wangguoz@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > I think while loop is fine for supporting blocking, just that we
> > need
> > > > to
> > > > > > add back off to avoid bombarding brokers with DescribeTopic
> > requests.
> > > > > >
> > > > > > Also I have linked KAFKA-1125
> > > > > > <https://issues.apache.org/jira/browse/KAFKA-1125> to your
> > proposal,
> > > > and
> > > > > > when KAFKA-1694 is done this ticket can also be closed.
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Great.
> > > > > > > I want to elaborate this a bit more, to see we are on the same
> > page
> > > > > > > concerning the client code.
> > > > > > >
> > > > > > > So with all topic commands being async a client (AdminClient in
> > our
> > > > > > > case or any other other client people would like to implement)
> to
> > > > > support
> > > > > > > a blocking operation (which seems to be a natural use-case e.g.
> > for
> > > > > topic
> > > > > > > creation): would have to do:
> > > > > > > 1. issue CreateTopicRequest
> > > > > > > 2. if successful, in a "while" loop send DescribeTopicRequest
> and
> > > > > > > break the loop once all topics are returned in response (or
> upon
> > > > > > timeout).
> > > > > > > 3. if unsuccessful throw exception
> > > > > > > Would it be okay?
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > >
> > > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Andrii,
> > > > > > > >
> > > > > > > > I think you are right. It seems that only ReassignPartitions
> > > needs
> > > > a
> > > > > > > > separate verification request.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >
> > > > > > > > > Guys,
> > > > > > > > > I like this idea too. Let's stick with that. I'll update
> KIP
> > > > > > > accordingly.
> > > > > > > > >
> > > > > > > > > I was also thinking we can avoid adding dedicated status
> > check
> > > > > > > > > requests for topic commands. - We have everything in
> > > > DescribeTopic
> > > > > > > > > for that! E.g.:
> > > > > > > > > User issued CreateTopic - to check the status client sends
> > > > > > > DescribeTopic
> > > > > > > > > and checks whether is something returned for that topic.
> The
> > > same
> > > > > for
> > > > > > > > > alteration, deletion.
> > > > > > > > > Btw, PreferredReplica status can be also checked with
> > > > > > > > DescribeTopicRequest
> > > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > > For ReassignPartitions as discussed we'll need to have a
> > > separate
> > > > > > > > Verify...
> > > > > > > > > request.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Andrii Biletskyi
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> > > > wangguoz@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > +1 on broker writing to ZK for async handling. I was
> > thinking
> > > > > that
> > > > > > in
> > > > > > > > the
> > > > > > > > > > end state the admin requests would be eventually sent to
> > > > > controller
> > > > > > > > > either
> > > > > > > > > > through re-routing or clients discovering them, instead
> of
> > > > > letting
> > > > > > > > > > controller listen on ZK admin path. But thinking about
> it a
> > > > > second
> > > > > > > > time,
> > > > > > > > > I
> > > > > > > > > > think it is actually simpler to let controller manage
> > > > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > > > >
> > > > > > > > > > Guozhang
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > > > jjkoshy.w@gmail.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > +1 as well. I think it helps to keep the rerouting
> > approach
> > > > > > > > orthogonal
> > > > > > > > > > > to this KIP.
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps
> > wrote:
> > > > > > > > > > > > I'm +1 on Jun's suggestion as long as it can work for
> > all
> > > > the
> > > > > > > > > requests.
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <
> > > jun@confluent.io
> > > > >
> > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Andrii,
> > > > > > > > > > > > >
> > > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > > >
> > > > > > > > > > > > > (a) Admin requests can be sent to and handled by
> any
> > > > > broker.
> > > > > > > > > > > > > (b) Admin requests are processed asynchronously, at
> > > least
> > > > > for
> > > > > > > > now.
> > > > > > > > > > > That is,
> > > > > > > > > > > > > when the client gets a response, it just means that
> > the
> > > > > > request
> > > > > > > > is
> > > > > > > > > > > > > initiated, but not necessarily completed. Then,
> it's
> > up
> > > > to
> > > > > > the
> > > > > > > > > client
> > > > > > > > > > > to
> > > > > > > > > > > > > issue another request to check the status for
> > > completion.
> > > > > > > > > > > > >
> > > > > > > > > > > > > To support (a), we were thinking of doing request
> > > > > forwarding
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > > > controller (utilizing KAFKA-1912). I am making an
> > > > > alternative
> > > > > > > > > > proposal.
> > > > > > > > > > > > > Basically, the broker can just write to ZooKeeper
> to
> > > > inform
> > > > > > the
> > > > > > > > > > > controller
> > > > > > > > > > > > > about the request. For example, to handle
> > > > > > > partitionReassignment,
> > > > > > > > > the
> > > > > > > > > > > broker
> > > > > > > > > > > > > will just write the requested partitions to
> > > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > > (like what AdminUtils currently does) and then
> send a
> > > > > > response
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > > client. This shouldn't take long and the
> > implementation
> > > > > will
> > > > > > be
> > > > > > > > > > simpler
> > > > > > > > > > > > > than forwarding the requests to the controller
> > through
> > > > RPC.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I might be wrong but didn't we agree we will let
> > any
> > > > > broker
> > > > > > > > from
> > > > > > > > > > the
> > > > > > > > > > > > > > cluster handle *long-running* admin requests (at
> > this
> > > > > time
> > > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > reassignPartitions), via zk admin path. Thus
> > > > CreateTopics
> > > > > > etc
> > > > > > > > > > should
> > > > > > > > > > > be
> > > > > > > > > > > > > > sent
> > > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> > > > > > jun@confluent.io>
> > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I think we agreed that those admin requests can
> > be
> > > > > issued
> > > > > > > to
> > > > > > > > > any
> > > > > > > > > > > > > broker.
> > > > > > > > > > > > > > > Because of that, there doesn't seem to be a
> > strong
> > > > need
> > > > > > to
> > > > > > > > know
> > > > > > > > > > the
> > > > > > > > > > > > > > > controller. So, perhaps we can proceed by not
> > > making
> > > > > any
> > > > > > > > change
> > > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > > > format of TMR right now. When we start using
> > create
> > > > > topic
> > > > > > > > > request
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > producer, we will need a new version of TMR
> that
> > > > > doesn't
> > > > > > > > > trigger
> > > > > > > > > > > auto
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > As a first cut implementation, I think the
> broker
> > > can
> > > > > > just
> > > > > > > > > write
> > > > > > > > > > > to ZK
> > > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > > >
> > > > > > > > > >
> > > > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > > requests, instead of forwarding them to the
> > > > controller.
> > > > > > > This
> > > > > > > > > will
> > > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> > > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > For (1) yes we will circle back on that
> shortly
> > > > after
> > > > > > > > syncing
> > > > > > > > > > up
> > > > > > > > > > > in
> > > > > > > > > > > > > > > > person. I think it is close to getting
> > committed
> > > > > > although
> > > > > > > > > > > development
> > > > > > > > > > > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > There is one more item we covered at the
> > hangout.
> > > > > i.e.,
> > > > > > > > > whether
> > > > > > > > > > > we
> > > > > > > > > > > > > > > > want to add the coordinator to the topic
> > metadata
> > > > > > > response
> > > > > > > > or
> > > > > > > > > > > provide
> > > > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > There are two reasons I think we should try
> and
> > > > avoid
> > > > > > > > adding
> > > > > > > > > > the
> > > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > > > > > - If we finally do request rerouting in Kafka
> > > then
> > > > > the
> > > > > > > > field
> > > > > > > > > > > would
> > > > > > > > > > > > > add
> > > > > > > > > > > > > > > >   little to no value. (It still helps to
> have a
> > > > > > separate
> > > > > > > > > > > > > > > >   ClusterMetadataRequest to query for
> > > cluster-wide
> > > > > > > > > information
> > > > > > > > > > > such
> > > > > > > > > > > > > as
> > > > > > > > > > > > > > > >   'which broker is the controller?' as Joe
> > > > > mentioned.)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > I think it would be cleaner to have an
> explicit
> > > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > > that you can send to any broker in order to
> > > obtain
> > > > > the
> > > > > > > > > > controller
> > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > in the future possibly other cluster-wide
> > > > > > information). I
> > > > > > > > > think
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > main argument against doing this and instead
> > > adding
> > > > > it
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > > metadata response was convenience - i.e., you
> > > don't
> > > > > > have
> > > > > > > to
> > > > > > > > > > > discover
> > > > > > > > > > > > > > > > the controller in advance. However, I don't
> see
> > > > much
> > > > > > > actual
> > > > > > > > > > > > > > > > benefit/convenience in this and in fact think
> > it
> > > > is a
> > > > > > > > > > non-issue.
> > > > > > > > > > > Let
> > > > > > > > > > > > > > > > me know if I'm overlooking something here.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > As an example, say we need to initiate
> > partition
> > > > > > > > reassignment
> > > > > > > > > > by
> > > > > > > > > > > > > > > > issuing the new ReassignPartitionsRequest to
> > the
> > > > > > > controller
> > > > > > > > > > > (assume
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > already have the desired manual partition
> > > > > assignment).
> > > > > > > If
> > > > > > > > we
> > > > > > > > > > > are to
> > > > > > > > > > > > > > > > augment topic metadata response then the flow
> > be
> > > > > > > something
> > > > > > > > > like
> > > > > > > > > > > this
> > > > > > > > > > > > > :
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - Issue topic metadata request to any broker
> > (and
> > > > > > > discover
> > > > > > > > > the
> > > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > > - Connect to controller if required (i.e., if
> > the
> > > > > > broker
> > > > > > > > > above
> > > > > > > > > > !=
> > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > - Issue the partition reassignment request to
> > the
> > > > > > > > controller.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > With an explicit cluster metadata request it
> > > would
> > > > > be:
> > > > > > > > > > > > > > > > - Issue cluster metadata request to any
> broker
> > > > > > > > > > > > > > > > - Connect to controller if required (i.e., if
> > the
> > > > > > broker
> > > > > > > > > above
> > > > > > > > > > !=
> > > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > > - Issue the partition reassignment request
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > So it seems to add little practical value and
> > > > bloats
> > > > > > > topic
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > The other angle to this is the following - is
> > it
> > > a
> > > > > > matter
> > > > > > > > of
> > > > > > > > > > > naming?
> > > > > > > > > > > > > > > > Should we just rename topic metadata
> > > > request/response
> > > > > > to
> > > > > > > > just
> > > > > > > > > > > > > > > > MetadataRequest/Response and add cluster
> > metadata
> > > > to
> > > > > > it?
> > > > > > > By
> > > > > > > > > > that
> > > > > > > > > > > same
> > > > > > > > > > > > > > > > token should we also allow querying for the
> > > > consumer
> > > > > > > > > > coordinator
> > > > > > > > > > > (and
> > > > > > > > > > > > > > > > in future transaction coordinator) as well?
> > This
> > > > > leads
> > > > > > > to a
> > > > > > > > > > > bloated
> > > > > > > > > > > > > > > > request which isn't very appealing and
> > altogether
> > > > > > > > confusing.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun
> > Rao
> > > > > > wrote:
> > > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 1. I just realized that in order to start
> > > working
> > > > > on
> > > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > need to merge the changes to
> > > OffsetCommitRequest
> > > > > > (from
> > > > > > > > > 0.8.2)
> > > > > > > > > > > to
> > > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > > This is planned to be done as part of
> > > KAFKA-1634.
> > > > > So,
> > > > > > > we
> > > > > > > > > will
> > > > > > > > > > > need
> > > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 2. Thinking about this a bit more, if the
> > > > semantic
> > > > > of
> > > > > > > > those
> > > > > > > > > > > "write"
> > > > > > > > > > > > > > > > > requests is async (i.e., after the client
> > gets
> > > a
> > > > > > > > response,
> > > > > > > > > it
> > > > > > > > > > > just
> > > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > > that the operation is initiated, but not
> > > > > necessarily
> > > > > > > > > > > completed), we
> > > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > really need to forward the requests to the
> > > > > > controller.
> > > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > > receiving broker can just write the
> operation
> > > to
> > > > ZK
> > > > > > as
> > > > > > > > the
> > > > > > > > > > > admin
> > > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > > line tool previously does. This will
> simplify
> > > the
> > > > > > > > > > > implementation.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 8. There is another implementation detail
> for
> > > > > > describe
> > > > > > > > > topic.
> > > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > want to read the topic config from the
> broker
> > > > > cache,
> > > > > > > > > instead
> > > > > > > > > > of
> > > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > > Currently, every broker reads the
> topic-level
> > > > > config
> > > > > > > for
> > > > > > > > > all
> > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > > However, it ignores those for topics not
> > hosted
> > > > on
> > > > > > > > itself.
> > > > > > > > > > So,
> > > > > > > > > > > we
> > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > to change TopicConfigManager a bit so that
> it
> > > > > caches
> > > > > > > the
> > > > > > > > > > > configs
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii
> > > > Biletskyi <
> > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala requests
> > objects,
> > > > use
> > > > > > java
> > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > > >     A: Gwen kindly took that
> (KAFKA-1927).
> > > It's
> > > > > > > > important
> > > > > > > > > > to
> > > > > > > > > > > > > speed
> > > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > > >          there since this ticket blocks
> > other
> > > > > > > important
> > > > > > > > > > > changes.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute facility vs
> client
> > > > > > > maintaining
> > > > > > > > > > > cluster
> > > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > > >     A: Jay has added pseudo code to
> > > KAFKA-1912
> > > > -
> > > > > > need
> > > > > > > > to
> > > > > > > > > > > consider
> > > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > > >         easy to implement as a
> server-side
> > > > > feature
> > > > > > > > > > (comments
> > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > > > > > > > > > > >     A: This might be useful for clients,
> > add
> > > > this
> > > > > > to
> > > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > > > > > > > > > > >     A: I will add proposed by Jun
> solution
> > > > (using
> > > > > > > > > clientId
> > > > > > > > > > > for
> > > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs
> > grabbing
> > > > all
> > > > > > > > > protocol
> > > > > > > > > > > > > changes
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > > >     A: It was decided to try to gather
> all
> > > > > changes
> > > > > > to
> > > > > > > > > > > protocol
> > > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > > >         In case of TMR it worth checking:
> > > > > > KAFKA-2020
> > > > > > > > and
> > > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to deserialize
> > > user's
> > > > > > input
> > > > > > > in
> > > > > > > > > CLI
> > > > > > > > > > > tool.
> > > > > > > > > > > > > > > > > >     A: Use jackson for that, /tools
> project
> > > is
> > > > a
> > > > > > > > separate
> > > > > > > > > > > jar so
> > > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs
> generic
> > > > status
> > > > > > > check
> > > > > > > > > > > command.
> > > > > > > > > > > > > > > > > >      A: For long-running requests like
> > > reassign
> > > > > > > > > partitions
> > > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > > >          it makes sense to introduce it.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >  Please add, correct me if I missed
> > > something.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii
> > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > You are right, I removed
> ClusterMetadata
> > > > > because
> > > > > > we
> > > > > > > > > have
> > > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > > what we need in TopicMetadata. Also, as
> > Jay
> > > > > > pointed
> > > > > > > > out
> > > > > > > > > > > > > earlier,
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > would like to have "orthogonal" API,
> but
> > at
> > > > the
> > > > > > > same
> > > > > > > > > time
> > > > > > > > > > > we
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > But I like your idea and even have some
> > > other
> > > > > > > > arguments
> > > > > > > > > > for
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > > There is also DescribeTopicRequest
> which
> > > was
> > > > > > > proposed
> > > > > > > > > in
> > > > > > > > > > > this
> > > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > > it returns topic configs, partitions,
> > > > > replication
> > > > > > > > > factor
> > > > > > > > > > > plus
> > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > > leader replica. The later part is
> really
> > > > > already
> > > > > > > > there
> > > > > > > > > in
> > > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > > So again we'll have to add stuff to
> TMR,
> > > not
> > > > to
> > > > > > > > > duplicate
> > > > > > > > > > > some
> > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > newly added requests. However, this way
> > > we'll
> > > > > end
> > > > > > > up
> > > > > > > > > with
> > > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > > request which returns cluster metadata,
> > > topic
> > > > > > > > > replication
> > > > > > > > > > > and
> > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > > plus partition replication data. Seems
> > > > logical
> > > > > to
> > > > > > > > split
> > > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > > - ClusterMetadata (brokers +
> controller,
> > > > maybe
> > > > > > smth
> > > > > > > > > else)
> > > > > > > > > > > > > > > > > > > - TopicMetadata (topic info + partition
> > > > > details)
> > > > > > > > > > > > > > > > > > > But since current TMR is involved in
> lots
> > > of
> > > > > > places
> > > > > > > > > > > (including
> > > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > > as I understand) this might be very
> > serious
> > > > > > change
> > > > > > > > and
> > > > > > > > > it
> > > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel
> > > Koshy <
> > > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >> I may be missing some context but
> > > hopefully
> > > > > this
> > > > > > > > will
> > > > > > > > > > > also be
> > > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > > >> today: I thought the earlier proposal
> > > where
> > > > > > there
> > > > > > > > was
> > > > > > > > > an
> > > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > > >> ClusterMetadata request was clearer
> and
> > > > > > explicit.
> > > > > > > > > During
> > > > > > > > > > > the
> > > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > >> this thread I think the conclusion was
> > > that
> > > > > the
> > > > > > > main
> > > > > > > > > > need
> > > > > > > > > > > was
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> controller information and that can be
> > > > rolled
> > > > > > into
> > > > > > > > the
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > >> response but that seems a bit
> irrelevant
> > > to
> > > > > > topic
> > > > > > > > > > > metadata.
> > > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > > >> think the full broker-list is also
> > > > irrelevant
> > > > > to
> > > > > > > > topic
> > > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > >> it is already there and in use. I
> think
> > > > there
> > > > > is
> > > > > > > > still
> > > > > > > > > > > room
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > > >> explicit ClusterMetadata request since
> > > there
> > > > > may
> > > > > > > be
> > > > > > > > > > other
> > > > > > > > > > > > > > > > > > >> cluster-level information that we may
> > want
> > > > to
> > > > > > add
> > > > > > > > over
> > > > > > > > > > > time
> > > > > > > > > > > > > (and
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> have nothing to do with topic
> metadata).
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM
> > +0200,
> > > > > Andrii
> > > > > > > > > > Biletskyi
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > 101. Okay, if you say that such use
> > case
> > > > is
> > > > > > > > > > important. I
> > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > >> > using clientId for these purposes is
> > > fine
> > > > -
> > > > > if
> > > > > > > we
> > > > > > > > > > > already
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > > >> > as part of all Wire protocol
> messages,
> > > why
> > > > > not
> > > > > > > use
> > > > > > > > > > that.
> > > > > > > > > > > > > > > > > > >> > I will update KIP-4 page if nobody
> has
> > > > other
> > > > > > > ideas
> > > > > > > > > > > (which
> > > > > > > > > > > > > may
> > > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP
> > > > > accordingly.
> > > > > > I
> > > > > > > > > think
> > > > > > > > > > > we can
> > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > > >> > fine-grained error codes if some
> error
> > > > code
> > > > > > > > received
> > > > > > > > > > in
> > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > >> > won't give enough context to return
> a
> > > > > > > descriptive
> > > > > > > > > > error
> > > > > > > > > > > > > > message
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > Look forward to discussing all
> > > outstanding
> > > > > > > issues
> > > > > > > > in
> > > > > > > > > > > detail
> > > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM,
> Jun
> > > Rao
> > > > <
> > > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > 101. There may be a use case where
> > you
> > > > > only
> > > > > > > want
> > > > > > > > > the
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > > >> > > manually by admins. Currently, you
> > can
> > > > do
> > > > > > that
> > > > > > > > by
> > > > > > > > > > > > > disabling
> > > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> > > creation and issue topic creation
> > from
> > > > the
> > > > > > > > > > > TopicCommand.
> > > > > > > > > > > > > If
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > > >> > > topic creation completely on the
> > > broker
> > > > > and
> > > > > > > > don't
> > > > > > > > > > > have a
> > > > > > > > > > > > > way
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > >> > > between topic creation requests
> from
> > > the
> > > > > > > regular
> > > > > > > > > > > clients
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > > >> > > can't support manual topic
> creation
> > > any
> > > > > > more.
> > > > > > > I
> > > > > > > > > was
> > > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > > >> > > way of distinguishing the clients
> > > making
> > > > > the
> > > > > > > > topic
> > > > > > > > > > > > > creation
> > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > >> > > using clientId. For example, the
> > admin
> > > > > tool
> > > > > > > can
> > > > > > > > > set
> > > > > > > > > > > it to
> > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > > >> > > admin and the broker can treat
> that
> > > > > clientId
> > > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > Also, there is a related
> discussion
> > in
> > > > > > > > KAFKA-2020.
> > > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > following in
> TopicMetadataResponse:
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > 1. If leader is not available, we
> > set
> > > > the
> > > > > > > > > partition
> > > > > > > > > > > level
> > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > > >> > > 2. If a non-leader replica is not
> > > > > available,
> > > > > > > we
> > > > > > > > > take
> > > > > > > > > > > that
> > > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > >> > > the assigned replica list and isr
> in
> > > the
> > > > > > > > response.
> > > > > > > > > > As
> > > > > > > > > > > an
> > > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > > >> > > doing that, we set the partition
> > level
> > > > > error
> > > > > > > > code
> > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > This has a few problems. First,
> > > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > > >> > > an error, at least for the normal
> > > > > > > > > producer/consumer
> > > > > > > > > > > > > clients
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > > >> > > to find out the leader. Second, it
> > can
> > > > > > happen
> > > > > > > > that
> > > > > > > > > > > both
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> > > another replica are not available
> at
> > > the
> > > > > > same
> > > > > > > > > time.
> > > > > > > > > > > There
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > >> > > to indicate both. Third, even if a
> > > > replica
> > > > > > is
> > > > > > > > not
> > > > > > > > > > > > > available,
> > > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > > >> > > useful to return its replica id
> > since
> > > > some
> > > > > > > > clients
> > > > > > > > > > > (e.g.
> > > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > One way to address this issue is
> to
> > > > always
> > > > > > > > return
> > > > > > > > > > the
> > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > > >> > > leader, assigned replicas, and isr
> > > > > > regardless
> > > > > > > of
> > > > > > > > > > > whether
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > > >> > > broker is live or not. Since we
> also
> > > > > return
> > > > > > > the
> > > > > > > > > list
> > > > > > > > > > > of
> > > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > client can figure out whether a
> > leader
> > > > or
> > > > > a
> > > > > > > > > replica
> > > > > > > > > > is
> > > > > > > > > > > > > live
> > > > > > > > > > > > > > or
> > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > > >> > > accordingly. This way, we don't
> need
> > > to
> > > > > set
> > > > > > > the
> > > > > > > > > > > partition
> > > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > > >> > > when the leader or a replica is
> not
> > > > > > available.
> > > > > > > > > This
> > > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > > >> > > protocol, but does change the
> > > semantics.
> > > > > > Since
> > > > > > > > we
> > > > > > > > > > are
> > > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > > >> > > of TopicMetadataRequest here, we
> can
> > > > > > > potentially
> > > > > > > > > > > piggyback
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > 102.1 For those types of errors
> due
> > to
> > > > > > invalid
> > > > > > > > > > input,
> > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > >> > > guard it at parameter validation
> > time
> > > > and
> > > > > > > throw
> > > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > > >> > > without even sending the request
> to
> > > the
> > > > > > > broker?
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM,
> > > Andrii
> > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly>
> wrote:
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > 101. If I understand you
> > correctly,
> > > > you
> > > > > > are
> > > > > > > > > saying
> > > > > > > > > > > > > future
> > > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > > >> > > > will be ported to TMR_V1) won't
> be
> > > > able
> > > > > to
> > > > > > > > > > > automatically
> > > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > >> > > > unconditionally remove topic
> > > creation
> > > > > from
> > > > > > > > > there).
> > > > > > > > > > > But
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > > >> > > > Ok, about your proposal: I'm
> not a
> > > big
> > > > > fan
> > > > > > > > too,
> > > > > > > > > > > when it
> > > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > > >> > > > clients directly in protocol
> > schema.
> > > > And
> > > > > > > also
> > > > > > > > > I'm
> > > > > > > > > > > not
> > > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > > >> > > > auto.create.topics.enable is a
> > > server
> > > > > side
> > > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > > >> > > > in future versions, add this
> > setting
> > > > to
> > > > > > > > producer
> > > > > > > > > > and
> > > > > > > > > > > > > based
> > > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > > >> > > > UnknownTopic create topic
> > explicitly
> > > > by
> > > > > a
> > > > > > > > > separate
> > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because we
> > want
> > > > to
> > > > > > > > support
> > > > > > > > > > > batching
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > > >> > > > want to give descriptive error
> > > > messages
> > > > > > for
> > > > > > > > > > clients.
> > > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > > >> > > > to construct such messages (e.g.
> > > > > > AdminClient
> > > > > > > > > layer
> > > > > > > > > > > can
> > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > > > >> > > > means two cases: either invalid
> > > > number -
> > > > > > > e.g.
> > > > > > > > > -1;
> > > > > > > > > > or
> > > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > > >> > > > partitions argument wasn't) - I
> > > > wrapped
> > > > > > > > > responses
> > > > > > > > > > in
> > > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > > >> > > > other ideas, this was just
> initial
> > > > > > version.
> > > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change
> > > that
> > > > to
> > > > > > > > > probably
> > > > > > > > > > > some
> > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM,
> > Jun
> > > > > Rao <
> > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > 101. That's what I was
> thinking
> > > too,
> > > > > but
> > > > > > > it
> > > > > > > > > may
> > > > > > > > > > > not be
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > > >> > > > > we can let it not trigger auto
> > > topic
> > > > > > > > creation.
> > > > > > > > > > > Then,
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > > UnknownTopicException,
> > > > > it
> > > > > > > can
> > > > > > > > > > > explicitly
> > > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > > >> > > > > createTopicRequest for auto
> > topic
> > > > > > > creation.
> > > > > > > > On
> > > > > > > > > > the
> > > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > > >> > > > > never issue
> createTopicRequest.
> > > This
> > > > > > works
> > > > > > > > > when
> > > > > > > > > > > auto
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > > >> > > > > enabled on the broker side.
> > > > However, I
> > > > > > am
> > > > > > > > not
> > > > > > > > > > > sure how
> > > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > > >> > > > > when auto topic creation is
> > > disabled
> > > > > on
> > > > > > > the
> > > > > > > > > > broker
> > > > > > > > > > > > > side.
> > > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > >> > > > > want to have a way to manually
> > > > create
> > > > > a
> > > > > > > > topic,
> > > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > >> > > > > commands. However, then we
> need
> > a
> > > > way
> > > > > to
> > > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > > >> > > > > issued from the producer
> clients
> > > and
> > > > > the
> > > > > > > > admin
> > > > > > > > > > > tools.
> > > > > > > > > > > > > > May
> > > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > >> > > > > new field in
> createTopicRequest
> > > and
> > > > > set
> > > > > > it
> > > > > > > > > > > differently
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > > >> > > > > client and the admin client.
> > > > However,
> > > > > I
> > > > > > am
> > > > > > > > not
> > > > > > > > > > > sure if
> > > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring existing
> > > > requests
> > > > > > is a
> > > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > > >> > > > > posted some comments in
> > > KAFKA-1927.
> > > > We
> > > > > > > will
> > > > > > > > > > > probably
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > > >> > > > > first, before adding the new
> > logic
> > > > in
> > > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird that
> we
> > > > return
> > > > > > > > > exception
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > >> > > > > we should either return error
> > code
> > > > or
> > > > > > > throw
> > > > > > > > an
> > > > > > > > > > > > > exception
> > > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > > >> > > > > 102.2. We probably shouldn't
> > > > > explicitly
> > > > > > > use
> > > > > > > > > the
> > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > > >> > > > > Not every request evolution
> > > requires
> > > > > an
> > > > > > > api
> > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08
> AM,
> > > > > Andrii
> > > > > > > > > > Biletskyi
> > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > Thanks for you comments.
> > Answers
> > > > > > inline:
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > 100. There are a few fields
> > such
> > > > as
> > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized
> > that
> > > > are
> > > > > > > > > > represented
> > > > > > > > > > > as a
> > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > >> > > > > > > composite structures in
> > json.
> > > > > Could
> > > > > > we
> > > > > > > > > > flatten
> > > > > > > > > > > > > them
> > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > Yes, now with Admin Client
> > this
> > > > > looks
> > > > > > a
> > > > > > > > bit
> > > > > > > > > > > weird.
> > > > > > > > > > > > > My
> > > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > > >> > > > > > ReassignPartitionCommand
> > accepts
> > > > > input
> > > > > > > in
> > > > > > > > > > json,
> > > > > > > > > > > we
> > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > > >> > > > > > interfaces unchanged, where
> > > > > possible.
> > > > > > > > > > > > > > > > > > >> > > > > > If we port it to
> deserialized
> > > > > format,
> > > > > > in
> > > > > > > > CLI
> > > > > > > > > > > (/tools
> > > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > > >> > > > > > json library since /tools is
> > > > written
> > > > > > in
> > > > > > > > java
> > > > > > > > > > and
> > > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > > >> > > > > > provided by a user. Can we
> > > quickly
> > > > > > agree
> > > > > > > > on
> > > > > > > > > > what
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > 101. Does
> TopicMetadataRequest
> > > v1
> > > > > > still
> > > > > > > > > > trigger
> > > > > > > > > > > auto
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now
> that
> > > we
> > > > > > have a
> > > > > > > > > > > separate
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > >> > > > > > > you thought about how the
> > new
> > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > > producer/consumer
> > > > > > > > > > > client,
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> ideally,
> > > we
> > > > > > don't
> > > > > > > > want
> > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto
> > topic
> > > > > > > creation.
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > I agree, this strange logic
> > > should
> > > > > be
> > > > > > > > fixed.
> > > > > > > > > > > I'm not
> > > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > > >> > > > > > correct me if I'm wrong, but
> > it
> > > > > > doesn't
> > > > > > > > look
> > > > > > > > > > > like a
> > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > > >> > > > > > leverage AdminClient for
> that
> > in
> > > > > > > Producer
> > > > > > > > > and
> > > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant getting
> > rid
> > > > of
> > > > > > > scala
> > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > >> > > > > > > like
> > HeartbeatRequestAndHeader
> > > > and
> > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> > > adding
> > > > > the
> > > > > > > new
> > > > > > > > > > > requests
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > >> > > > > > > However, the long term
> plan
> > is
> > > > to
> > > > > > get
> > > > > > > > rid
> > > > > > > > > of
> > > > > > > > > > > all
> > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > java request/response in
> the
> > > > > client.
> > > > > > > > Since
> > > > > > > > > > > this
> > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > > > > requests,
> > > > > > > > > perhaps
> > > > > > > > > > we
> > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing
> scala
> > > > > requests
> > > > > > > > first
> > > > > > > > > > > before
> > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > Yes, looks like I
> > misunderstood
> > > > the
> > > > > > > point
> > > > > > > > of
> > > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > > >> > > > > > rework that. The only thing
> is
> > > > that
> > > > > I
> > > > > > > > don't
> > > > > > > > > > see
> > > > > > > > > > > any
> > > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > > >> > > > > > least one existing protocol
> > > > message.
> > > > > > > Thus,
> > > > > > > > > as
> > > > > > > > > > I
> > > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > > > > > >> > > > > > Re porting all existing
> RQ/RP
> > in
> > > > > this
> > > > > > > > patch.
> > > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > > > >> > > > > > requirement to have Admin
> KIP
> > > > done,
> > > > > > I'm
> > > > > > > > > afraid
> > > > > > > > > > > this
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > > > >> > > > > > There are 13 protocol
> messages
> > > and
> > > > > all
> > > > > > > > that
> > > > > > > > > > > would
> > > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > > > >> > > > > > intensive manual testing,
> no?
> > > I'm
> > > > > > afraid
> > > > > > > > I'm
> > > > > > > > > > > not the
> > > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > > > >> > > > > > Kafka core internals :). Let
> > me
> > > > know
> > > > > > > your
> > > > > > > > > > > thoughts
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40
> > AM,
> > > > Jun
> > > > > > > Rao <
> > > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > 100. There are a few
> fields
> > > such
> > > > > as
> > > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized
> > that
> > > > are
> > > > > > > > > > represented
> > > > > > > > > > > as a
> > > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > > >> > > > > > > composite structures in
> > json.
> > > > > Could
> > > > > > we
> > > > > > > > > > flatten
> > > > > > > > > > > > > them
> > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > > arrays/records?
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> > TopicMetadataRequest
> > > > v1
> > > > > > > still
> > > > > > > > > > > trigger
> > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now
> that
> > > we
> > > > > > have a
> > > > > > > > > > > separate
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > > >> > > > > > > you thought about how the
> > new
> > > > > > > > > > > createTopicRequest
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > > producer/consumer
> > > > > > > > > > > client,
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > > >> > > > > > > tools? For example,
> ideally,
> > > we
> > > > > > don't
> > > > > > > > want
> > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto
> > topic
> > > > > > > creation.
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant
> getting
> > > rid
> > > > > of
> > > > > > > > scala
> > > > > > > > > > > classes
> > > > > > > > > > > > > > > > > > >> > > > > > > like
> > HeartbeatRequestAndHeader
> > > > and
> > > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> > > adding
> > > > > the
> > > > > > > new
> > > > > > > > > > > requests
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > > >> > > > > > > However, the long term
> plan
> > is
> > > > to
> > > > > > get
> > > > > > > > rid
> > > > > > > > > of
> > > > > > > > > > > all
> > > > > > > > > > > > > > those
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > java request/response in
> the
> > > > > client.
> > > > > > > > Since
> > > > > > > > > > > this
> > > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > > > > requests,
> > > > > > > > > perhaps
> > > > > > > > > > we
> > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing
> scala
> > > > > requests
> > > > > > > > first
> > > > > > > > > > > before
> > > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at
> 3:37
> > > PM,
> > > > > > > Andrii
> > > > > > > > > > > Biletskyi
> > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > >
> andrii.biletskyi@stealth.ly
> > >
> > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > As said above - I list
> > again
> > > > all
> > > > > > > > > comments
> > > > > > > > > > > from
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > > >> > > > > > > > can see what's left and
> > > > finalize
> > > > > > all
> > > > > > > > > > pending
> > > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is much needed
> > > > > > > functionality,
> > > > > > > > > but
> > > > > > > > > > > there
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > > >> > > > > > > > really think these
> > protocols
> > > > > > > through.
> > > > > > > > We
> > > > > > > > > > > really
> > > > > > > > > > > > > > want
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > > >> > > > > > > > of well thought-out,
> > > > orthoganol
> > > > > > > apis.
> > > > > > > > > For
> > > > > > > > > > > this
> > > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > > >> > > > > > > > important to think
> through
> > > the
> > > > > end
> > > > > > > > state
> > > > > > > > > > > even if
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > won't implement in the
> > first
> > > > > > phase.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Definitely behind
> this.
> > > > Would
> > > > > > > > > > appreciate
> > > > > > > > > > > if
> > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > > >> > > > > > > > how this can be
> improved.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's please please
> > > please
> > > > > wait
> > > > > > > > until
> > > > > > > > > > we
> > > > > > > > > > > have
> > > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > > >> > > > > > > > to the new java protocol
> > > > > > > definitions.
> > > > > > > > If
> > > > > > > > > > we
> > > > > > > > > > > add
> > > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > > >> > > > > > > > objects that is just
> > > > generating
> > > > > > more
> > > > > > > > > work
> > > > > > > > > > > for
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > patch -
> > > > > > > removed
> > > > > > > > > > scala
> > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 3. This proposal
> > introduces
> > > a
> > > > > new
> > > > > > > type
> > > > > > > > > of
> > > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > > >> > > > > > > > inconsistent with
> > everything
> > > > > else
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > > >> > > > > > > > other marker value. You
> > > could
> > > > > > argue
> > > > > > > > > either
> > > > > > > > > > > way
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > > for consistency. For
> > clients
> > > > > that
> > > > > > > > > > > implemented
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > > >> > > > > > > > than our scala code
> these
> > > > basic
> > > > > > > > > primitives
> > > > > > > > > > > are
> > > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > patch -
> > > > > > > removed
> > > > > > > > > > > MaybeOf
> > > > > > > > > > > > > > type
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This
> > > seems
> > > > > to
> > > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > >> > > > > > > > brokers, topics, and
> > > > > partitions. I
> > > > > > > > think
> > > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > > >> > > > > > > > ClusterMetadataRequest
> (or
> > > > just
> > > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > controller. Or are there
> > > other
> > > > > > > things
> > > > > > > > we
> > > > > > > > > > > could
> > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: I agree. Updated the
> > KIP.
> > > > > Let's
> > > > > > > > > extends
> > > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a tendency to
> > try
> > > > to
> > > > > > > make a
> > > > > > > > > lot
> > > > > > > > > > > of
> > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > particular nodes. This
> > adds
> > > a
> > > > > lot
> > > > > > of
> > > > > > > > > > burden
> > > > > > > > > > > for
> > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > > >> > > > > > > > sounds easy but each
> > > discovery
> > > > > can
> > > > > > > > fail
> > > > > > > > > in
> > > > > > > > > > > many
> > > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > > >> > > > > > > > full state machine to do
> > > > > right). I
> > > > > > > > think
> > > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > > >> > > > > > > > commands and ideally as
> > many
> > > > of
> > > > > > the
> > > > > > > > > other
> > > > > > > > > > > apis
> > > > > > > > > > > > > as
> > > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > > >> > > > > > > > brokers and just
> redirect
> > to
> > > > the
> > > > > > > > > > controller
> > > > > > > > > > > on
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > > >> > > > > > > > there would be a general
> > way
> > > > to
> > > > > > > > > > encapsulate
> > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a very
> interesting
> > > > idea,
> > > > > > but
> > > > > > > > > seems
> > > > > > > > > > > there
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > > >> > > > > > > > feature (like
> performance
> > > > > > > > > considerations,
> > > > > > > > > > > how
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > > >> > > > > > > > I believe this shouldn't
> > be
> > > a
> > > > > > > blocker.
> > > > > > > > > If
> > > > > > > > > > > this
> > > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > > >> > > > > > > > point it won't affect
> > Admin
> > > > > > changes
> > > > > > > -
> > > > > > > > at
> > > > > > > > > > > least
> > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 6. We should probably
> > > > normalize
> > > > > > the
> > > > > > > > key
> > > > > > > > > > > value
> > > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > >> > > > > > > > than embedding a new
> > > > formatting.
> > > > > > So
> > > > > > > > two
> > > > > > > > > > > strings
> > > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > patch -
> > > > > > > > > normalized
> > > > > > > > > > > > > configs
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the postcondition
> of
> > > > these
> > > > > > > APIs
> > > > > > > > > that
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > > the command has been
> > > > completed?
> > > > > It
> > > > > > > is
> > > > > > > > a
> > > > > > > > > > lot
> > > > > > > > > > > more
> > > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > > >> > > > > > > > been completed so you
> know
> > > > that
> > > > > if
> > > > > > > you
> > > > > > > > > > > create a
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > it you won't get an
> > > exception
> > > > > > about
> > > > > > > > > there
> > > > > > > > > > > being
> > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: For long running
> > requests
> > > > > (like
> > > > > > > > > > reassign
> > > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > > > >> > > > > > > > command has begun - so
> we
> > > > don't
> > > > > > > block
> > > > > > > > > the
> > > > > > > > > > > > > client.
> > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > > >> > > > > > > > topic commands, this
> will
> > be
> > > > > > > > refactored
> > > > > > > > > > and
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > > >> > > > > > > > immediately, since the
> > > > > Controller
> > > > > > > will
> > > > > > > > > > serve
> > > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up ticket
> > > KAFKA-1777).
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe topic and
> list
> > > > > topics
> > > > > > > > > > duplicate
> > > > > > > > > > > a
> > > > > > > > > > > > > lot
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > > >> > > > > > > > request. Is there a
> reason
> > > to
> > > > > give
> > > > > > > > back
> > > > > > > > > > > topics
> > > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > > >> > > > > > > > like if we just make the
> > > > > > > > post-condition
> > > > > > > > > of
> > > > > > > > > > > the
> > > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > topic is deleted that
> will
> > > get
> > > > > rid
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > > need
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > > patch -
> > > > > > > removed
> > > > > > > > > > > topics
> > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we consider
> > > batching
> > > > > > these
> > > > > > > > > > > requests?
> > > > > > > > > > > > > We
> > > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > allow multiple
> operations
> > to
> > > > be
> > > > > > > > batched.
> > > > > > > > > > My
> > > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > >> > > > > > > > we will get a lot of
> code
> > > that
> > > > > > does
> > > > > > > > > > > something
> > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > > >> > > > > > > > this code will work
> great
> > > when
> > > > > you
> > > > > > > > test
> > > > > > > > > > on 5
> > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > please
> > > > > check
> > > > > > > > "Topic
> > > > > > > > > > > Admin
> > > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 10. I think we should
> also
> > > > > discuss
> > > > > > > how
> > > > > > > > > we
> > > > > > > > > > > want
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > > >> > > > > > > > client api for these
> > > > operations.
> > > > > > > > > Currently
> > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > > >> > > > > > > > is totally sketchy. I
> > think
> > > we
> > > > > > > > probably
> > > > > > > > > > need
> > > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > > >> > > > > > > > that exposes
> > administrative
> > > > > > > > > functionality.
> > > > > > > > > > > We
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > properly test the new
> > apis,
> > > I
> > > > > > > suspect.
> > > > > > > > > We
> > > > > > > > > > > should
> > > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > please
> > > > > check
> > > > > > > > "Admin
> > > > > > > > > > > Client"
> > > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 11. The other
> information
> > > that
> > > > > > would
> > > > > > > > be
> > > > > > > > > > > really
> > > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > >> > > > > > > > information about
> > > > > partitions--how
> > > > > > > much
> > > > > > > > > > data
> > > > > > > > > > > is
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > > >> > > > > > > > the segment offsets,
> what
> > is
> > > > the
> > > > > > > > log-end
> > > > > > > > > > > offset
> > > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > > >> > > > > > > > the compaction point,
> > etc. I
> > > > > think
> > > > > > > > that
> > > > > > > > > > done
> > > > > > > > > > > > > right
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > successor to the very
> > > awkward
> > > > > > > > > > OffsetRequest
> > > > > > > > > > > we
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > > > >> > > > > > > > be resolved in a
> separate
> > > KIP
> > > > /
> > > > > > jira
> > > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 12. Generally we can do
> > good
> > > > > error
> > > > > > > > > > handling
> > > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > > > >> > > > > > > > messages. I.e. generally
> > the
> > > > > > client
> > > > > > > > has
> > > > > > > > > > the
> > > > > > > > > > > > > > context
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > > >> > > > > > > > an error that the topic
> > > > doesn't
> > > > > > > exist
> > > > > > > > to
> > > > > > > > > > say
> > > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > > >> > > > > > > > than "error code 14" (or
> > > > > > whatever).
> > > > > > > > > Maybe
> > > > > > > > > > > there
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > > >> > > > > > > > this is hard? If we want
> > to
> > > > add
> > > > > > > > > > server-side
> > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > > >> > > > > > > > need to do this in a
> > > > consistent
> > > > > > way
> > > > > > > > > across
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> > please
> > > > > check
> > > > > > > > > > "Protocol
> > > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > comprehensive,
> > fine-grained
> > > > list
> > > > > > of
> > > > > > > > > error
> > > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe topic
> > request:
> > > it
> > > > > > would
> > > > > > > > be
> > > > > > > > > > > great to
> > > > > > > > > > > > > > go
> > > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex for
> this
> > > > > request.
> > > > > > > For
> > > > > > > > > > > example,
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > > >> > > > > > > > the topic command is to
> > list
> > > > all
> > > > > > > > topics
> > > > > > > > > > > whose
> > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex then we
> > > have
> > > > to
> > > > > > > first
> > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > > >> > > > > > > > description info and
> then
> > > > filter
> > > > > > at
> > > > > > > > the
> > > > > > > > > > > client
> > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in
> create
> > > > topic:
> > > > > > > this
> > > > > > > > is
> > > > > > > > > > > related
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > > >> > > > > > > > maybe we can add another
> > > > > metadata
> > > > > > > K-V
> > > > > > > > or
> > > > > > > > > > > just a
> > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > > >> > > > > > > > with config K-V in
> create
> > > > topic
> > > > > > like
> > > > > > > > we
> > > > > > > > > > did
> > > > > > > > > > > for
> > > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > > >> > > > > > > > field can be quite
> useful
> > in
> > > > > > storing
> > > > > > > > > > > information
> > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > > >> > > > > > > > who issue the create
> > > command,
> > > > > etc,
> > > > > > > > which
> > > > > > > > > > is
> > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant setting.
> Then
> > > in
> > > > > the
> > > > > > > > > describe
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > > >> > > > > > > > on regex of the metadata
> > > > field.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: As discussed it is
> very
> > > > > > > interesting
> > > > > > > > > but
> > > > > > > > > > > can
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > > >> > > > > > > > we have some basic
> > > > functionality
> > > > > > > > there.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today all the admin
> > > > > operations
> > > > > > > are
> > > > > > > > > > > async in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > > >> > > > > > > > return once it is
> written
> > in
> > > > ZK,
> > > > > > and
> > > > > > > > > that
> > > > > > > > > > > is why
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > > testUtil.waitForTopicCreated()
> > > > > > > /
> > > > > > > > > > verify
> > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > > >> > > > > > > > request, etc. With admin
> > > > > requests
> > > > > > we
> > > > > > > > > could
> > > > > > > > > > > add a
> > > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > > >> > > > > > > > synchronous requests;
> when
> > > it
> > > > is
> > > > > > > > turned
> > > > > > > > > > on,
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > > >> > > > > > > > until the request has
> been
> > > > > > > completed.
> > > > > > > > > And
> > > > > > > > > > > for
> > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > > >> > > > > > > > "token" field in the
> > > response,
> > > > > and
> > > > > > > > then
> > > > > > > > > > only
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > > >> > > > > > > > verification request"
> with
> > > the
> > > > > > given
> > > > > > > > > token
> > > > > > > > > > > to
> > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: I see your point. My
> > idea
> > > > was
> > > > > > to
> > > > > > > > > > provide
> > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > > >> > > > > > > > long running request,
> > where
> > > > > > needed.
> > > > > > > We
> > > > > > > > > can
> > > > > > > > > > > do it
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > > >> > > > > > > > concern is that
> > introducing
> > > a
> > > > > > token
> > > > > > > we
> > > > > > > > > > again
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > > >> > > > > > > > to do similar thing
> > > > introducing
> > > > > > > single
> > > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > > > >> > > > > > > > this idea because we
> > wanted
> > > to
> > > > > > have
> > > > > > > > > schema
> > > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > > > > > >> > > > > > > > a) have fixed schema but
> > > > > introduce
> > > > > > > > each
> > > > > > > > > > > time new
> > > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > > > > > > > > > >> > > > > > > > b) use one request for
> > > > > > verification
> > > > > > > > but
> > > > > > > > > > > > > generalize
> > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine with whatever
> > > > decision
> > > > > > > > > community
> > > > > > > > > > > come
> > > > > > > > > > > > > to.
> > > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > > > > > > >> > > > > > > > 16. Specifically for
> > > > ownership,
> > > > > I
> > > > > > > > think
> > > > > > > > > > the
> > > > > > > > > > > plan
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > > >> > > > > > > > like you are describing
> > ACL)
> > > > via
> > > > > > an
> > > > > > > > > > external
> > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > > >> > > > > > > > I remember KIP-11
> > described
> > > > > this,
> > > > > > > but
> > > > > > > > I
> > > > > > > > > > > can't
> > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no problem. Not
> > > sure
> > > > > > though
> > > > > > > > how
> > > > > > > > > > we
> > > > > > > > > > > are
> > > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > > >> > > > > > > > will be committed first
> > and
> > > > > > include
> > > > > > > > > > changes
> > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I added this
> note
> > to
> > > > > "Open
> > > > > > > > > > > Questions"
> > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at
> > > 12:34
> > > > > AM,
> > > > > > > > Andrii
> > > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > andrii.biletskyi@stealth.ly
> > > >
> > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > Today I uploaded the
> > patch
> > > > > that
> > > > > > > > covers
> > > > > > > > > > > some of
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf
> > optional
> > > > > type
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - switched to java
> > > protocol
> > > > > > > > > definitions
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - simplified messages
> > > > > > (normalized
> > > > > > > > > > configs,
> > > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > I also updated the
> KIP-4
> > > > with
> > > > > > > > > respective
> > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin
> Operations
> > > ->
> > > > > > > updated
> > > > > > > > > Wire
> > > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove
> ClusterMetadata
> > > ->
> > > > > > > changed
> > > > > > > > to
> > > > > > > > > > > extend
> > > > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin Client ->
> > updated
> > > my
> > > > > > > initial
> > > > > > > > > > > proposal
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > > >> > > > > > > > > - Error codes ->
> > proposed
> > > > > > > > fine-grained
> > > > > > > > > > > error
> > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > I will also send a
> > > separate
> > > > > > email
> > > > > > > to
> > > > > > > > > > > cover all
> > > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015
> at
> > > 9:26
> > > > > PM,
> > > > > > > > Gwen
> > > > > > > > > > > Shapira
> > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> It actually specifies
> > > > changes
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > Metadata
> > > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are
> > consistent
> > > in
> > > > > > this
> > > > > > > > > regard
> > > > > > > > > > > will
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015
> at
> > > > 12:21
> > > > > > PM,
> > > > > > > > > Gwen
> > > > > > > > > > > > > Shapira
> > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Specifically for
> > > > > ownership, I
> > > > > > > > think
> > > > > > > > > > the
> > > > > > > > > > > > > plan
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > like you are
> > describing
> > > > > ACL)
> > > > > > > via
> > > > > > > > an
> > > > > > > > > > > > > external
> > > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > I remember KIP-11
> > > > described
> > > > > > > this,
> > > > > > > > > > but I
> > > > > > > > > > > > > can't
> > > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I think
> > > KIP-4
> > > > > > > focuses
> > > > > > > > > on
> > > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > exists from Kafka
> > > > brokers,
> > > > > > not
> > > > > > > on
> > > > > > > > > > > adding
> > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > should exist but
> > > doesn't
> > > > > yet?
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12,
> 2015
> > at
> > > > > 6:37
> > > > > > > AM,
> > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just want to
> > > elaborate a
> > > > > bit
> > > > > > > > more
> > > > > > > > > on
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> describe-topic
> based
> > > on
> > > > > > > config /
> > > > > > > > > > > metadata
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The
> > > main
> > > > > > > > motivation
> > > > > > > > > > is
> > > > > > > > > > > to
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms, which
> I
> > > > think
> > > > > is
> > > > > > > > quite
> > > > > > > > > > > > > important
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> architecture:
> today
> > > > anyone
> > > > > > can
> > > > > > > > > > create
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> there is no
> concept
> > or
> > > > > > > > "ownership"
> > > > > > > > > > of
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> users. For
> example,
> > at
> > > > > > > LinkedIn
> > > > > > > > we
> > > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some casual topic
> > name
> > > > > > prefix,
> > > > > > > > > which
> > > > > > > > > > > is a
> > > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale our
> > > customers.
> > > > It
> > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > great
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> topics
> > > that
> > > > > is
> > > > > > > > > created
> > > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> topics
> > > > whose
> > > > > > > > > retention
> > > > > > > > > > > time
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all
> topics
> > > > whose
> > > > > > > > writable
> > > > > > > > > > > group
> > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> authorization),
> > etc..
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One possible way
> to
> > > > > achieve
> > > > > > > this
> > > > > > > > > is
> > > > > > > > > > to
> > > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> create-topic
> > request,
> > > > > whose
> > > > > > > > value
> > > > > > > > > > will
> > > > > > > > > > > > > also
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic; then
> > > > > describe-topics
> > > > > > > can
> > > > > > > > > > > choose to
> > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2) config
> K-V
> > > > > > matching,
> > > > > > > > 3)
> > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5,
> 2015
> > at
> > > > > 4:37
> > > > > > > PM,
> > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for the
> > > updated
> > > > > > wiki.
> > > > > > > A
> > > > > > > > > few
> > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error
> > description
> > > in
> > > > > > > > > response: I
> > > > > > > > > > > think
> > > > > > > > > > > > > > if
> > > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> several different
> > > error
> > > > > > cases
> > > > > > > > > then
> > > > > > > > > > we
> > > > > > > > > > > > > > should
> > > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In general
> > the
> > > > > > > errorCode
> > > > > > > > > > > itself
> > > > > > > > > > > > > > would
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> describing the
> > server
> > > > > side
> > > > > > > > > errors.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic
> > > > > request:
> > > > > > it
> > > > > > > > > would
> > > > > > > > > > > be
> > > > > > > > > > > > > > great
> > > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex
> > for
> > > > this
> > > > > > > > > request.
> > > > > > > > > > > For
> > > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic command
> > is
> > > to
> > > > > > list
> > > > > > > > all
> > > > > > > > > > > topics
> > > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex
> > then
> > > > we
> > > > > > have
> > > > > > > > to
> > > > > > > > > > > first
> > > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> description info
> > and
> > > > then
> > > > > > > > filter
> > > > > > > > > at
> > > > > > > > > > > the
> > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in
> > > > create
> > > > > > > topic:
> > > > > > > > > > this
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can add
> > > > another
> > > > > > > > metadata
> > > > > > > > > > > K-V or
> > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with config K-V
> in
> > > > create
> > > > > > > topic
> > > > > > > > > > like
> > > > > > > > > > > we
> > > > > > > > > > > > > did
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field can be
> quite
> > > > useful
> > > > > > in
> > > > > > > > > > storing
> > > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue the
> > create
> > > > > > command,
> > > > > > > > > etc,
> > > > > > > > > > > which
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant
> > setting.
> > > > > Then
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > > > describe
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of the
> > > > metadata
> > > > > > > field.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all the
> > > admin
> > > > > > > > operations
> > > > > > > > > > are
> > > > > > > > > > > > > async
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> return once it is
> > > > written
> > > > > > in
> > > > > > > > ZK,
> > > > > > > > > > and
> > > > > > > > > > > that
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> request, etc.
> With
> > > > admin
> > > > > > > > requests
> > > > > > > > > > we
> > > > > > > > > > > > > could
> > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous
> > requests;
> > > > > when
> > > > > > it
> > > > > > > > is
> > > > > > > > > > > turned
> > > > > > > > > > > > > on,
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until the request
> > has
> > > > > been
> > > > > > > > > > > completed. And
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token" field in
> > the
> > > > > > > response,
> > > > > > > > > and
> > > > > > > > > > > then
> > > > > > > > > > > > > > only
> > > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> verification
> > request"
> > > > > with
> > > > > > > the
> > > > > > > > > > given
> > > > > > > > > > > > > token
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has been
> completed.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for
> extending
> > > > > > Metadata
> > > > > > > > > > request
> > > > > > > > > > > to
> > > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> information, and
> > then
> > > > we
> > > > > > can
> > > > > > > > > remove
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3,
> 2015
> > > at
> > > > > > 10:23
> > > > > > > > AM,
> > > > > > > > > > Joel
> > > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for
> sending
> > > > that
> > > > > > out
> > > > > > > > Joe
> > > > > > > > > -
> > > > > > > > > > I
> > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so if
> > > notes
> > > > > can
> > > > > > be
> > > > > > > > > sent
> > > > > > > > > > > out
> > > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02,
> > 2015
> > > > at
> > > > > > > > > 09:16:13AM
> > > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for
> > sending
> > > > > this
> > > > > > > out
> > > > > > > > > Joe.
> > > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2,
> > > 2015
> > > > at
> > > > > > > 6:46
> > > > > > > > > AM,
> > > > > > > > > > > Joe
> > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just
> > sent
> > > > > out a
> > > > > > > > > google
> > > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I
> > found
> > > > > > working
> > > > > > > > on
> > > > > > > > > a
> > > > > > > > > > > KIP.
> > > > > > > > > > > > > If
> > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me know
> > and
> > > > can
> > > > > > > update
> > > > > > > > > it,
> > > > > > > > > > > np.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We should do
> > > this
> > > > > > every
> > > > > > > > > > Tuesday
> > > > > > > > > > > @
> > > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to
> make a
> > > > > google
> > > > > > > > > account
> > > > > > > > > > > so we
> > > > > > > > > > > > > > can
> > > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in progress
> > and
> > > > > > related
> > > > > > > > JIRA
> > > > > > > > > > > that
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb
> > 24,
> > > > 2015
> > > > > > at
> > > > > > > > 2:59
> > > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay
> on
> > > > > Google
> > > > > > > > > hangouts
> > > > > > > > > > > that
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> available
> on
> > > > > youtube.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb
> > 24,
> > > > > 2015
> > > > > > at
> > > > > > > > > 11:49
> > > > > > > > > > > AM,
> > > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > jholoman@cloudera.com
> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're
> happy
> > > to
> > > > > send
> > > > > > > > out a
> > > > > > > > > > > Webex
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions
> if
> > > > there
> > > > > > is
> > > > > > > > > > > interest and
> > > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue,
> Feb
> > > 24,
> > > > > > 2015
> > > > > > > at
> > > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's
> try
> > > to
> > > > > get
> > > > > > > the
> > > > > > > > > > > technical
> > > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there
> is
> > > some
> > > > > > > benefit
> > > > > > > > > to
> > > > > > > > > > > live
> > > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post
> > > > > > > instructions
> > > > > > > > > and
> > > > > > > > > > > give
> > > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> working.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday
> > at
> > > > that
> > > > > > > time
> > > > > > > > > > would
> > > > > > > > > > > work
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue,
> > Feb
> > > > 24,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > 8:18
> > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> Weekly
> > > > would
> > > > > be
> > > > > > > > great
> > > > > > > > > > > maybe
> > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I
> don't
> > > > mind
> > > > > > > google
> > > > > > > > > > > hangout
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we
> know
> > > the
> > > > > > > apache
> > > > > > > > > irc
> > > > > > > > > > > > > channel
> > > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes?
> > We
> > > > can
> > > > > > pull
> > > > > > > > > > > transcripts
> > > > > > > > > > > > > > too
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it
> > > helpful
> > > > > for
> > > > > > > > > things.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~
> > > Joestein
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On
> Tue,
> > > Feb
> > > > > 24,
> > > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > 11:10
> > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > jay.kreps@gmail.com
> > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> We'd
> > > > talked
> > > > > > > about
> > > > > > > > > > > doing a
> > > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > generalizing
> > > > > > > > that a
> > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > everyone
> > > > > > > > spending a
> > > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> sync
> > up
> > > > > once
> > > > > > a
> > > > > > > > > week.
> > > > > > > > > > I
> > > > > > > > > > > > > think
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > stuff,
> > > > make
> > > > > > > sure
> > > > > > > > we
> > > > > > > > > > > are on
> > > > > > > > > > > > > > top
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > issues,
> > > > > etc.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We
> > can
> > > > make
> > > > > > it
> > > > > > > > > > publicly
> > > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > likes.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any
> > > > > interest
> > > > > > in
> > > > > > > > > doing
> > > > > > > > > > > this?
> > > > > > > > > > > > > > If
> > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> -Jay
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On
> > Tue,
> > > > Feb
> > > > > > 24,
> > > > > > > > > 2015
> > > > > > > > > > at
> > > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> Hi
> > > all,
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > I've
> > > > > > updated
> > > > > > > > KIP
> > > > > > > > > > > page,
> > > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > some
> > > > > > > > >
> > > > > > > > ...
> > > > > > > >
> > > > > > > > [Message clipped]
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > >
> > >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Jun,

Not that I was saying we need to make requests blocking on server,
it was just to emphasize that with async requests a client implementations
may be a little bit more than just issue request - get the response.
Thanks for the explanation, I understand now that we can go with agreed
solution though it may not be perfect.
I believe this was one of the last controversial questions from the list.

Thanks,
Andrii Biletskyi

On Sat, Mar 21, 2015 at 12:18 AM, Jun Rao <ju...@confluent.io> wrote:

> Andrii,
>
> A few points.
>
> 1. Create/Alter can typically complete quickly. So, it's possible to make
> the request block until it's completed. However, currently, doing this at
> the broker is a bit involved. To make Create block, we will need to add
> some callbacks in KafkaController. This is possible. However, the
> controller logic currently is pretty completed. It would probably be better
> if we clean it up first before adding more complexity to it. Alter is even
> trickier. Adding partition is currently handled through KafkaController. So
> it can be dealt with in a similar way. However, Alter config is done
> completely differently. It doesn't go through the controller. Instead, each
> broker listens to ZooKeeper directly. So, it's not clear if there is an
> easy way on the broker to figure out whether a config is applied on every
> broker.
>
> 2. Delete can potentially take long if a replica to be deleted is offline.
> PreferredLeader/PartitionReassign can also take long. So, we can't really
> make those requests block on the broker.
>
> As you can see, at this moment it's not easy to make all admin requests
> block on the broker. So, if we want the blocking feature in the admin
> utility in the short term, doing the completion check at the admin client
> is probably an easier route, even though it may not be ideal.
>
> Thanks,
>
> Jun
>
> On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Jun,
> >
> > I see your point. But wouldn't that lead to a "fat" client
> implementations?
> > Suppose someone would like to implement client for Admin Wire protocol.
> > Not only people will have to code quite complicated logic like "send
> > describe
> > request to each broker" (again state machin?) but it will also mean
> people
> > must understand internal kafka logic related to topic storage and how
> > information is propageted from the controller to brokers.
> > I see this like a dilemma between having a concise Wire Protocol and
> > self-sufficient API to make client implementations simple.
> > I don't have a win-win solution though.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> >
> > On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > For 1), 2) and 3), blocking would probably mean that the new metadata
> is
> > > propagated to every broker. To achieve that, the client can keep
> issuing
> > > the describe topic request to every broker until it sees the new
> metadata
> > > in the response.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Hm, actually the ticket you linked, Guozhang, brings as back
> > > > to the problem what should be considered a post-condition for
> > > > each of the admin commands.
> > > > In my understanding:
> > > >
> > > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > > (Not the controller picked up changes from zk and broadcasted
> > > > LeaderAndIsr and UpdateMetadata)
> > > >
> > > > 2) AlterTopic - same as 1) - broker changed assignment data
> > > > in zookeeper or created admin path for topic config change
> > > >
> > > > 3) DeleteTopic - admin path /admin/delete_topics is created
> > > >
> > > > 4) ReassignPartitions and PreferredReplica - corresponding admin
> > > > path is created
> > > >
> > > > Now what can be considered a completed operation from the client's
> > > > perspective?
> > > > 1) Topic is created once corresponding data is in zk
> > > > (I remember there were some thoughts that it'd be good to consider
> > > > topic created once all replicas receive information about it and thus
> > > > clients can produce/consume from it, but as was discussed this seems
> > > > to be a hard thing to do)
> > > >
> > > > 2) Probably same as 1), so right after AlterTopic is issued
> > > >
> > > > 3) The topic has been removed from /brokers/topics
> > > >
> > > > 4) ReassignPartitions and PrefferedReplica were discussed earlier -
> > > > in short the former is completed once partition state info in zk
> > matches
> > > > reassignment request and admin path is empty, the latter - once data
> > > > in zk shows that head of assignned replicas of the partition and
> leader
> > > > is the same replica
> > > >
> > > > Thoughts?
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > I think while loop is fine for supporting blocking, just that we
> need
> > > to
> > > > > add back off to avoid bombarding brokers with DescribeTopic
> requests.
> > > > >
> > > > > Also I have linked KAFKA-1125
> > > > > <https://issues.apache.org/jira/browse/KAFKA-1125> to your
> proposal,
> > > and
> > > > > when KAFKA-1694 is done this ticket can also be closed.
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Great.
> > > > > > I want to elaborate this a bit more, to see we are on the same
> page
> > > > > > concerning the client code.
> > > > > >
> > > > > > So with all topic commands being async a client (AdminClient in
> our
> > > > > > case or any other other client people would like to implement) to
> > > > support
> > > > > > a blocking operation (which seems to be a natural use-case e.g.
> for
> > > > topic
> > > > > > creation): would have to do:
> > > > > > 1. issue CreateTopicRequest
> > > > > > 2. if successful, in a "while" loop send DescribeTopicRequest and
> > > > > > break the loop once all topics are returned in response (or upon
> > > > > timeout).
> > > > > > 3. if unsuccessful throw exception
> > > > > > Would it be okay?
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > >
> > > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Andrii,
> > > > > > >
> > > > > > > I think you are right. It seems that only ReassignPartitions
> > needs
> > > a
> > > > > > > separate verification request.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >
> > > > > > > > Guys,
> > > > > > > > I like this idea too. Let's stick with that. I'll update KIP
> > > > > > accordingly.
> > > > > > > >
> > > > > > > > I was also thinking we can avoid adding dedicated status
> check
> > > > > > > > requests for topic commands. - We have everything in
> > > DescribeTopic
> > > > > > > > for that! E.g.:
> > > > > > > > User issued CreateTopic - to check the status client sends
> > > > > > DescribeTopic
> > > > > > > > and checks whether is something returned for that topic. The
> > same
> > > > for
> > > > > > > > alteration, deletion.
> > > > > > > > Btw, PreferredReplica status can be also checked with
> > > > > > > DescribeTopicRequest
> > > > > > > > (head of assigned replicas list == leader).
> > > > > > > > For ReassignPartitions as discussed we'll need to have a
> > separate
> > > > > > > Verify...
> > > > > > > > request.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Andrii Biletskyi
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> > > wangguoz@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > +1 on broker writing to ZK for async handling. I was
> thinking
> > > > that
> > > > > in
> > > > > > > the
> > > > > > > > > end state the admin requests would be eventually sent to
> > > > controller
> > > > > > > > either
> > > > > > > > > through re-routing or clients discovering them, instead of
> > > > letting
> > > > > > > > > controller listen on ZK admin path. But thinking about it a
> > > > second
> > > > > > > time,
> > > > > > > > I
> > > > > > > > > think it is actually simpler to let controller manage
> > > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > > >
> > > > > > > > > Guozhang
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > > jjkoshy.w@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > +1 as well. I think it helps to keep the rerouting
> approach
> > > > > > > orthogonal
> > > > > > > > > > to this KIP.
> > > > > > > > > >
> > > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps
> wrote:
> > > > > > > > > > > I'm +1 on Jun's suggestion as long as it can work for
> all
> > > the
> > > > > > > > requests.
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <
> > jun@confluent.io
> > > >
> > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Andrii,
> > > > > > > > > > > >
> > > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > > >
> > > > > > > > > > > > (a) Admin requests can be sent to and handled by any
> > > > broker.
> > > > > > > > > > > > (b) Admin requests are processed asynchronously, at
> > least
> > > > for
> > > > > > > now.
> > > > > > > > > > That is,
> > > > > > > > > > > > when the client gets a response, it just means that
> the
> > > > > request
> > > > > > > is
> > > > > > > > > > > > initiated, but not necessarily completed. Then, it's
> up
> > > to
> > > > > the
> > > > > > > > client
> > > > > > > > > > to
> > > > > > > > > > > > issue another request to check the status for
> > completion.
> > > > > > > > > > > >
> > > > > > > > > > > > To support (a), we were thinking of doing request
> > > > forwarding
> > > > > to
> > > > > > > the
> > > > > > > > > > > > controller (utilizing KAFKA-1912). I am making an
> > > > alternative
> > > > > > > > > proposal.
> > > > > > > > > > > > Basically, the broker can just write to ZooKeeper to
> > > inform
> > > > > the
> > > > > > > > > > controller
> > > > > > > > > > > > about the request. For example, to handle
> > > > > > partitionReassignment,
> > > > > > > > the
> > > > > > > > > > broker
> > > > > > > > > > > > will just write the requested partitions to
> > > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > > (like what AdminUtils currently does) and then send a
> > > > > response
> > > > > > to
> > > > > > > > the
> > > > > > > > > > > > client. This shouldn't take long and the
> implementation
> > > > will
> > > > > be
> > > > > > > > > simpler
> > > > > > > > > > > > than forwarding the requests to the controller
> through
> > > RPC.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Jun,
> > > > > > > > > > > > >
> > > > > > > > > > > > > I might be wrong but didn't we agree we will let
> any
> > > > broker
> > > > > > > from
> > > > > > > > > the
> > > > > > > > > > > > > cluster handle *long-running* admin requests (at
> this
> > > > time
> > > > > > > > > > > > preferredReplica
> > > > > > > > > > > > > and
> > > > > > > > > > > > > reassignPartitions), via zk admin path. Thus
> > > CreateTopics
> > > > > etc
> > > > > > > > > should
> > > > > > > > > > be
> > > > > > > > > > > > > sent
> > > > > > > > > > > > > only to the controller.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> > > > > jun@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I think we agreed that those admin requests can
> be
> > > > issued
> > > > > > to
> > > > > > > > any
> > > > > > > > > > > > broker.
> > > > > > > > > > > > > > Because of that, there doesn't seem to be a
> strong
> > > need
> > > > > to
> > > > > > > know
> > > > > > > > > the
> > > > > > > > > > > > > > controller. So, perhaps we can proceed by not
> > making
> > > > any
> > > > > > > change
> > > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > > > format of TMR right now. When we start using
> create
> > > > topic
> > > > > > > > request
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > producer, we will need a new version of TMR that
> > > > doesn't
> > > > > > > > trigger
> > > > > > > > > > auto
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > As a first cut implementation, I think the broker
> > can
> > > > > just
> > > > > > > > write
> > > > > > > > > > to ZK
> > > > > > > > > > > > > > directly for
> > > > > > > > > > > > > >
> > > > > > > > >
> > > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > > requests, instead of forwarding them to the
> > > controller.
> > > > > > This
> > > > > > > > will
> > > > > > > > > > > > > simplify
> > > > > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> > > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > For (1) yes we will circle back on that shortly
> > > after
> > > > > > > syncing
> > > > > > > > > up
> > > > > > > > > > in
> > > > > > > > > > > > > > > person. I think it is close to getting
> committed
> > > > > although
> > > > > > > > > > development
> > > > > > > > > > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > There is one more item we covered at the
> hangout.
> > > > i.e.,
> > > > > > > > whether
> > > > > > > > > > we
> > > > > > > > > > > > > > > want to add the coordinator to the topic
> metadata
> > > > > > response
> > > > > > > or
> > > > > > > > > > provide
> > > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > There are two reasons I think we should try and
> > > avoid
> > > > > > > adding
> > > > > > > > > the
> > > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > > > > - If we finally do request rerouting in Kafka
> > then
> > > > the
> > > > > > > field
> > > > > > > > > > would
> > > > > > > > > > > > add
> > > > > > > > > > > > > > >   little to no value. (It still helps to have a
> > > > > separate
> > > > > > > > > > > > > > >   ClusterMetadataRequest to query for
> > cluster-wide
> > > > > > > > information
> > > > > > > > > > such
> > > > > > > > > > > > as
> > > > > > > > > > > > > > >   'which broker is the controller?' as Joe
> > > > mentioned.)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > I think it would be cleaner to have an explicit
> > > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > > that you can send to any broker in order to
> > obtain
> > > > the
> > > > > > > > > controller
> > > > > > > > > > > > (and
> > > > > > > > > > > > > > > in the future possibly other cluster-wide
> > > > > information). I
> > > > > > > > think
> > > > > > > > > > the
> > > > > > > > > > > > > > > main argument against doing this and instead
> > adding
> > > > it
> > > > > to
> > > > > > > the
> > > > > > > > > > topic
> > > > > > > > > > > > > > > metadata response was convenience - i.e., you
> > don't
> > > > > have
> > > > > > to
> > > > > > > > > > discover
> > > > > > > > > > > > > > > the controller in advance. However, I don't see
> > > much
> > > > > > actual
> > > > > > > > > > > > > > > benefit/convenience in this and in fact think
> it
> > > is a
> > > > > > > > > non-issue.
> > > > > > > > > > Let
> > > > > > > > > > > > > > > me know if I'm overlooking something here.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > As an example, say we need to initiate
> partition
> > > > > > > reassignment
> > > > > > > > > by
> > > > > > > > > > > > > > > issuing the new ReassignPartitionsRequest to
> the
> > > > > > controller
> > > > > > > > > > (assume
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > already have the desired manual partition
> > > > assignment).
> > > > > > If
> > > > > > > we
> > > > > > > > > > are to
> > > > > > > > > > > > > > > augment topic metadata response then the flow
> be
> > > > > > something
> > > > > > > > like
> > > > > > > > > > this
> > > > > > > > > > > > :
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > - Issue topic metadata request to any broker
> (and
> > > > > > discover
> > > > > > > > the
> > > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > > - Connect to controller if required (i.e., if
> the
> > > > > broker
> > > > > > > > above
> > > > > > > > > !=
> > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > - Issue the partition reassignment request to
> the
> > > > > > > controller.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > With an explicit cluster metadata request it
> > would
> > > > be:
> > > > > > > > > > > > > > > - Issue cluster metadata request to any broker
> > > > > > > > > > > > > > > - Connect to controller if required (i.e., if
> the
> > > > > broker
> > > > > > > > above
> > > > > > > > > !=
> > > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > > - Issue the partition reassignment request
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > So it seems to add little practical value and
> > > bloats
> > > > > > topic
> > > > > > > > > > metadata
> > > > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > The other angle to this is the following - is
> it
> > a
> > > > > matter
> > > > > > > of
> > > > > > > > > > naming?
> > > > > > > > > > > > > > > Should we just rename topic metadata
> > > request/response
> > > > > to
> > > > > > > just
> > > > > > > > > > > > > > > MetadataRequest/Response and add cluster
> metadata
> > > to
> > > > > it?
> > > > > > By
> > > > > > > > > that
> > > > > > > > > > same
> > > > > > > > > > > > > > > token should we also allow querying for the
> > > consumer
> > > > > > > > > coordinator
> > > > > > > > > > (and
> > > > > > > > > > > > > > > in future transaction coordinator) as well?
> This
> > > > leads
> > > > > > to a
> > > > > > > > > > bloated
> > > > > > > > > > > > > > > request which isn't very appealing and
> altogether
> > > > > > > confusing.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun
> Rao
> > > > > wrote:
> > > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1. I just realized that in order to start
> > working
> > > > on
> > > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > need to merge the changes to
> > OffsetCommitRequest
> > > > > (from
> > > > > > > > 0.8.2)
> > > > > > > > > > to
> > > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > > This is planned to be done as part of
> > KAFKA-1634.
> > > > So,
> > > > > > we
> > > > > > > > will
> > > > > > > > > > need
> > > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 2. Thinking about this a bit more, if the
> > > semantic
> > > > of
> > > > > > > those
> > > > > > > > > > "write"
> > > > > > > > > > > > > > > > requests is async (i.e., after the client
> gets
> > a
> > > > > > > response,
> > > > > > > > it
> > > > > > > > > > just
> > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > that the operation is initiated, but not
> > > > necessarily
> > > > > > > > > > completed), we
> > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > really need to forward the requests to the
> > > > > controller.
> > > > > > > > > > Instead, the
> > > > > > > > > > > > > > > > receiving broker can just write the operation
> > to
> > > ZK
> > > > > as
> > > > > > > the
> > > > > > > > > > admin
> > > > > > > > > > > > > > command
> > > > > > > > > > > > > > > > line tool previously does. This will simplify
> > the
> > > > > > > > > > implementation.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 8. There is another implementation detail for
> > > > > describe
> > > > > > > > topic.
> > > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > want to read the topic config from the broker
> > > > cache,
> > > > > > > > instead
> > > > > > > > > of
> > > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > > Currently, every broker reads the topic-level
> > > > config
> > > > > > for
> > > > > > > > all
> > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > > However, it ignores those for topics not
> hosted
> > > on
> > > > > > > itself.
> > > > > > > > > So,
> > > > > > > > > > we
> > > > > > > > > > > > may
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > to change TopicConfigManager a bit so that it
> > > > caches
> > > > > > the
> > > > > > > > > > configs
> > > > > > > > > > > > for
> > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii
> > > Biletskyi <
> > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 1. Q: Get rid of all scala requests
> objects,
> > > use
> > > > > java
> > > > > > > > > > protocol
> > > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > > >     A: Gwen kindly took that (KAFKA-1927).
> > It's
> > > > > > > important
> > > > > > > > > to
> > > > > > > > > > > > speed
> > > > > > > > > > > > > up
> > > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > > >          there since this ticket blocks
> other
> > > > > > important
> > > > > > > > > > changes.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 2. Q: Generic re-reroute facility vs client
> > > > > > maintaining
> > > > > > > > > > cluster
> > > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > > >     A: Jay has added pseudo code to
> > KAFKA-1912
> > > -
> > > > > need
> > > > > > > to
> > > > > > > > > > consider
> > > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > > >         easy to implement as a server-side
> > > > feature
> > > > > > > > > (comments
> > > > > > > > > > are
> > > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > > > > > > > > > >     A: This might be useful for clients,
> add
> > > this
> > > > > to
> > > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > > > > > > > > > >     A: I will add proposed by Jun solution
> > > (using
> > > > > > > > clientId
> > > > > > > > > > for
> > > > > > > > > > > > > that)
> > > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs
> grabbing
> > > all
> > > > > > > > protocol
> > > > > > > > > > > > changes
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > > >     A: It was decided to try to gather all
> > > > changes
> > > > > to
> > > > > > > > > > protocol
> > > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > > >         In case of TMR it worth checking:
> > > > > KAFKA-2020
> > > > > > > and
> > > > > > > > > > KIP-13
> > > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to deserialize
> > user's
> > > > > input
> > > > > > in
> > > > > > > > CLI
> > > > > > > > > > tool.
> > > > > > > > > > > > > > > > >     A: Use jackson for that, /tools project
> > is
> > > a
> > > > > > > separate
> > > > > > > > > > jar so
> > > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs generic
> > > status
> > > > > > check
> > > > > > > > > > command.
> > > > > > > > > > > > > > > > >      A: For long-running requests like
> > reassign
> > > > > > > > partitions
> > > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > > >          it makes sense to introduce it.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >  Please add, correct me if I missed
> > something.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii
> > > > Biletskyi <
> > > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > You are right, I removed ClusterMetadata
> > > > because
> > > > > we
> > > > > > > > have
> > > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > > what we need in TopicMetadata. Also, as
> Jay
> > > > > pointed
> > > > > > > out
> > > > > > > > > > > > earlier,
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > would like to have "orthogonal" API, but
> at
> > > the
> > > > > > same
> > > > > > > > time
> > > > > > > > > > we
> > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > But I like your idea and even have some
> > other
> > > > > > > arguments
> > > > > > > > > for
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > > There is also DescribeTopicRequest which
> > was
> > > > > > proposed
> > > > > > > > in
> > > > > > > > > > this
> > > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > > it returns topic configs, partitions,
> > > > replication
> > > > > > > > factor
> > > > > > > > > > plus
> > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > > leader replica. The later part is really
> > > > already
> > > > > > > there
> > > > > > > > in
> > > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > > So again we'll have to add stuff to TMR,
> > not
> > > to
> > > > > > > > duplicate
> > > > > > > > > > some
> > > > > > > > > > > > > info
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > newly added requests. However, this way
> > we'll
> > > > end
> > > > > > up
> > > > > > > > with
> > > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > > request which returns cluster metadata,
> > topic
> > > > > > > > replication
> > > > > > > > > > and
> > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > > plus partition replication data. Seems
> > > logical
> > > > to
> > > > > > > split
> > > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > > - ClusterMetadata (brokers + controller,
> > > maybe
> > > > > smth
> > > > > > > > else)
> > > > > > > > > > > > > > > > > > - TopicMetadata (topic info + partition
> > > > details)
> > > > > > > > > > > > > > > > > > But since current TMR is involved in lots
> > of
> > > > > places
> > > > > > > > > > (including
> > > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > > as I understand) this might be very
> serious
> > > > > change
> > > > > > > and
> > > > > > > > it
> > > > > > > > > > > > > probably
> > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel
> > Koshy <
> > > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >> I may be missing some context but
> > hopefully
> > > > this
> > > > > > > will
> > > > > > > > > > also be
> > > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > > >> today: I thought the earlier proposal
> > where
> > > > > there
> > > > > > > was
> > > > > > > > an
> > > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > > >> ClusterMetadata request was clearer and
> > > > > explicit.
> > > > > > > > During
> > > > > > > > > > the
> > > > > > > > > > > > > > course
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > >> this thread I think the conclusion was
> > that
> > > > the
> > > > > > main
> > > > > > > > > need
> > > > > > > > > > was
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> controller information and that can be
> > > rolled
> > > > > into
> > > > > > > the
> > > > > > > > > > topic
> > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > >> response but that seems a bit irrelevant
> > to
> > > > > topic
> > > > > > > > > > metadata.
> > > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > > >> think the full broker-list is also
> > > irrelevant
> > > > to
> > > > > > > topic
> > > > > > > > > > > > metadata,
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > >> it is already there and in use. I think
> > > there
> > > > is
> > > > > > > still
> > > > > > > > > > room
> > > > > > > > > > > > for
> > > > > > > > > > > > > an
> > > > > > > > > > > > > > > > > >> explicit ClusterMetadata request since
> > there
> > > > may
> > > > > > be
> > > > > > > > > other
> > > > > > > > > > > > > > > > > >> cluster-level information that we may
> want
> > > to
> > > > > add
> > > > > > > over
> > > > > > > > > > time
> > > > > > > > > > > > (and
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> have nothing to do with topic metadata).
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM
> +0200,
> > > > Andrii
> > > > > > > > > Biletskyi
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > 101. Okay, if you say that such use
> case
> > > is
> > > > > > > > > important. I
> > > > > > > > > > > > also
> > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > >> > using clientId for these purposes is
> > fine
> > > -
> > > > if
> > > > > > we
> > > > > > > > > > already
> > > > > > > > > > > > have
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > > >> > as part of all Wire protocol messages,
> > why
> > > > not
> > > > > > use
> > > > > > > > > that.
> > > > > > > > > > > > > > > > > >> > I will update KIP-4 page if nobody has
> > > other
> > > > > > ideas
> > > > > > > > > > (which
> > > > > > > > > > > > may
> > > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP
> > > > accordingly.
> > > > > I
> > > > > > > > think
> > > > > > > > > > we can
> > > > > > > > > > > > > add
> > > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > > >> > fine-grained error codes if some error
> > > code
> > > > > > > received
> > > > > > > > > in
> > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > >> > won't give enough context to return a
> > > > > > descriptive
> > > > > > > > > error
> > > > > > > > > > > > > message
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > Look forward to discussing all
> > outstanding
> > > > > > issues
> > > > > > > in
> > > > > > > > > > detail
> > > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun
> > Rao
> > > <
> > > > > > > > > > jun@confluent.io
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > 101. There may be a use case where
> you
> > > > only
> > > > > > want
> > > > > > > > the
> > > > > > > > > > > > topics
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > > >> > > manually by admins. Currently, you
> can
> > > do
> > > > > that
> > > > > > > by
> > > > > > > > > > > > disabling
> > > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> > > creation and issue topic creation
> from
> > > the
> > > > > > > > > > TopicCommand.
> > > > > > > > > > > > If
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > > >> > > topic creation completely on the
> > broker
> > > > and
> > > > > > > don't
> > > > > > > > > > have a
> > > > > > > > > > > > way
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > >> > > between topic creation requests from
> > the
> > > > > > regular
> > > > > > > > > > clients
> > > > > > > > > > > > and
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > > >> > > can't support manual topic creation
> > any
> > > > > more.
> > > > > > I
> > > > > > > > was
> > > > > > > > > > > > thinking
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > > >> > > way of distinguishing the clients
> > making
> > > > the
> > > > > > > topic
> > > > > > > > > > > > creation
> > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > >> > > using clientId. For example, the
> admin
> > > > tool
> > > > > > can
> > > > > > > > set
> > > > > > > > > > it to
> > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > > >> > > admin and the broker can treat that
> > > > clientId
> > > > > > > > > > specially.
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > Also, there is a related discussion
> in
> > > > > > > KAFKA-2020.
> > > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > 1. If leader is not available, we
> set
> > > the
> > > > > > > > partition
> > > > > > > > > > level
> > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > > >> > > 2. If a non-leader replica is not
> > > > available,
> > > > > > we
> > > > > > > > take
> > > > > > > > > > that
> > > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > >> > > the assigned replica list and isr in
> > the
> > > > > > > response.
> > > > > > > > > As
> > > > > > > > > > an
> > > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > > >> > > doing that, we set the partition
> level
> > > > error
> > > > > > > code
> > > > > > > > to
> > > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > This has a few problems. First,
> > > > > > > > ReplicaNotAvailable
> > > > > > > > > > > > probably
> > > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > > >> > > an error, at least for the normal
> > > > > > > > producer/consumer
> > > > > > > > > > > > clients
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > > >> > > to find out the leader. Second, it
> can
> > > > > happen
> > > > > > > that
> > > > > > > > > > both
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> > > another replica are not available at
> > the
> > > > > same
> > > > > > > > time.
> > > > > > > > > > There
> > > > > > > > > > > > is
> > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > >> > > to indicate both. Third, even if a
> > > replica
> > > > > is
> > > > > > > not
> > > > > > > > > > > > available,
> > > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > > >> > > useful to return its replica id
> since
> > > some
> > > > > > > clients
> > > > > > > > > > (e.g.
> > > > > > > > > > > > > admin
> > > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > One way to address this issue is to
> > > always
> > > > > > > return
> > > > > > > > > the
> > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > > >> > > leader, assigned replicas, and isr
> > > > > regardless
> > > > > > of
> > > > > > > > > > whether
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > > >> > > broker is live or not. Since we also
> > > > return
> > > > > > the
> > > > > > > > list
> > > > > > > > > > of
> > > > > > > > > > > > live
> > > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > client can figure out whether a
> leader
> > > or
> > > > a
> > > > > > > > replica
> > > > > > > > > is
> > > > > > > > > > > > live
> > > > > > > > > > > > > or
> > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > > >> > > accordingly. This way, we don't need
> > to
> > > > set
> > > > > > the
> > > > > > > > > > partition
> > > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > > >> > > when the leader or a replica is not
> > > > > available.
> > > > > > > > This
> > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > > >> > > protocol, but does change the
> > semantics.
> > > > > Since
> > > > > > > we
> > > > > > > > > are
> > > > > > > > > > > > > evolving
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > > >> > > of TopicMetadataRequest here, we can
> > > > > > potentially
> > > > > > > > > > piggyback
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > 102.1 For those types of errors due
> to
> > > > > invalid
> > > > > > > > > input,
> > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > >> > > guard it at parameter validation
> time
> > > and
> > > > > > throw
> > > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > > >> > > without even sending the request to
> > the
> > > > > > broker?
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM,
> > Andrii
> > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > 101. If I understand you
> correctly,
> > > you
> > > > > are
> > > > > > > > saying
> > > > > > > > > > > > future
> > > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > > >> > > > will be ported to TMR_V1) won't be
> > > able
> > > > to
> > > > > > > > > > automatically
> > > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > >> > > > unconditionally remove topic
> > creation
> > > > from
> > > > > > > > there).
> > > > > > > > > > But
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > > >> > > > Ok, about your proposal: I'm not a
> > big
> > > > fan
> > > > > > > too,
> > > > > > > > > > when it
> > > > > > > > > > > > > > comes
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > > >> > > > clients directly in protocol
> schema.
> > > And
> > > > > > also
> > > > > > > > I'm
> > > > > > > > > > not
> > > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > > >> > > > auto.create.topics.enable is a
> > server
> > > > side
> > > > > > > > > > > > configuration.
> > > > > > > > > > > > > > Can
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > > >> > > > in future versions, add this
> setting
> > > to
> > > > > > > producer
> > > > > > > > > and
> > > > > > > > > > > > based
> > > > > > > > > > > > > > on
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > > >> > > > UnknownTopic create topic
> explicitly
> > > by
> > > > a
> > > > > > > > separate
> > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because we
> want
> > > to
> > > > > > > support
> > > > > > > > > > batching
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > > >> > > > want to give descriptive error
> > > messages
> > > > > for
> > > > > > > > > clients.
> > > > > > > > > > > > Since
> > > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > > >> > > > to construct such messages (e.g.
> > > > > AdminClient
> > > > > > > > layer
> > > > > > > > > > can
> > > > > > > > > > > > > know
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > > >> > > > means two cases: either invalid
> > > number -
> > > > > > e.g.
> > > > > > > > -1;
> > > > > > > > > or
> > > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > > >> > > > partitions argument wasn't) - I
> > > wrapped
> > > > > > > > responses
> > > > > > > > > in
> > > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > > >> > > > other ideas, this was just initial
> > > > > version.
> > > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change
> > that
> > > to
> > > > > > > > probably
> > > > > > > > > > some
> > > > > > > > > > > > > other
> > > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM,
> Jun
> > > > Rao <
> > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > 101. That's what I was thinking
> > too,
> > > > but
> > > > > > it
> > > > > > > > may
> > > > > > > > > > not be
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > > >> > > > > we can let it not trigger auto
> > topic
> > > > > > > creation.
> > > > > > > > > > Then,
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > > >> > > > > if it gets an
> > UnknownTopicException,
> > > > it
> > > > > > can
> > > > > > > > > > explicitly
> > > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > > >> > > > > createTopicRequest for auto
> topic
> > > > > > creation.
> > > > > > > On
> > > > > > > > > the
> > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > > >> > > > > never issue createTopicRequest.
> > This
> > > > > works
> > > > > > > > when
> > > > > > > > > > auto
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > > >> > > > > enabled on the broker side.
> > > However, I
> > > > > am
> > > > > > > not
> > > > > > > > > > sure how
> > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > > >> > > > > when auto topic creation is
> > disabled
> > > > on
> > > > > > the
> > > > > > > > > broker
> > > > > > > > > > > > side.
> > > > > > > > > > > > > > In
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > >> > > > > want to have a way to manually
> > > create
> > > > a
> > > > > > > topic,
> > > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > >> > > > > commands. However, then we need
> a
> > > way
> > > > to
> > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > > >> > > > > issued from the producer clients
> > and
> > > > the
> > > > > > > admin
> > > > > > > > > > tools.
> > > > > > > > > > > > > May
> > > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > >> > > > > new field in createTopicRequest
> > and
> > > > set
> > > > > it
> > > > > > > > > > differently
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > > >> > > > > client and the admin client.
> > > However,
> > > > I
> > > > > am
> > > > > > > not
> > > > > > > > > > sure if
> > > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring existing
> > > requests
> > > > > is a
> > > > > > > > > > non-trivial
> > > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > > >> > > > > posted some comments in
> > KAFKA-1927.
> > > We
> > > > > > will
> > > > > > > > > > probably
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > > >> > > > > first, before adding the new
> logic
> > > in
> > > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird that we
> > > return
> > > > > > > > exception
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > >> > > > > we should either return error
> code
> > > or
> > > > > > throw
> > > > > > > an
> > > > > > > > > > > > exception
> > > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > > >> > > > > 102.2. We probably shouldn't
> > > > explicitly
> > > > > > use
> > > > > > > > the
> > > > > > > > > > > > request
> > > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > > >> > > > > Not every request evolution
> > requires
> > > > an
> > > > > > api
> > > > > > > > > > change.
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM,
> > > > Andrii
> > > > > > > > > Biletskyi
> > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly>
> > wrote:
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > Thanks for you comments.
> Answers
> > > > > inline:
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > 100. There are a few fields
> such
> > > as
> > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized
> that
> > > are
> > > > > > > > > represented
> > > > > > > > > > as a
> > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > >> > > > > > > composite structures in
> json.
> > > > Could
> > > > > we
> > > > > > > > > flatten
> > > > > > > > > > > > them
> > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > arrays/records?
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > Yes, now with Admin Client
> this
> > > > looks
> > > > > a
> > > > > > > bit
> > > > > > > > > > weird.
> > > > > > > > > > > > My
> > > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > > >> > > > > > ReassignPartitionCommand
> accepts
> > > > input
> > > > > > in
> > > > > > > > > json,
> > > > > > > > > > we
> > > > > > > > > > > > > want
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > > >> > > > > > interfaces unchanged, where
> > > > possible.
> > > > > > > > > > > > > > > > > >> > > > > > If we port it to deserialized
> > > > format,
> > > > > in
> > > > > > > CLI
> > > > > > > > > > (/tools
> > > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > > >> > > > > > json library since /tools is
> > > written
> > > > > in
> > > > > > > java
> > > > > > > > > and
> > > > > > > > > > > > we'll
> > > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > > >> > > > > > provided by a user. Can we
> > quickly
> > > > > agree
> > > > > > > on
> > > > > > > > > what
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > 101. Does TopicMetadataRequest
> > v1
> > > > > still
> > > > > > > > > trigger
> > > > > > > > > > auto
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now that
> > we
> > > > > have a
> > > > > > > > > > separate
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > >> > > > > > > you thought about how the
> new
> > > > > > > > > > createTopicRequest
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > producer/consumer
> > > > > > > > > > client,
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > >> > > > > > > tools? For example, ideally,
> > we
> > > > > don't
> > > > > > > want
> > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto
> topic
> > > > > > creation.
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > I agree, this strange logic
> > should
> > > > be
> > > > > > > fixed.
> > > > > > > > > > I'm not
> > > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > > >> > > > > > correct me if I'm wrong, but
> it
> > > > > doesn't
> > > > > > > look
> > > > > > > > > > like a
> > > > > > > > > > > > > hard
> > > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > > >> > > > > > leverage AdminClient for that
> in
> > > > > > Producer
> > > > > > > > and
> > > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant getting
> rid
> > > of
> > > > > > scala
> > > > > > > > > > classes
> > > > > > > > > > > > > > > > > >> > > > > > > like
> HeartbeatRequestAndHeader
> > > and
> > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> > adding
> > > > the
> > > > > > new
> > > > > > > > > > requests
> > > > > > > > > > > > for
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > >> > > > > > > However, the long term plan
> is
> > > to
> > > > > get
> > > > > > > rid
> > > > > > > > of
> > > > > > > > > > all
> > > > > > > > > > > > > those
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > java request/response in the
> > > > client.
> > > > > > > Since
> > > > > > > > > > this
> > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > > > requests,
> > > > > > > > perhaps
> > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing scala
> > > > requests
> > > > > > > first
> > > > > > > > > > before
> > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > Yes, looks like I
> misunderstood
> > > the
> > > > > > point
> > > > > > > of
> > > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > > >> > > > > > rework that. The only thing is
> > > that
> > > > I
> > > > > > > don't
> > > > > > > > > see
> > > > > > > > > > any
> > > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > > >> > > > > > least one existing protocol
> > > message.
> > > > > > Thus,
> > > > > > > > as
> > > > > > > > > I
> > > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > > > > >> > > > > > Re porting all existing RQ/RP
> in
> > > > this
> > > > > > > patch.
> > > > > > > > > > Sounds
> > > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > > >> > > > > > requirement to have Admin KIP
> > > done,
> > > > > I'm
> > > > > > > > afraid
> > > > > > > > > > this
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > > >> > > > > > There are 13 protocol messages
> > and
> > > > all
> > > > > > > that
> > > > > > > > > > would
> > > > > > > > > > > > > > require
> > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > > >> > > > > > intensive manual testing, no?
> > I'm
> > > > > afraid
> > > > > > > I'm
> > > > > > > > > > not the
> > > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > > >> > > > > > Kafka core internals :). Let
> me
> > > know
> > > > > > your
> > > > > > > > > > thoughts
> > > > > > > > > > > > on
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40
> AM,
> > > Jun
> > > > > > Rao <
> > > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > 100. There are a few fields
> > such
> > > > as
> > > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized
> that
> > > are
> > > > > > > > > represented
> > > > > > > > > > as a
> > > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > > >> > > > > > > composite structures in
> json.
> > > > Could
> > > > > we
> > > > > > > > > flatten
> > > > > > > > > > > > them
> > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > > arrays/records?
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > 101. Does
> TopicMetadataRequest
> > > v1
> > > > > > still
> > > > > > > > > > trigger
> > > > > > > > > > > > auto
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now that
> > we
> > > > > have a
> > > > > > > > > > separate
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > > >> > > > > > > you thought about how the
> new
> > > > > > > > > > createTopicRequest
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > > producer/consumer
> > > > > > > > > > client,
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > > >> > > > > > > tools? For example, ideally,
> > we
> > > > > don't
> > > > > > > want
> > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto
> topic
> > > > > > creation.
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant getting
> > rid
> > > > of
> > > > > > > scala
> > > > > > > > > > classes
> > > > > > > > > > > > > > > > > >> > > > > > > like
> HeartbeatRequestAndHeader
> > > and
> > > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> > adding
> > > > the
> > > > > > new
> > > > > > > > > > requests
> > > > > > > > > > > > for
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > > >> > > > > > > However, the long term plan
> is
> > > to
> > > > > get
> > > > > > > rid
> > > > > > > > of
> > > > > > > > > > all
> > > > > > > > > > > > > those
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > java request/response in the
> > > > client.
> > > > > > > Since
> > > > > > > > > > this
> > > > > > > > > > > > KIP
> > > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > > > requests,
> > > > > > > > perhaps
> > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > clean up the existing scala
> > > > requests
> > > > > > > first
> > > > > > > > > > before
> > > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37
> > PM,
> > > > > > Andrii
> > > > > > > > > > Biletskyi
> > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly
> >
> > > > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > As said above - I list
> again
> > > all
> > > > > > > > comments
> > > > > > > > > > from
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > > >> > > > > > > > can see what's left and
> > > finalize
> > > > > all
> > > > > > > > > pending
> > > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > > > > > >> > > > > > > > 1. This is much needed
> > > > > > functionality,
> > > > > > > > but
> > > > > > > > > > there
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > > >> > > > > > > > really think these
> protocols
> > > > > > through.
> > > > > > > We
> > > > > > > > > > really
> > > > > > > > > > > > > want
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > > >> > > > > > > > of well thought-out,
> > > orthoganol
> > > > > > apis.
> > > > > > > > For
> > > > > > > > > > this
> > > > > > > > > > > > > > reason
> > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > > >> > > > > > > > important to think through
> > the
> > > > end
> > > > > > > state
> > > > > > > > > > even if
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > won't implement in the
> first
> > > > > phase.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Definitely behind this.
> > > Would
> > > > > > > > > appreciate
> > > > > > > > > > if
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's please please
> > please
> > > > wait
> > > > > > > until
> > > > > > > > > we
> > > > > > > > > > have
> > > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > > >> > > > > > > > to the new java protocol
> > > > > > definitions.
> > > > > > > If
> > > > > > > > > we
> > > > > > > > > > add
> > > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > > >> > > > > > > > objects that is just
> > > generating
> > > > > more
> > > > > > > > work
> > > > > > > > > > for
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > patch -
> > > > > > removed
> > > > > > > > > scala
> > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 3. This proposal
> introduces
> > a
> > > > new
> > > > > > type
> > > > > > > > of
> > > > > > > > > > > > optional
> > > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > > >> > > > > > > > inconsistent with
> everything
> > > > else
> > > > > in
> > > > > > > the
> > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > > >> > > > > > > > other marker value. You
> > could
> > > > > argue
> > > > > > > > either
> > > > > > > > > > way
> > > > > > > > > > > > but
> > > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > >> > > > > > > > for consistency. For
> clients
> > > > that
> > > > > > > > > > implemented
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > > >> > > > > > > > than our scala code these
> > > basic
> > > > > > > > primitives
> > > > > > > > > > are
> > > > > > > > > > > > > hard
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > patch -
> > > > > > removed
> > > > > > > > > > MaybeOf
> > > > > > > > > > > > > type
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This
> > seems
> > > > to
> > > > > > > > > duplicate
> > > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > >> > > > > > > > brokers, topics, and
> > > > partitions. I
> > > > > > > think
> > > > > > > > > we
> > > > > > > > > > > > should
> > > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > > >> > > > > > > > ClusterMetadataRequest (or
> > > just
> > > > > > > > > > MetadataRequest)
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > controller. Or are there
> > other
> > > > > > things
> > > > > > > we
> > > > > > > > > > could
> > > > > > > > > > > > add
> > > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: I agree. Updated the
> KIP.
> > > > Let's
> > > > > > > > extends
> > > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a tendency to
> try
> > > to
> > > > > > make a
> > > > > > > > lot
> > > > > > > > > > of
> > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > particular nodes. This
> adds
> > a
> > > > lot
> > > > > of
> > > > > > > > > burden
> > > > > > > > > > for
> > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > > >> > > > > > > > sounds easy but each
> > discovery
> > > > can
> > > > > > > fail
> > > > > > > > in
> > > > > > > > > > many
> > > > > > > > > > > > > > parts
> > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > > >> > > > > > > > full state machine to do
> > > > right). I
> > > > > > > think
> > > > > > > > > we
> > > > > > > > > > > > should
> > > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > > >> > > > > > > > commands and ideally as
> many
> > > of
> > > > > the
> > > > > > > > other
> > > > > > > > > > apis
> > > > > > > > > > > > as
> > > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > > >> > > > > > > > brokers and just redirect
> to
> > > the
> > > > > > > > > controller
> > > > > > > > > > on
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > > >> > > > > > > > there would be a general
> way
> > > to
> > > > > > > > > encapsulate
> > > > > > > > > > this
> > > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: It's a very interesting
> > > idea,
> > > > > but
> > > > > > > > seems
> > > > > > > > > > there
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > > >> > > > > > > > feature (like performance
> > > > > > > > considerations,
> > > > > > > > > > how
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > > >> > > > > > > > I believe this shouldn't
> be
> > a
> > > > > > blocker.
> > > > > > > > If
> > > > > > > > > > this
> > > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > > >> > > > > > > > point it won't affect
> Admin
> > > > > changes
> > > > > > -
> > > > > > > at
> > > > > > > > > > least
> > > > > > > > > > > > no
> > > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 6. We should probably
> > > normalize
> > > > > the
> > > > > > > key
> > > > > > > > > > value
> > > > > > > > > > > > > pairs
> > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > >> > > > > > > > than embedding a new
> > > formatting.
> > > > > So
> > > > > > > two
> > > > > > > > > > strings
> > > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > patch -
> > > > > > > > normalized
> > > > > > > > > > > > configs
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the postcondition of
> > > these
> > > > > > APIs
> > > > > > > > that
> > > > > > > > > > the
> > > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > > >> > > > > > > > the command has been
> > > completed?
> > > > It
> > > > > > is
> > > > > > > a
> > > > > > > > > lot
> > > > > > > > > > more
> > > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > > >> > > > > > > > been completed so you know
> > > that
> > > > if
> > > > > > you
> > > > > > > > > > create a
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > it you won't get an
> > exception
> > > > > about
> > > > > > > > there
> > > > > > > > > > being
> > > > > > > > > > > > no
> > > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: For long running
> requests
> > > > (like
> > > > > > > > > reassign
> > > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > > >> > > > > > > > command has begun - so we
> > > don't
> > > > > > block
> > > > > > > > the
> > > > > > > > > > > > client.
> > > > > > > > > > > > > In
> > > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > > >> > > > > > > > topic commands, this will
> be
> > > > > > > refactored
> > > > > > > > > and
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > > >> > > > > > > > immediately, since the
> > > > Controller
> > > > > > will
> > > > > > > > > serve
> > > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > > >> > > > > > > > (follow-up ticket
> > KAFKA-1777).
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe topic and list
> > > > topics
> > > > > > > > > duplicate
> > > > > > > > > > a
> > > > > > > > > > > > lot
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > > >> > > > > > > > request. Is there a reason
> > to
> > > > give
> > > > > > > back
> > > > > > > > > > topics
> > > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > > >> > > > > > > > like if we just make the
> > > > > > > post-condition
> > > > > > > > of
> > > > > > > > > > the
> > > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > topic is deleted that will
> > get
> > > > rid
> > > > > > of
> > > > > > > > the
> > > > > > > > > > need
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> > patch -
> > > > > > removed
> > > > > > > > > > topics
> > > > > > > > > > > > > marked
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we consider
> > batching
> > > > > these
> > > > > > > > > > requests?
> > > > > > > > > > > > We
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > allow multiple operations
> to
> > > be
> > > > > > > batched.
> > > > > > > > > My
> > > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > >> > > > > > > > we will get a lot of code
> > that
> > > > > does
> > > > > > > > > > something
> > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > > >> > > > > > > > this code will work great
> > when
> > > > you
> > > > > > > test
> > > > > > > > > on 5
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> please
> > > > check
> > > > > > > "Topic
> > > > > > > > > > Admin
> > > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 10. I think we should also
> > > > discuss
> > > > > > how
> > > > > > > > we
> > > > > > > > > > want
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > > >> > > > > > > > client api for these
> > > operations.
> > > > > > > > Currently
> > > > > > > > > > > > people
> > > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > > >> > > > > > > > is totally sketchy. I
> think
> > we
> > > > > > > probably
> > > > > > > > > need
> > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > > >> > > > > > > > that exposes
> administrative
> > > > > > > > functionality.
> > > > > > > > > > We
> > > > > > > > > > > > will
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > properly test the new
> apis,
> > I
> > > > > > suspect.
> > > > > > > > We
> > > > > > > > > > should
> > > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> please
> > > > check
> > > > > > > "Admin
> > > > > > > > > > Client"
> > > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 11. The other information
> > that
> > > > > would
> > > > > > > be
> > > > > > > > > > really
> > > > > > > > > > > > > > useful
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > >> > > > > > > > information about
> > > > partitions--how
> > > > > > much
> > > > > > > > > data
> > > > > > > > > > is
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > > >> > > > > > > > the segment offsets, what
> is
> > > the
> > > > > > > log-end
> > > > > > > > > > offset
> > > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > > >> > > > > > > > the compaction point,
> etc. I
> > > > think
> > > > > > > that
> > > > > > > > > done
> > > > > > > > > > > > right
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > successor to the very
> > awkward
> > > > > > > > > OffsetRequest
> > > > > > > > > > we
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > > >> > > > > > > > be resolved in a separate
> > KIP
> > > /
> > > > > jira
> > > > > > > > > ticket.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 12. Generally we can do
> good
> > > > error
> > > > > > > > > handling
> > > > > > > > > > > > > without
> > > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > > >> > > > > > > > messages. I.e. generally
> the
> > > > > client
> > > > > > > has
> > > > > > > > > the
> > > > > > > > > > > > > context
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > > >> > > > > > > > an error that the topic
> > > doesn't
> > > > > > exist
> > > > > > > to
> > > > > > > > > say
> > > > > > > > > > > > > "Topic
> > > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > > >> > > > > > > > than "error code 14" (or
> > > > > whatever).
> > > > > > > > Maybe
> > > > > > > > > > there
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > > >> > > > > > > > this is hard? If we want
> to
> > > add
> > > > > > > > > server-side
> > > > > > > > > > > > error
> > > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > > >> > > > > > > > need to do this in a
> > > consistent
> > > > > way
> > > > > > > > across
> > > > > > > > > > the
> > > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP -
> please
> > > > check
> > > > > > > > > "Protocol
> > > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > comprehensive,
> fine-grained
> > > list
> > > > > of
> > > > > > > > error
> > > > > > > > > > codes.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe topic
> request:
> > it
> > > > > would
> > > > > > > be
> > > > > > > > > > great to
> > > > > > > > > > > > > go
> > > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex for this
> > > > request.
> > > > > > For
> > > > > > > > > > example,
> > > > > > > > > > > > a
> > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > > >> > > > > > > > the topic command is to
> list
> > > all
> > > > > > > topics
> > > > > > > > > > whose
> > > > > > > > > > > > > config
> > > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > > >> > > > > > > > topic name regex then we
> > have
> > > to
> > > > > > first
> > > > > > > > > > retrieve
> > > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > > >> > > > > > > > description info and then
> > > filter
> > > > > at
> > > > > > > the
> > > > > > > > > > client
> > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in create
> > > topic:
> > > > > > this
> > > > > > > is
> > > > > > > > > > related
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > > >> > > > > > > > maybe we can add another
> > > > metadata
> > > > > > K-V
> > > > > > > or
> > > > > > > > > > just a
> > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > > >> > > > > > > > with config K-V in create
> > > topic
> > > > > like
> > > > > > > we
> > > > > > > > > did
> > > > > > > > > > for
> > > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > > >> > > > > > > > field can be quite useful
> in
> > > > > storing
> > > > > > > > > > information
> > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > > >> > > > > > > > who issue the create
> > command,
> > > > etc,
> > > > > > > which
> > > > > > > > > is
> > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant setting. Then
> > in
> > > > the
> > > > > > > > describe
> > > > > > > > > > topic
> > > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > > >> > > > > > > > on regex of the metadata
> > > field.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: As discussed it is very
> > > > > > interesting
> > > > > > > > but
> > > > > > > > > > can
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > > >> > > > > > > > we have some basic
> > > functionality
> > > > > > > there.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > 15. Today all the admin
> > > > operations
> > > > > > are
> > > > > > > > > > async in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > > >> > > > > > > > return once it is written
> in
> > > ZK,
> > > > > and
> > > > > > > > that
> > > > > > > > > > is why
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > > testUtil.waitForTopicCreated()
> > > > > > /
> > > > > > > > > verify
> > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > > >> > > > > > > > request, etc. With admin
> > > > requests
> > > > > we
> > > > > > > > could
> > > > > > > > > > add a
> > > > > > > > > > > > > > flag
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > > >> > > > > > > > synchronous requests; when
> > it
> > > is
> > > > > > > turned
> > > > > > > > > on,
> > > > > > > > > > the
> > > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > > >> > > > > > > > until the request has been
> > > > > > completed.
> > > > > > > > And
> > > > > > > > > > for
> > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > > >> > > > > > > > "token" field in the
> > response,
> > > > and
> > > > > > > then
> > > > > > > > > only
> > > > > > > > > > > > need
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > > >> > > > > > > > verification request" with
> > the
> > > > > given
> > > > > > > > token
> > > > > > > > > > to
> > > > > > > > > > > > > check
> > > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: I see your point. My
> idea
> > > was
> > > > > to
> > > > > > > > > provide
> > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > > >> > > > > > > > long running request,
> where
> > > > > needed.
> > > > > > We
> > > > > > > > can
> > > > > > > > > > do it
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > > >> > > > > > > > concern is that
> introducing
> > a
> > > > > token
> > > > > > we
> > > > > > > > > again
> > > > > > > > > > > > will
> > > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > > >> > > > > > > > to do similar thing
> > > introducing
> > > > > > single
> > > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > > >> > > > > > > > this idea because we
> wanted
> > to
> > > > > have
> > > > > > > > schema
> > > > > > > > > > > > > defined.
> > > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > > > > >> > > > > > > > a) have fixed schema but
> > > > introduce
> > > > > > > each
> > > > > > > > > > time new
> > > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > > > > > > > > >> > > > > > > > b) use one request for
> > > > > verification
> > > > > > > but
> > > > > > > > > > > > generalize
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > > >> > > > > > > > I'm fine with whatever
> > > decision
> > > > > > > > community
> > > > > > > > > > come
> > > > > > > > > > > > to.
> > > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > > > > > >> > > > > > > > 16. Specifically for
> > > ownership,
> > > > I
> > > > > > > think
> > > > > > > > > the
> > > > > > > > > > plan
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > > >> > > > > > > > like you are describing
> ACL)
> > > via
> > > > > an
> > > > > > > > > external
> > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > > >> > > > > > > > I remember KIP-11
> described
> > > > this,
> > > > > > but
> > > > > > > I
> > > > > > > > > > can't
> > > > > > > > > > > > find
> > > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no problem. Not
> > sure
> > > > > though
> > > > > > > how
> > > > > > > > > we
> > > > > > > > > > are
> > > > > > > > > > > > > > going
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > > >> > > > > > > > will be committed first
> and
> > > > > include
> > > > > > > > > changes
> > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I added this note
> to
> > > > "Open
> > > > > > > > > > Questions"
> > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at
> > 12:34
> > > > AM,
> > > > > > > Andrii
> > > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > > >> > > > > > > >
> andrii.biletskyi@stealth.ly
> > >
> > > > > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > Today I uploaded the
> patch
> > > > that
> > > > > > > covers
> > > > > > > > > > some of
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf
> optional
> > > > type
> > > > > > > > > > > > > > > > > >> > > > > > > > > - switched to java
> > protocol
> > > > > > > > definitions
> > > > > > > > > > > > > > > > > >> > > > > > > > > - simplified messages
> > > > > (normalized
> > > > > > > > > configs,
> > > > > > > > > > > > > removed
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > I also updated the KIP-4
> > > with
> > > > > > > > respective
> > > > > > > > > > > > changes
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin Operations
> > ->
> > > > > > updated
> > > > > > > > Wire
> > > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > > >> > > > > > > > > - Remove ClusterMetadata
> > ->
> > > > > > changed
> > > > > > > to
> > > > > > > > > > extend
> > > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > > >> > > > > > > > > - Admin Client ->
> updated
> > my
> > > > > > initial
> > > > > > > > > > proposal
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > > >> > > > > > > > > - Error codes ->
> proposed
> > > > > > > fine-grained
> > > > > > > > > > error
> > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > I will also send a
> > separate
> > > > > email
> > > > > > to
> > > > > > > > > > cover all
> > > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at
> > 9:26
> > > > PM,
> > > > > > > Gwen
> > > > > > > > > > Shapira
> > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > > >> > > > > > > > >> It actually specifies
> > > changes
> > > > > to
> > > > > > > the
> > > > > > > > > > Metadata
> > > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are
> consistent
> > in
> > > > > this
> > > > > > > > regard
> > > > > > > > > > will
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at
> > > 12:21
> > > > > PM,
> > > > > > > > Gwen
> > > > > > > > > > > > Shapira
> > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > Specifically for
> > > > ownership, I
> > > > > > > think
> > > > > > > > > the
> > > > > > > > > > > > plan
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > like you are
> describing
> > > > ACL)
> > > > > > via
> > > > > > > an
> > > > > > > > > > > > external
> > > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > I remember KIP-11
> > > described
> > > > > > this,
> > > > > > > > > but I
> > > > > > > > > > > > can't
> > > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I think
> > KIP-4
> > > > > > focuses
> > > > > > > > on
> > > > > > > > > > > > getting
> > > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > exists from Kafka
> > > brokers,
> > > > > not
> > > > > > on
> > > > > > > > > > adding
> > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > should exist but
> > doesn't
> > > > yet?
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015
> at
> > > > 6:37
> > > > > > AM,
> > > > > > > > > > Guozhang
> > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just want to
> > elaborate a
> > > > bit
> > > > > > > more
> > > > > > > > on
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> describe-topic based
> > on
> > > > > > config /
> > > > > > > > > > metadata
> > > > > > > > > > > > in
> > > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The
> > main
> > > > > > > motivation
> > > > > > > > > is
> > > > > > > > > > to
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms, which I
> > > think
> > > > is
> > > > > > > quite
> > > > > > > > > > > > important
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> architecture: today
> > > anyone
> > > > > can
> > > > > > > > > create
> > > > > > > > > > > > topics
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> there is no concept
> or
> > > > > > > "ownership"
> > > > > > > > > of
> > > > > > > > > > > > topics
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> users. For example,
> at
> > > > > > LinkedIn
> > > > > > > we
> > > > > > > > > > > > basically
> > > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> some casual topic
> name
> > > > > prefix,
> > > > > > > > which
> > > > > > > > > > is a
> > > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale our
> > customers.
> > > It
> > > > > > would
> > > > > > > > be
> > > > > > > > > > great
> > > > > > > > > > > > to
> > > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics
> > that
> > > > is
> > > > > > > > created
> > > > > > > > > > by me.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics
> > > whose
> > > > > > > > retention
> > > > > > > > > > time
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics
> > > whose
> > > > > > > writable
> > > > > > > > > > group
> > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> authorization),
> etc..
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> One possible way to
> > > > achieve
> > > > > > this
> > > > > > > > is
> > > > > > > > > to
> > > > > > > > > > > > add a
> > > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> create-topic
> request,
> > > > whose
> > > > > > > value
> > > > > > > > > will
> > > > > > > > > > > > also
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic; then
> > > > describe-topics
> > > > > > can
> > > > > > > > > > choose to
> > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2) config K-V
> > > > > matching,
> > > > > > > 3)
> > > > > > > > > > metadata
> > > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015
> at
> > > > 4:37
> > > > > > PM,
> > > > > > > > > > Guozhang
> > > > > > > > > > > > > Wang
> > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for the
> > updated
> > > > > wiki.
> > > > > > A
> > > > > > > > few
> > > > > > > > > > > > comments
> > > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error
> description
> > in
> > > > > > > > response: I
> > > > > > > > > > think
> > > > > > > > > > > > > if
> > > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> several different
> > error
> > > > > cases
> > > > > > > > then
> > > > > > > > > we
> > > > > > > > > > > > > should
> > > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In general
> the
> > > > > > errorCode
> > > > > > > > > > itself
> > > > > > > > > > > > > would
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> describing the
> server
> > > > side
> > > > > > > > errors.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic
> > > > request:
> > > > > it
> > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > > great
> > > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex
> for
> > > this
> > > > > > > > request.
> > > > > > > > > > For
> > > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic command
> is
> > to
> > > > > list
> > > > > > > all
> > > > > > > > > > topics
> > > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex
> then
> > > we
> > > > > have
> > > > > > > to
> > > > > > > > > > first
> > > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> description info
> and
> > > then
> > > > > > > filter
> > > > > > > > at
> > > > > > > > > > the
> > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in
> > > create
> > > > > > topic:
> > > > > > > > > this
> > > > > > > > > > is
> > > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can add
> > > another
> > > > > > > metadata
> > > > > > > > > > K-V or
> > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with config K-V in
> > > create
> > > > > > topic
> > > > > > > > > like
> > > > > > > > > > we
> > > > > > > > > > > > did
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field can be quite
> > > useful
> > > > > in
> > > > > > > > > storing
> > > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue the
> create
> > > > > command,
> > > > > > > > etc,
> > > > > > > > > > which
> > > > > > > > > > > > is
> > > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant
> setting.
> > > > Then
> > > > > in
> > > > > > > the
> > > > > > > > > > > > describe
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of the
> > > metadata
> > > > > > field.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all the
> > admin
> > > > > > > operations
> > > > > > > > > are
> > > > > > > > > > > > async
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> return once it is
> > > written
> > > > > in
> > > > > > > ZK,
> > > > > > > > > and
> > > > > > > > > > that
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> request, etc. With
> > > admin
> > > > > > > requests
> > > > > > > > > we
> > > > > > > > > > > > could
> > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous
> requests;
> > > > when
> > > > > it
> > > > > > > is
> > > > > > > > > > turned
> > > > > > > > > > > > on,
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until the request
> has
> > > > been
> > > > > > > > > > completed. And
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token" field in
> the
> > > > > > response,
> > > > > > > > and
> > > > > > > > > > then
> > > > > > > > > > > > > only
> > > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> verification
> request"
> > > > with
> > > > > > the
> > > > > > > > > given
> > > > > > > > > > > > token
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for extending
> > > > > Metadata
> > > > > > > > > request
> > > > > > > > > > to
> > > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> information, and
> then
> > > we
> > > > > can
> > > > > > > > remove
> > > > > > > > > > the
> > > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015
> > at
> > > > > 10:23
> > > > > > > AM,
> > > > > > > > > Joel
> > > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for sending
> > > that
> > > > > out
> > > > > > > Joe
> > > > > > > > -
> > > > > > > > > I
> > > > > > > > > > > > don't
> > > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so if
> > notes
> > > > can
> > > > > be
> > > > > > > > sent
> > > > > > > > > > out
> > > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02,
> 2015
> > > at
> > > > > > > > 09:16:13AM
> > > > > > > > > > > > -0800,
> > > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for
> sending
> > > > this
> > > > > > out
> > > > > > > > Joe.
> > > > > > > > > > > > Looking
> > > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2,
> > 2015
> > > at
> > > > > > 6:46
> > > > > > > > AM,
> > > > > > > > > > Joe
> > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just
> sent
> > > > out a
> > > > > > > > google
> > > > > > > > > > > > hangout
> > > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I
> found
> > > > > working
> > > > > > > on
> > > > > > > > a
> > > > > > > > > > KIP.
> > > > > > > > > > > > If
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me know
> and
> > > can
> > > > > > update
> > > > > > > > it,
> > > > > > > > > > np.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We should do
> > this
> > > > > every
> > > > > > > > > Tuesday
> > > > > > > > > > @
> > > > > > > > > > > > 2pm
> > > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to make a
> > > > google
> > > > > > > > account
> > > > > > > > > > so we
> > > > > > > > > > > > > can
> > > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in progress
> and
> > > > > related
> > > > > > > JIRA
> > > > > > > > > > that
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb
> 24,
> > > 2015
> > > > > at
> > > > > > > 2:59
> > > > > > > > > > PM, Jay
> > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on
> > > > Google
> > > > > > > > hangouts
> > > > > > > > > > that
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> available on
> > > > youtube.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb
> 24,
> > > > 2015
> > > > > at
> > > > > > > > 11:49
> > > > > > > > > > AM,
> > > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > jholoman@cloudera.com
> > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're happy
> > to
> > > > send
> > > > > > > out a
> > > > > > > > > > Webex
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions if
> > > there
> > > > > is
> > > > > > > > > > interest and
> > > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb
> > 24,
> > > > > 2015
> > > > > > at
> > > > > > > > > > 11:28 AM,
> > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try
> > to
> > > > get
> > > > > > the
> > > > > > > > > > technical
> > > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there is
> > some
> > > > > > benefit
> > > > > > > > to
> > > > > > > > > > live
> > > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post
> > > > > > instructions
> > > > > > > > and
> > > > > > > > > > give
> > > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday
> at
> > > that
> > > > > > time
> > > > > > > > > would
> > > > > > > > > > work
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue,
> Feb
> > > 24,
> > > > > > 2015
> > > > > > > at
> > > > > > > > > > 8:18
> > > > > > > > > > > > AM,
> > > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly
> > > would
> > > > be
> > > > > > > great
> > > > > > > > > > maybe
> > > > > > > > > > > > > like
> > > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't
> > > mind
> > > > > > google
> > > > > > > > > > hangout
> > > > > > > > > > > > but
> > > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we know
> > the
> > > > > > apache
> > > > > > > > irc
> > > > > > > > > > > > channel
> > > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes?
> We
> > > can
> > > > > pull
> > > > > > > > > > transcripts
> > > > > > > > > > > > > too
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it
> > helpful
> > > > for
> > > > > > > > things.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~
> > Joestein
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue,
> > Feb
> > > > 24,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > 11:10
> > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> jay.kreps@gmail.com
> > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd
> > > talked
> > > > > > about
> > > > > > > > > > doing a
> > > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > generalizing
> > > > > > > that a
> > > > > > > > > > little
> > > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > everyone
> > > > > > > spending a
> > > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync
> up
> > > > once
> > > > > a
> > > > > > > > week.
> > > > > > > > > I
> > > > > > > > > > > > think
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> stuff,
> > > make
> > > > > > sure
> > > > > > > we
> > > > > > > > > > are on
> > > > > > > > > > > > > top
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> issues,
> > > > etc.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We
> can
> > > make
> > > > > it
> > > > > > > > > publicly
> > > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> likes.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any
> > > > interest
> > > > > in
> > > > > > > > doing
> > > > > > > > > > this?
> > > > > > > > > > > > > If
> > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On
> Tue,
> > > Feb
> > > > > 24,
> > > > > > > > 2015
> > > > > > > > > at
> > > > > > > > > > > > 3:57
> > > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi
> > all,
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> I've
> > > > > updated
> > > > > > > KIP
> > > > > > > > > > page,
> > > > > > > > > > > > > fixed
> > > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> some
> > > > > > > >
> > > > > > > ...
> > > > > > >
> > > > > > > [Message clipped]
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
Andrii,

A few points.

1. Create/Alter can typically complete quickly. So, it's possible to make
the request block until it's completed. However, currently, doing this at
the broker is a bit involved. To make Create block, we will need to add
some callbacks in KafkaController. This is possible. However, the
controller logic currently is pretty completed. It would probably be better
if we clean it up first before adding more complexity to it. Alter is even
trickier. Adding partition is currently handled through KafkaController. So
it can be dealt with in a similar way. However, Alter config is done
completely differently. It doesn't go through the controller. Instead, each
broker listens to ZooKeeper directly. So, it's not clear if there is an
easy way on the broker to figure out whether a config is applied on every
broker.

2. Delete can potentially take long if a replica to be deleted is offline.
PreferredLeader/PartitionReassign can also take long. So, we can't really
make those requests block on the broker.

As you can see, at this moment it's not easy to make all admin requests
block on the broker. So, if we want the blocking feature in the admin
utility in the short term, doing the completion check at the admin client
is probably an easier route, even though it may not be ideal.

Thanks,

Jun

On Fri, Mar 20, 2015 at 2:38 PM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Jun,
>
> I see your point. But wouldn't that lead to a "fat" client implementations?
> Suppose someone would like to implement client for Admin Wire protocol.
> Not only people will have to code quite complicated logic like "send
> describe
> request to each broker" (again state machin?) but it will also mean people
> must understand internal kafka logic related to topic storage and how
> information is propageted from the controller to brokers.
> I see this like a dilemma between having a concise Wire Protocol and
> self-sufficient API to make client implementations simple.
> I don't have a win-win solution though.
>
> Thanks,
> Andrii Biletskyi
>
>
> On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > For 1), 2) and 3), blocking would probably mean that the new metadata is
> > propagated to every broker. To achieve that, the client can keep issuing
> > the describe topic request to every broker until it sees the new metadata
> > in the response.
> >
> > Thanks,
> >
> > Jun
> >
> > On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Hm, actually the ticket you linked, Guozhang, brings as back
> > > to the problem what should be considered a post-condition for
> > > each of the admin commands.
> > > In my understanding:
> > >
> > > 1) CreateTopic - broker created /brokers/topics/<topic>
> > > (Not the controller picked up changes from zk and broadcasted
> > > LeaderAndIsr and UpdateMetadata)
> > >
> > > 2) AlterTopic - same as 1) - broker changed assignment data
> > > in zookeeper or created admin path for topic config change
> > >
> > > 3) DeleteTopic - admin path /admin/delete_topics is created
> > >
> > > 4) ReassignPartitions and PreferredReplica - corresponding admin
> > > path is created
> > >
> > > Now what can be considered a completed operation from the client's
> > > perspective?
> > > 1) Topic is created once corresponding data is in zk
> > > (I remember there were some thoughts that it'd be good to consider
> > > topic created once all replicas receive information about it and thus
> > > clients can produce/consume from it, but as was discussed this seems
> > > to be a hard thing to do)
> > >
> > > 2) Probably same as 1), so right after AlterTopic is issued
> > >
> > > 3) The topic has been removed from /brokers/topics
> > >
> > > 4) ReassignPartitions and PrefferedReplica were discussed earlier -
> > > in short the former is completed once partition state info in zk
> matches
> > > reassignment request and admin path is empty, the latter - once data
> > > in zk shows that head of assignned replicas of the partition and leader
> > > is the same replica
> > >
> > > Thoughts?
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > I think while loop is fine for supporting blocking, just that we need
> > to
> > > > add back off to avoid bombarding brokers with DescribeTopic requests.
> > > >
> > > > Also I have linked KAFKA-1125
> > > > <https://issues.apache.org/jira/browse/KAFKA-1125> to your proposal,
> > and
> > > > when KAFKA-1694 is done this ticket can also be closed.
> > > >
> > > > Guozhang
> > > >
> > > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Great.
> > > > > I want to elaborate this a bit more, to see we are on the same page
> > > > > concerning the client code.
> > > > >
> > > > > So with all topic commands being async a client (AdminClient in our
> > > > > case or any other other client people would like to implement) to
> > > support
> > > > > a blocking operation (which seems to be a natural use-case e.g. for
> > > topic
> > > > > creation): would have to do:
> > > > > 1. issue CreateTopicRequest
> > > > > 2. if successful, in a "while" loop send DescribeTopicRequest and
> > > > > break the loop once all topics are returned in response (or upon
> > > > timeout).
> > > > > 3. if unsuccessful throw exception
> > > > > Would it be okay?
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > >
> > > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Andrii,
> > > > > >
> > > > > > I think you are right. It seems that only ReassignPartitions
> needs
> > a
> > > > > > separate verification request.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Guys,
> > > > > > > I like this idea too. Let's stick with that. I'll update KIP
> > > > > accordingly.
> > > > > > >
> > > > > > > I was also thinking we can avoid adding dedicated status check
> > > > > > > requests for topic commands. - We have everything in
> > DescribeTopic
> > > > > > > for that! E.g.:
> > > > > > > User issued CreateTopic - to check the status client sends
> > > > > DescribeTopic
> > > > > > > and checks whether is something returned for that topic. The
> same
> > > for
> > > > > > > alteration, deletion.
> > > > > > > Btw, PreferredReplica status can be also checked with
> > > > > > DescribeTopicRequest
> > > > > > > (head of assigned replicas list == leader).
> > > > > > > For ReassignPartitions as discussed we'll need to have a
> separate
> > > > > > Verify...
> > > > > > > request.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> > wangguoz@gmail.com
> > > >
> > > > > > wrote:
> > > > > > >
> > > > > > > > +1 on broker writing to ZK for async handling. I was thinking
> > > that
> > > > in
> > > > > > the
> > > > > > > > end state the admin requests would be eventually sent to
> > > controller
> > > > > > > either
> > > > > > > > through re-routing or clients discovering them, instead of
> > > letting
> > > > > > > > controller listen on ZK admin path. But thinking about it a
> > > second
> > > > > > time,
> > > > > > > I
> > > > > > > > think it is actually simpler to let controller manage
> > > > > > > > incoming queued-up admin requests through ZK.
> > > > > > > >
> > > > > > > > Guozhang
> > > > > > > >
> > > > > > > >
> > > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> > jjkoshy.w@gmail.com
> > > >
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > +1 as well. I think it helps to keep the rerouting approach
> > > > > > orthogonal
> > > > > > > > > to this KIP.
> > > > > > > > >
> > > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps wrote:
> > > > > > > > > > I'm +1 on Jun's suggestion as long as it can work for all
> > the
> > > > > > > requests.
> > > > > > > > > >
> > > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <
> jun@confluent.io
> > >
> > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Andrii,
> > > > > > > > > > >
> > > > > > > > > > > I think we agreed on the following.
> > > > > > > > > > >
> > > > > > > > > > > (a) Admin requests can be sent to and handled by any
> > > broker.
> > > > > > > > > > > (b) Admin requests are processed asynchronously, at
> least
> > > for
> > > > > > now.
> > > > > > > > > That is,
> > > > > > > > > > > when the client gets a response, it just means that the
> > > > request
> > > > > > is
> > > > > > > > > > > initiated, but not necessarily completed. Then, it's up
> > to
> > > > the
> > > > > > > client
> > > > > > > > > to
> > > > > > > > > > > issue another request to check the status for
> completion.
> > > > > > > > > > >
> > > > > > > > > > > To support (a), we were thinking of doing request
> > > forwarding
> > > > to
> > > > > > the
> > > > > > > > > > > controller (utilizing KAFKA-1912). I am making an
> > > alternative
> > > > > > > > proposal.
> > > > > > > > > > > Basically, the broker can just write to ZooKeeper to
> > inform
> > > > the
> > > > > > > > > controller
> > > > > > > > > > > about the request. For example, to handle
> > > > > partitionReassignment,
> > > > > > > the
> > > > > > > > > broker
> > > > > > > > > > > will just write the requested partitions to
> > > > > > > > /admin/reassign_partitions
> > > > > > > > > > > (like what AdminUtils currently does) and then send a
> > > > response
> > > > > to
> > > > > > > the
> > > > > > > > > > > client. This shouldn't take long and the implementation
> > > will
> > > > be
> > > > > > > > simpler
> > > > > > > > > > > than forwarding the requests to the controller through
> > RPC.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Jun,
> > > > > > > > > > > >
> > > > > > > > > > > > I might be wrong but didn't we agree we will let any
> > > broker
> > > > > > from
> > > > > > > > the
> > > > > > > > > > > > cluster handle *long-running* admin requests (at this
> > > time
> > > > > > > > > > > preferredReplica
> > > > > > > > > > > > and
> > > > > > > > > > > > reassignPartitions), via zk admin path. Thus
> > CreateTopics
> > > > etc
> > > > > > > > should
> > > > > > > > > be
> > > > > > > > > > > > sent
> > > > > > > > > > > > only to the controller.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> > > > jun@confluent.io>
> > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > > >
> > > > > > > > > > > > > I think we agreed that those admin requests can be
> > > issued
> > > > > to
> > > > > > > any
> > > > > > > > > > > broker.
> > > > > > > > > > > > > Because of that, there doesn't seem to be a strong
> > need
> > > > to
> > > > > > know
> > > > > > > > the
> > > > > > > > > > > > > controller. So, perhaps we can proceed by not
> making
> > > any
> > > > > > change
> > > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > > > format of TMR right now. When we start using create
> > > topic
> > > > > > > request
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > producer, we will need a new version of TMR that
> > > doesn't
> > > > > > > trigger
> > > > > > > > > auto
> > > > > > > > > > > > topic
> > > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > > >
> > > > > > > > > > > > > As a first cut implementation, I think the broker
> can
> > > > just
> > > > > > > write
> > > > > > > > > to ZK
> > > > > > > > > > > > > directly for
> > > > > > > > > > > > >
> > > > > > > >
> > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > > requests, instead of forwarding them to the
> > controller.
> > > > > This
> > > > > > > will
> > > > > > > > > > > > simplify
> > > > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> > > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > For (1) yes we will circle back on that shortly
> > after
> > > > > > syncing
> > > > > > > > up
> > > > > > > > > in
> > > > > > > > > > > > > > person. I think it is close to getting committed
> > > > although
> > > > > > > > > development
> > > > > > > > > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > There is one more item we covered at the hangout.
> > > i.e.,
> > > > > > > whether
> > > > > > > > > we
> > > > > > > > > > > > > > want to add the coordinator to the topic metadata
> > > > > response
> > > > > > or
> > > > > > > > > provide
> > > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > There are two reasons I think we should try and
> > avoid
> > > > > > adding
> > > > > > > > the
> > > > > > > > > > > > > > field:
> > > > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > > > - If we finally do request rerouting in Kafka
> then
> > > the
> > > > > > field
> > > > > > > > > would
> > > > > > > > > > > add
> > > > > > > > > > > > > >   little to no value. (It still helps to have a
> > > > separate
> > > > > > > > > > > > > >   ClusterMetadataRequest to query for
> cluster-wide
> > > > > > > information
> > > > > > > > > such
> > > > > > > > > > > as
> > > > > > > > > > > > > >   'which broker is the controller?' as Joe
> > > mentioned.)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > I think it would be cleaner to have an explicit
> > > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > > that you can send to any broker in order to
> obtain
> > > the
> > > > > > > > controller
> > > > > > > > > > > (and
> > > > > > > > > > > > > > in the future possibly other cluster-wide
> > > > information). I
> > > > > > > think
> > > > > > > > > the
> > > > > > > > > > > > > > main argument against doing this and instead
> adding
> > > it
> > > > to
> > > > > > the
> > > > > > > > > topic
> > > > > > > > > > > > > > metadata response was convenience - i.e., you
> don't
> > > > have
> > > > > to
> > > > > > > > > discover
> > > > > > > > > > > > > > the controller in advance. However, I don't see
> > much
> > > > > actual
> > > > > > > > > > > > > > benefit/convenience in this and in fact think it
> > is a
> > > > > > > > non-issue.
> > > > > > > > > Let
> > > > > > > > > > > > > > me know if I'm overlooking something here.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > As an example, say we need to initiate partition
> > > > > > reassignment
> > > > > > > > by
> > > > > > > > > > > > > > issuing the new ReassignPartitionsRequest to the
> > > > > controller
> > > > > > > > > (assume
> > > > > > > > > > > we
> > > > > > > > > > > > > > already have the desired manual partition
> > > assignment).
> > > > > If
> > > > > > we
> > > > > > > > > are to
> > > > > > > > > > > > > > augment topic metadata response then the flow be
> > > > > something
> > > > > > > like
> > > > > > > > > this
> > > > > > > > > > > :
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > - Issue topic metadata request to any broker (and
> > > > > discover
> > > > > > > the
> > > > > > > > > > > > > >   controller
> > > > > > > > > > > > > > - Connect to controller if required (i.e., if the
> > > > broker
> > > > > > > above
> > > > > > > > !=
> > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > - Issue the partition reassignment request to the
> > > > > > controller.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > With an explicit cluster metadata request it
> would
> > > be:
> > > > > > > > > > > > > > - Issue cluster metadata request to any broker
> > > > > > > > > > > > > > - Connect to controller if required (i.e., if the
> > > > broker
> > > > > > > above
> > > > > > > > !=
> > > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > > - Issue the partition reassignment request
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > So it seems to add little practical value and
> > bloats
> > > > > topic
> > > > > > > > > metadata
> > > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > The other angle to this is the following - is it
> a
> > > > matter
> > > > > > of
> > > > > > > > > naming?
> > > > > > > > > > > > > > Should we just rename topic metadata
> > request/response
> > > > to
> > > > > > just
> > > > > > > > > > > > > > MetadataRequest/Response and add cluster metadata
> > to
> > > > it?
> > > > > By
> > > > > > > > that
> > > > > > > > > same
> > > > > > > > > > > > > > token should we also allow querying for the
> > consumer
> > > > > > > > coordinator
> > > > > > > > > (and
> > > > > > > > > > > > > > in future transaction coordinator) as well? This
> > > leads
> > > > > to a
> > > > > > > > > bloated
> > > > > > > > > > > > > > request which isn't very appealing and altogether
> > > > > > confusing.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Joel
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao
> > > > wrote:
> > > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1. I just realized that in order to start
> working
> > > on
> > > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > need to merge the changes to
> OffsetCommitRequest
> > > > (from
> > > > > > > 0.8.2)
> > > > > > > > > to
> > > > > > > > > > > > trunk.
> > > > > > > > > > > > > > > This is planned to be done as part of
> KAFKA-1634.
> > > So,
> > > > > we
> > > > > > > will
> > > > > > > > > need
> > > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 2. Thinking about this a bit more, if the
> > semantic
> > > of
> > > > > > those
> > > > > > > > > "write"
> > > > > > > > > > > > > > > requests is async (i.e., after the client gets
> a
> > > > > > response,
> > > > > > > it
> > > > > > > > > just
> > > > > > > > > > > > > means
> > > > > > > > > > > > > > > that the operation is initiated, but not
> > > necessarily
> > > > > > > > > completed), we
> > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > really need to forward the requests to the
> > > > controller.
> > > > > > > > > Instead, the
> > > > > > > > > > > > > > > receiving broker can just write the operation
> to
> > ZK
> > > > as
> > > > > > the
> > > > > > > > > admin
> > > > > > > > > > > > > command
> > > > > > > > > > > > > > > line tool previously does. This will simplify
> the
> > > > > > > > > implementation.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 8. There is another implementation detail for
> > > > describe
> > > > > > > topic.
> > > > > > > > > > > > Ideally,
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > want to read the topic config from the broker
> > > cache,
> > > > > > > instead
> > > > > > > > of
> > > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > > Currently, every broker reads the topic-level
> > > config
> > > > > for
> > > > > > > all
> > > > > > > > > > > topics.
> > > > > > > > > > > > > > > However, it ignores those for topics not hosted
> > on
> > > > > > itself.
> > > > > > > > So,
> > > > > > > > > we
> > > > > > > > > > > may
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > to change TopicConfigManager a bit so that it
> > > caches
> > > > > the
> > > > > > > > > configs
> > > > > > > > > > > for
> > > > > > > > > > > > > all
> > > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii
> > Biletskyi <
> > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 1. Q: Get rid of all scala requests objects,
> > use
> > > > java
> > > > > > > > > protocol
> > > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > > >     A: Gwen kindly took that (KAFKA-1927).
> It's
> > > > > > important
> > > > > > > > to
> > > > > > > > > > > speed
> > > > > > > > > > > > up
> > > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > > >          there since this ticket blocks other
> > > > > important
> > > > > > > > > changes.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 2. Q: Generic re-reroute facility vs client
> > > > > maintaining
> > > > > > > > > cluster
> > > > > > > > > > > > > state.
> > > > > > > > > > > > > > > >     A: Jay has added pseudo code to
> KAFKA-1912
> > -
> > > > need
> > > > > > to
> > > > > > > > > consider
> > > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > > >         easy to implement as a server-side
> > > feature
> > > > > > > > (comments
> > > > > > > > > are
> > > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > > > > > > > > >     A: This might be useful for clients, add
> > this
> > > > to
> > > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > > > > > > > > >     A: I will add proposed by Jun solution
> > (using
> > > > > > > clientId
> > > > > > > > > for
> > > > > > > > > > > > that)
> > > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs grabbing
> > all
> > > > > > > protocol
> > > > > > > > > > > changes
> > > > > > > > > > > > in
> > > > > > > > > > > > > > one
> > > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > > >     A: It was decided to try to gather all
> > > changes
> > > > to
> > > > > > > > > protocol
> > > > > > > > > > > > > (before
> > > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > > >         In case of TMR it worth checking:
> > > > KAFKA-2020
> > > > > > and
> > > > > > > > > KIP-13
> > > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 6. Q: JSON lib is needed to deserialize
> user's
> > > > input
> > > > > in
> > > > > > > CLI
> > > > > > > > > tool.
> > > > > > > > > > > > > > > >     A: Use jackson for that, /tools project
> is
> > a
> > > > > > separate
> > > > > > > > > jar so
> > > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs generic
> > status
> > > > > check
> > > > > > > > > command.
> > > > > > > > > > > > > > > >      A: For long-running requests like
> reassign
> > > > > > > partitions
> > > > > > > > > > > > *progress*
> > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > > >          it makes sense to introduce it.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >  Please add, correct me if I missed
> something.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii
> > > Biletskyi <
> > > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > You are right, I removed ClusterMetadata
> > > because
> > > > we
> > > > > > > have
> > > > > > > > > > > > partially
> > > > > > > > > > > > > > > > > what we need in TopicMetadata. Also, as Jay
> > > > pointed
> > > > > > out
> > > > > > > > > > > earlier,
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > would like to have "orthogonal" API, but at
> > the
> > > > > same
> > > > > > > time
> > > > > > > > > we
> > > > > > > > > > > need
> > > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > But I like your idea and even have some
> other
> > > > > > arguments
> > > > > > > > for
> > > > > > > > > > > this
> > > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > > There is also DescribeTopicRequest which
> was
> > > > > proposed
> > > > > > > in
> > > > > > > > > this
> > > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > > it returns topic configs, partitions,
> > > replication
> > > > > > > factor
> > > > > > > > > plus
> > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > > leader replica. The later part is really
> > > already
> > > > > > there
> > > > > > > in
> > > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > > So again we'll have to add stuff to TMR,
> not
> > to
> > > > > > > duplicate
> > > > > > > > > some
> > > > > > > > > > > > info
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > newly added requests. However, this way
> we'll
> > > end
> > > > > up
> > > > > > > with
> > > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > > request which returns cluster metadata,
> topic
> > > > > > > replication
> > > > > > > > > and
> > > > > > > > > > > > > config
> > > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > > plus partition replication data. Seems
> > logical
> > > to
> > > > > > split
> > > > > > > > > TMR to
> > > > > > > > > > > > > > > > > - ClusterMetadata (brokers + controller,
> > maybe
> > > > smth
> > > > > > > else)
> > > > > > > > > > > > > > > > > - TopicMetadata (topic info + partition
> > > details)
> > > > > > > > > > > > > > > > > But since current TMR is involved in lots
> of
> > > > places
> > > > > > > > > (including
> > > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > > as I understand) this might be very serious
> > > > change
> > > > > > and
> > > > > > > it
> > > > > > > > > > > > probably
> > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel
> Koshy <
> > > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >> I may be missing some context but
> hopefully
> > > this
> > > > > > will
> > > > > > > > > also be
> > > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > > >> today: I thought the earlier proposal
> where
> > > > there
> > > > > > was
> > > > > > > an
> > > > > > > > > > > > explicit
> > > > > > > > > > > > > > > > >> ClusterMetadata request was clearer and
> > > > explicit.
> > > > > > > During
> > > > > > > > > the
> > > > > > > > > > > > > course
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > >> this thread I think the conclusion was
> that
> > > the
> > > > > main
> > > > > > > > need
> > > > > > > > > was
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > >> controller information and that can be
> > rolled
> > > > into
> > > > > > the
> > > > > > > > > topic
> > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > >> response but that seems a bit irrelevant
> to
> > > > topic
> > > > > > > > > metadata.
> > > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > > >> think the full broker-list is also
> > irrelevant
> > > to
> > > > > > topic
> > > > > > > > > > > metadata,
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > >> it is already there and in use. I think
> > there
> > > is
> > > > > > still
> > > > > > > > > room
> > > > > > > > > > > for
> > > > > > > > > > > > an
> > > > > > > > > > > > > > > > >> explicit ClusterMetadata request since
> there
> > > may
> > > > > be
> > > > > > > > other
> > > > > > > > > > > > > > > > >> cluster-level information that we may want
> > to
> > > > add
> > > > > > over
> > > > > > > > > time
> > > > > > > > > > > (and
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > >> have nothing to do with topic metadata).
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200,
> > > Andrii
> > > > > > > > Biletskyi
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > 101. Okay, if you say that such use case
> > is
> > > > > > > > important. I
> > > > > > > > > > > also
> > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > >> > using clientId for these purposes is
> fine
> > -
> > > if
> > > > > we
> > > > > > > > > already
> > > > > > > > > > > have
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > > >> > as part of all Wire protocol messages,
> why
> > > not
> > > > > use
> > > > > > > > that.
> > > > > > > > > > > > > > > > >> > I will update KIP-4 page if nobody has
> > other
> > > > > ideas
> > > > > > > > > (which
> > > > > > > > > > > may
> > > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP
> > > accordingly.
> > > > I
> > > > > > > think
> > > > > > > > > we can
> > > > > > > > > > > > add
> > > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > > >> > fine-grained error codes if some error
> > code
> > > > > > received
> > > > > > > > in
> > > > > > > > > > > > specific
> > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > >> > won't give enough context to return a
> > > > > descriptive
> > > > > > > > error
> > > > > > > > > > > > message
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > Look forward to discussing all
> outstanding
> > > > > issues
> > > > > > in
> > > > > > > > > detail
> > > > > > > > > > > > > today
> > > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun
> Rao
> > <
> > > > > > > > > jun@confluent.io
> > > > > > > > > > > >
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > > 101. There may be a use case where you
> > > only
> > > > > want
> > > > > > > the
> > > > > > > > > > > topics
> > > > > > > > > > > > to
> > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > > >> > > manually by admins. Currently, you can
> > do
> > > > that
> > > > > > by
> > > > > > > > > > > disabling
> > > > > > > > > > > > > auto
> > > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > >> > > creation and issue topic creation from
> > the
> > > > > > > > > TopicCommand.
> > > > > > > > > > > If
> > > > > > > > > > > > we
> > > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > > >> > > topic creation completely on the
> broker
> > > and
> > > > > > don't
> > > > > > > > > have a
> > > > > > > > > > > way
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > >> > > between topic creation requests from
> the
> > > > > regular
> > > > > > > > > clients
> > > > > > > > > > > and
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > > >> > > can't support manual topic creation
> any
> > > > more.
> > > > > I
> > > > > > > was
> > > > > > > > > > > thinking
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > > >> > > way of distinguishing the clients
> making
> > > the
> > > > > > topic
> > > > > > > > > > > creation
> > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > >> > > using clientId. For example, the admin
> > > tool
> > > > > can
> > > > > > > set
> > > > > > > > > it to
> > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > > >> > > admin and the broker can treat that
> > > clientId
> > > > > > > > > specially.
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > Also, there is a related discussion in
> > > > > > KAFKA-2020.
> > > > > > > > > > > > Currently,
> > > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > 1. If leader is not available, we set
> > the
> > > > > > > partition
> > > > > > > > > level
> > > > > > > > > > > > > error
> > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > > >> > > 2. If a non-leader replica is not
> > > available,
> > > > > we
> > > > > > > take
> > > > > > > > > that
> > > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > >> > > the assigned replica list and isr in
> the
> > > > > > response.
> > > > > > > > As
> > > > > > > > > an
> > > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > > >> > > doing that, we set the partition level
> > > error
> > > > > > code
> > > > > > > to
> > > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > This has a few problems. First,
> > > > > > > ReplicaNotAvailable
> > > > > > > > > > > probably
> > > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > > >> > > an error, at least for the normal
> > > > > > > producer/consumer
> > > > > > > > > > > clients
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > > >> > > to find out the leader. Second, it can
> > > > happen
> > > > > > that
> > > > > > > > > both
> > > > > > > > > > > the
> > > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > >> > > another replica are not available at
> the
> > > > same
> > > > > > > time.
> > > > > > > > > There
> > > > > > > > > > > is
> > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > >> > > to indicate both. Third, even if a
> > replica
> > > > is
> > > > > > not
> > > > > > > > > > > available,
> > > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > > >> > > useful to return its replica id since
> > some
> > > > > > clients
> > > > > > > > > (e.g.
> > > > > > > > > > > > admin
> > > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > One way to address this issue is to
> > always
> > > > > > return
> > > > > > > > the
> > > > > > > > > > > > replica
> > > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > > >> > > leader, assigned replicas, and isr
> > > > regardless
> > > > > of
> > > > > > > > > whether
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > > >> > > broker is live or not. Since we also
> > > return
> > > > > the
> > > > > > > list
> > > > > > > > > of
> > > > > > > > > > > live
> > > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > >> > > client can figure out whether a leader
> > or
> > > a
> > > > > > > replica
> > > > > > > > is
> > > > > > > > > > > live
> > > > > > > > > > > > or
> > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > > >> > > accordingly. This way, we don't need
> to
> > > set
> > > > > the
> > > > > > > > > partition
> > > > > > > > > > > > > level
> > > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > > >> > > when the leader or a replica is not
> > > > available.
> > > > > > > This
> > > > > > > > > > > doesn't
> > > > > > > > > > > > > > change
> > > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > > >> > > protocol, but does change the
> semantics.
> > > > Since
> > > > > > we
> > > > > > > > are
> > > > > > > > > > > > evolving
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > > >> > > of TopicMetadataRequest here, we can
> > > > > potentially
> > > > > > > > > piggyback
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > 102.1 For those types of errors due to
> > > > invalid
> > > > > > > > input,
> > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > >> > > guard it at parameter validation time
> > and
> > > > > throw
> > > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > > >> > > without even sending the request to
> the
> > > > > broker?
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM,
> Andrii
> > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > > > 101. If I understand you correctly,
> > you
> > > > are
> > > > > > > saying
> > > > > > > > > > > future
> > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > > >> > > > will be ported to TMR_V1) won't be
> > able
> > > to
> > > > > > > > > automatically
> > > > > > > > > > > > > > create
> > > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > >> > > > unconditionally remove topic
> creation
> > > from
> > > > > > > there).
> > > > > > > > > But
> > > > > > > > > > > we
> > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > > >> > > > Ok, about your proposal: I'm not a
> big
> > > fan
> > > > > > too,
> > > > > > > > > when it
> > > > > > > > > > > > > comes
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > > >> > > > clients directly in protocol schema.
> > And
> > > > > also
> > > > > > > I'm
> > > > > > > > > not
> > > > > > > > > > > > sure I
> > > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > > >> > > > auto.create.topics.enable is a
> server
> > > side
> > > > > > > > > > > configuration.
> > > > > > > > > > > > > Can
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > > >> > > > in future versions, add this setting
> > to
> > > > > > producer
> > > > > > > > and
> > > > > > > > > > > based
> > > > > > > > > > > > > on
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > > >> > > > UnknownTopic create topic explicitly
> > by
> > > a
> > > > > > > separate
> > > > > > > > > > > > producer
> > > > > > > > > > > > > > call
> > > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because we want
> > to
> > > > > > support
> > > > > > > > > batching
> > > > > > > > > > > > and
> > > > > > > > > > > > > > at
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > > >> > > > want to give descriptive error
> > messages
> > > > for
> > > > > > > > clients.
> > > > > > > > > > > Since
> > > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > > >> > > > to construct such messages (e.g.
> > > > AdminClient
> > > > > > > layer
> > > > > > > > > can
> > > > > > > > > > > > know
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > > >> > > > means two cases: either invalid
> > number -
> > > > > e.g.
> > > > > > > -1;
> > > > > > > > or
> > > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > > >> > > > partitions argument wasn't) - I
> > wrapped
> > > > > > > responses
> > > > > > > > in
> > > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > > >> > > > other ideas, this was just initial
> > > > version.
> > > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change
> that
> > to
> > > > > > > probably
> > > > > > > > > some
> > > > > > > > > > > > other
> > > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun
> > > Rao <
> > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > 101. That's what I was thinking
> too,
> > > but
> > > > > it
> > > > > > > may
> > > > > > > > > not be
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > > >> > > > > we can let it not trigger auto
> topic
> > > > > > creation.
> > > > > > > > > Then,
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > > >> > > > > if it gets an
> UnknownTopicException,
> > > it
> > > > > can
> > > > > > > > > explicitly
> > > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > > >> > > > > createTopicRequest for auto topic
> > > > > creation.
> > > > > > On
> > > > > > > > the
> > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > > >> > > > > never issue createTopicRequest.
> This
> > > > works
> > > > > > > when
> > > > > > > > > auto
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > > >> > > > > enabled on the broker side.
> > However, I
> > > > am
> > > > > > not
> > > > > > > > > sure how
> > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > > >> > > > > when auto topic creation is
> disabled
> > > on
> > > > > the
> > > > > > > > broker
> > > > > > > > > > > side.
> > > > > > > > > > > > > In
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > >> > > > > want to have a way to manually
> > create
> > > a
> > > > > > topic,
> > > > > > > > > > > > potentially
> > > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > >> > > > > commands. However, then we need a
> > way
> > > to
> > > > > > > > > distinguish
> > > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > > >> > > > > issued from the producer clients
> and
> > > the
> > > > > > admin
> > > > > > > > > tools.
> > > > > > > > > > > > May
> > > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > >> > > > > new field in createTopicRequest
> and
> > > set
> > > > it
> > > > > > > > > differently
> > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > > >> > > > > client and the admin client.
> > However,
> > > I
> > > > am
> > > > > > not
> > > > > > > > > sure if
> > > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring existing
> > requests
> > > > is a
> > > > > > > > > non-trivial
> > > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > > >> > > > > posted some comments in
> KAFKA-1927.
> > We
> > > > > will
> > > > > > > > > probably
> > > > > > > > > > > > have
> > > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > > >> > > > > first, before adding the new logic
> > in
> > > > > > > > KAFKA-1694.
> > > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird that we
> > return
> > > > > > > exception
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > >> > > > > we should either return error code
> > or
> > > > > throw
> > > > > > an
> > > > > > > > > > > exception
> > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > > >> > > > > 102.2. We probably shouldn't
> > > explicitly
> > > > > use
> > > > > > > the
> > > > > > > > > > > request
> > > > > > > > > > > > > > object
> > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > > >> > > > > Not every request evolution
> requires
> > > an
> > > > > api
> > > > > > > > > change.
> > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM,
> > > Andrii
> > > > > > > > Biletskyi
> > > > > > > > > <
> > > > > > > > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly>
> wrote:
> > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > > Thanks for you comments. Answers
> > > > inline:
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > > 100. There are a few fields such
> > as
> > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized that
> > are
> > > > > > > > represented
> > > > > > > > > as a
> > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > >> > > > > > > composite structures in json.
> > > Could
> > > > we
> > > > > > > > flatten
> > > > > > > > > > > them
> > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > arrays/records?
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > > Yes, now with Admin Client this
> > > looks
> > > > a
> > > > > > bit
> > > > > > > > > weird.
> > > > > > > > > > > My
> > > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > > >> > > > > > ReassignPartitionCommand accepts
> > > input
> > > > > in
> > > > > > > > json,
> > > > > > > > > we
> > > > > > > > > > > > want
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > > >> > > > > > interfaces unchanged, where
> > > possible.
> > > > > > > > > > > > > > > > >> > > > > > If we port it to deserialized
> > > format,
> > > > in
> > > > > > CLI
> > > > > > > > > (/tools
> > > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > > >> > > > > > json library since /tools is
> > written
> > > > in
> > > > > > java
> > > > > > > > and
> > > > > > > > > > > we'll
> > > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > > >> > > > > > provided by a user. Can we
> quickly
> > > > agree
> > > > > > on
> > > > > > > > what
> > > > > > > > > > > this
> > > > > > > > > > > > > > library
> > > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > > 101. Does TopicMetadataRequest
> v1
> > > > still
> > > > > > > > trigger
> > > > > > > > > auto
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now that
> we
> > > > have a
> > > > > > > > > separate
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > > > > > > createTopicRequest
> > > > > > > > > > > and
> > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > producer/consumer
> > > > > > > > > client,
> > > > > > > > > > > in
> > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > >> > > > > > > tools? For example, ideally,
> we
> > > > don't
> > > > > > want
> > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto topic
> > > > > creation.
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > > I agree, this strange logic
> should
> > > be
> > > > > > fixed.
> > > > > > > > > I'm not
> > > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > > >> > > > > > correct me if I'm wrong, but it
> > > > doesn't
> > > > > > look
> > > > > > > > > like a
> > > > > > > > > > > > hard
> > > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > > >> > > > > > leverage AdminClient for that in
> > > > > Producer
> > > > > > > and
> > > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant getting rid
> > of
> > > > > scala
> > > > > > > > > classes
> > > > > > > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader
> > and
> > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> adding
> > > the
> > > > > new
> > > > > > > > > requests
> > > > > > > > > > > for
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > >> > > > > > > However, the long term plan is
> > to
> > > > get
> > > > > > rid
> > > > > > > of
> > > > > > > > > all
> > > > > > > > > > > > those
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > >> > > > > > > java request/response in the
> > > client.
> > > > > > Since
> > > > > > > > > this
> > > > > > > > > > > KIP
> > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > > requests,
> > > > > > > perhaps
> > > > > > > > we
> > > > > > > > > > > > should
> > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > >> > > > > > > clean up the existing scala
> > > requests
> > > > > > first
> > > > > > > > > before
> > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > > Yes, looks like I misunderstood
> > the
> > > > > point
> > > > > > of
> > > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > > >> > > > > > rework that. The only thing is
> > that
> > > I
> > > > > > don't
> > > > > > > > see
> > > > > > > > > any
> > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > > >> > > > > > least one existing protocol
> > message.
> > > > > Thus,
> > > > > > > as
> > > > > > > > I
> > > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > > > >> > > > > > Re porting all existing RQ/RP in
> > > this
> > > > > > patch.
> > > > > > > > > Sounds
> > > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > > >> > > > > > requirement to have Admin KIP
> > done,
> > > > I'm
> > > > > > > afraid
> > > > > > > > > this
> > > > > > > > > > > > can
> > > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > > >> > > > > > There are 13 protocol messages
> and
> > > all
> > > > > > that
> > > > > > > > > would
> > > > > > > > > > > > > require
> > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > > >> > > > > > intensive manual testing, no?
> I'm
> > > > afraid
> > > > > > I'm
> > > > > > > > > not the
> > > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > > >> > > > > > Kafka core internals :). Let me
> > know
> > > > > your
> > > > > > > > > thoughts
> > > > > > > > > > > on
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > > > > >> > >
> > > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM,
> > Jun
> > > > > Rao <
> > > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > 100. There are a few fields
> such
> > > as
> > > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized that
> > are
> > > > > > > > represented
> > > > > > > > > as a
> > > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > > >> > > > > > > composite structures in json.
> > > Could
> > > > we
> > > > > > > > flatten
> > > > > > > > > > > them
> > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > > arrays/records?
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > 101. Does TopicMetadataRequest
> > v1
> > > > > still
> > > > > > > > > trigger
> > > > > > > > > > > auto
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now that
> we
> > > > have a
> > > > > > > > > separate
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > > > > > > createTopicRequest
> > > > > > > > > > > and
> > > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > > producer/consumer
> > > > > > > > > client,
> > > > > > > > > > > in
> > > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > > >> > > > > > > tools? For example, ideally,
> we
> > > > don't
> > > > > > want
> > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto topic
> > > > > creation.
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant getting
> rid
> > > of
> > > > > > scala
> > > > > > > > > classes
> > > > > > > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader
> > and
> > > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when
> adding
> > > the
> > > > > new
> > > > > > > > > requests
> > > > > > > > > > > for
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > > >> > > > > > > However, the long term plan is
> > to
> > > > get
> > > > > > rid
> > > > > > > of
> > > > > > > > > all
> > > > > > > > > > > > those
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > >> > > > > > > java request/response in the
> > > client.
> > > > > > Since
> > > > > > > > > this
> > > > > > > > > > > KIP
> > > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > > requests,
> > > > > > > perhaps
> > > > > > > > we
> > > > > > > > > > > > should
> > > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > >> > > > > > > clean up the existing scala
> > > requests
> > > > > > first
> > > > > > > > > before
> > > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37
> PM,
> > > > > Andrii
> > > > > > > > > Biletskyi
> > > > > > > > > > > <
> > > > > > > > > > > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > As said above - I list again
> > all
> > > > > > > comments
> > > > > > > > > from
> > > > > > > > > > > > this
> > > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > > >> > > > > > > > can see what's left and
> > finalize
> > > > all
> > > > > > > > pending
> > > > > > > > > > > > issues.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > > > > >> > > > > > > > 1. This is much needed
> > > > > functionality,
> > > > > > > but
> > > > > > > > > there
> > > > > > > > > > > > are
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > > >> > > > > > > > really think these protocols
> > > > > through.
> > > > > > We
> > > > > > > > > really
> > > > > > > > > > > > want
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > > >> > > > > > > > of well thought-out,
> > orthoganol
> > > > > apis.
> > > > > > > For
> > > > > > > > > this
> > > > > > > > > > > > > reason
> > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > > >> > > > > > > > important to think through
> the
> > > end
> > > > > > state
> > > > > > > > > even if
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > >> > > > > > > > won't implement in the first
> > > > phase.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: Definitely behind this.
> > Would
> > > > > > > > appreciate
> > > > > > > > > if
> > > > > > > > > > > > there
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 2. Let's please please
> please
> > > wait
> > > > > > until
> > > > > > > > we
> > > > > > > > > have
> > > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > > >> > > > > > > > to the new java protocol
> > > > > definitions.
> > > > > > If
> > > > > > > > we
> > > > > > > > > add
> > > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > > >> > > > > > > > objects that is just
> > generating
> > > > more
> > > > > > > work
> > > > > > > > > for
> > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> patch -
> > > > > removed
> > > > > > > > scala
> > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 3. This proposal introduces
> a
> > > new
> > > > > type
> > > > > > > of
> > > > > > > > > > > optional
> > > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > > >> > > > > > > > inconsistent with everything
> > > else
> > > > in
> > > > > > the
> > > > > > > > > > > protocol
> > > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > > >> > > > > > > > other marker value. You
> could
> > > > argue
> > > > > > > either
> > > > > > > > > way
> > > > > > > > > > > but
> > > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > >> > > > > > > > for consistency. For clients
> > > that
> > > > > > > > > implemented
> > > > > > > > > > > the
> > > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > > >> > > > > > > > than our scala code these
> > basic
> > > > > > > primitives
> > > > > > > > > are
> > > > > > > > > > > > hard
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> patch -
> > > > > removed
> > > > > > > > > MaybeOf
> > > > > > > > > > > > type
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This
> seems
> > > to
> > > > > > > > duplicate
> > > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > >> > > > > > > > brokers, topics, and
> > > partitions. I
> > > > > > think
> > > > > > > > we
> > > > > > > > > > > should
> > > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > > >> > > > > > > > ClusterMetadataRequest (or
> > just
> > > > > > > > > MetadataRequest)
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > >> > > > > > > > controller. Or are there
> other
> > > > > things
> > > > > > we
> > > > > > > > > could
> > > > > > > > > > > add
> > > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: I agree. Updated the KIP.
> > > Let's
> > > > > > > extends
> > > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 5. We have a tendency to try
> > to
> > > > > make a
> > > > > > > lot
> > > > > > > > > of
> > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > > >> > > > > > > > particular nodes. This adds
> a
> > > lot
> > > > of
> > > > > > > > burden
> > > > > > > > > for
> > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > > >> > > > > > > > sounds easy but each
> discovery
> > > can
> > > > > > fail
> > > > > > > in
> > > > > > > > > many
> > > > > > > > > > > > > parts
> > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > > >> > > > > > > > full state machine to do
> > > right). I
> > > > > > think
> > > > > > > > we
> > > > > > > > > > > should
> > > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > > >> > > > > > > > commands and ideally as many
> > of
> > > > the
> > > > > > > other
> > > > > > > > > apis
> > > > > > > > > > > as
> > > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > > >> > > > > > > > brokers and just redirect to
> > the
> > > > > > > > controller
> > > > > > > > > on
> > > > > > > > > > > the
> > > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > > >> > > > > > > > there would be a general way
> > to
> > > > > > > > encapsulate
> > > > > > > > > this
> > > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: It's a very interesting
> > idea,
> > > > but
> > > > > > > seems
> > > > > > > > > there
> > > > > > > > > > > > are
> > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > > >> > > > > > > > feature (like performance
> > > > > > > considerations,
> > > > > > > > > how
> > > > > > > > > > > this
> > > > > > > > > > > > > > will
> > > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > > >> > > > > > > > I believe this shouldn't be
> a
> > > > > blocker.
> > > > > > > If
> > > > > > > > > this
> > > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > > >> > > > > > > > point it won't affect Admin
> > > > changes
> > > > > -
> > > > > > at
> > > > > > > > > least
> > > > > > > > > > > no
> > > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 6. We should probably
> > normalize
> > > > the
> > > > > > key
> > > > > > > > > value
> > > > > > > > > > > > pairs
> > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > >> > > > > > > > than embedding a new
> > formatting.
> > > > So
> > > > > > two
> > > > > > > > > strings
> > > > > > > > > > > > > rather
> > > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> patch -
> > > > > > > normalized
> > > > > > > > > > > configs
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 7. Is the postcondition of
> > these
> > > > > APIs
> > > > > > > that
> > > > > > > > > the
> > > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > > >> > > > > > > > the command has been
> > completed?
> > > It
> > > > > is
> > > > > > a
> > > > > > > > lot
> > > > > > > > > more
> > > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > > >> > > > > > > > been completed so you know
> > that
> > > if
> > > > > you
> > > > > > > > > create a
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > >> > > > > > > > it you won't get an
> exception
> > > > about
> > > > > > > there
> > > > > > > > > being
> > > > > > > > > > > no
> > > > > > > > > > > > > > such
> > > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: For long running requests
> > > (like
> > > > > > > > reassign
> > > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > > >> > > > > > > > command has begun - so we
> > don't
> > > > > block
> > > > > > > the
> > > > > > > > > > > client.
> > > > > > > > > > > > In
> > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > > >> > > > > > > > topic commands, this will be
> > > > > > refactored
> > > > > > > > and
> > > > > > > > > > > topic
> > > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > > >> > > > > > > > immediately, since the
> > > Controller
> > > > > will
> > > > > > > > serve
> > > > > > > > > > > Admin
> > > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > > >> > > > > > > > (follow-up ticket
> KAFKA-1777).
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 8. Describe topic and list
> > > topics
> > > > > > > > duplicate
> > > > > > > > > a
> > > > > > > > > > > lot
> > > > > > > > > > > > of
> > > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > > >> > > > > > > > request. Is there a reason
> to
> > > give
> > > > > > back
> > > > > > > > > topics
> > > > > > > > > > > > > marked
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > > >> > > > > > > > like if we just make the
> > > > > > post-condition
> > > > > > > of
> > > > > > > > > the
> > > > > > > > > > > > > delete
> > > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > >> > > > > > > > topic is deleted that will
> get
> > > rid
> > > > > of
> > > > > > > the
> > > > > > > > > need
> > > > > > > > > > > for
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest
> patch -
> > > > > removed
> > > > > > > > > topics
> > > > > > > > > > > > marked
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 9. Should we consider
> batching
> > > > these
> > > > > > > > > requests?
> > > > > > > > > > > We
> > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > >> > > > > > > > allow multiple operations to
> > be
> > > > > > batched.
> > > > > > > > My
> > > > > > > > > > > > > suspicion
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > >> > > > > > > > we will get a lot of code
> that
> > > > does
> > > > > > > > > something
> > > > > > > > > > > like
> > > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > > >> > > > > > > > this code will work great
> when
> > > you
> > > > > > test
> > > > > > > > on 5
> > > > > > > > > > > > topics
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please
> > > check
> > > > > > "Topic
> > > > > > > > > Admin
> > > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 10. I think we should also
> > > discuss
> > > > > how
> > > > > > > we
> > > > > > > > > want
> > > > > > > > > > > to
> > > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > > >> > > > > > > > client api for these
> > operations.
> > > > > > > Currently
> > > > > > > > > > > people
> > > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > > >> > > > > > > > is totally sketchy. I think
> we
> > > > > > probably
> > > > > > > > need
> > > > > > > > > > > > another
> > > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > > >> > > > > > > > that exposes administrative
> > > > > > > functionality.
> > > > > > > > > We
> > > > > > > > > > > will
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > >> > > > > > > > properly test the new apis,
> I
> > > > > suspect.
> > > > > > > We
> > > > > > > > > should
> > > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please
> > > check
> > > > > > "Admin
> > > > > > > > > Client"
> > > > > > > > > > > > > > section
> > > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 11. The other information
> that
> > > > would
> > > > > > be
> > > > > > > > > really
> > > > > > > > > > > > > useful
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > >> > > > > > > > information about
> > > partitions--how
> > > > > much
> > > > > > > > data
> > > > > > > > > is
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > > >> > > > > > > > the segment offsets, what is
> > the
> > > > > > log-end
> > > > > > > > > offset
> > > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > > last
> > > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > > >> > > > > > > > the compaction point, etc. I
> > > think
> > > > > > that
> > > > > > > > done
> > > > > > > > > > > right
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > >> > > > > > > > successor to the very
> awkward
> > > > > > > > OffsetRequest
> > > > > > > > > we
> > > > > > > > > > > > have
> > > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > > ConsumerGroupOffsetsRequest
> > > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > > latest
> > > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > > >> > > > > > > > be resolved in a separate
> KIP
> > /
> > > > jira
> > > > > > > > ticket.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 12. Generally we can do good
> > > error
> > > > > > > > handling
> > > > > > > > > > > > without
> > > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > > >> > > > > > > > messages. I.e. generally the
> > > > client
> > > > > > has
> > > > > > > > the
> > > > > > > > > > > > context
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > > >> > > > > > > > an error that the topic
> > doesn't
> > > > > exist
> > > > > > to
> > > > > > > > say
> > > > > > > > > > > > "Topic
> > > > > > > > > > > > > X
> > > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > > >> > > > > > > > than "error code 14" (or
> > > > whatever).
> > > > > > > Maybe
> > > > > > > > > there
> > > > > > > > > > > > are
> > > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > > >> > > > > > > > this is hard? If we want to
> > add
> > > > > > > > server-side
> > > > > > > > > > > error
> > > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > > >> > > > > > > > need to do this in a
> > consistent
> > > > way
> > > > > > > across
> > > > > > > > > the
> > > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please
> > > check
> > > > > > > > "Protocol
> > > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > >> > > > > > > > comprehensive, fine-grained
> > list
> > > > of
> > > > > > > error
> > > > > > > > > codes.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > > > > > > > > >> > > > > > > > 13. Describe topic request:
> it
> > > > would
> > > > > > be
> > > > > > > > > great to
> > > > > > > > > > > > go
> > > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > >> > > > > > > > topic name regex for this
> > > request.
> > > > > For
> > > > > > > > > example,
> > > > > > > > > > > a
> > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > > >> > > > > > > > the topic command is to list
> > all
> > > > > > topics
> > > > > > > > > whose
> > > > > > > > > > > > config
> > > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > > >> > > > > > > > topic name regex then we
> have
> > to
> > > > > first
> > > > > > > > > retrieve
> > > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > > >> > > > > > > > description info and then
> > filter
> > > > at
> > > > > > the
> > > > > > > > > client
> > > > > > > > > > > > end,
> > > > > > > > > > > > > > which
> > > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in create
> > topic:
> > > > > this
> > > > > > is
> > > > > > > > > related
> > > > > > > > > > > > to
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > > >> > > > > > > > maybe we can add another
> > > metadata
> > > > > K-V
> > > > > > or
> > > > > > > > > just a
> > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > > >> > > > > > > > with config K-V in create
> > topic
> > > > like
> > > > > > we
> > > > > > > > did
> > > > > > > > > for
> > > > > > > > > > > > > offset
> > > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > > >> > > > > > > > field can be quite useful in
> > > > storing
> > > > > > > > > information
> > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > > >> > > > > > > > who issue the create
> command,
> > > etc,
> > > > > > which
> > > > > > > > is
> > > > > > > > > > > quite
> > > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > > >> > > > > > > > multi-tenant setting. Then
> in
> > > the
> > > > > > > describe
> > > > > > > > > topic
> > > > > > > > > > > > > > request
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > > >> > > > > > > > on regex of the metadata
> > field.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: As discussed it is very
> > > > > interesting
> > > > > > > but
> > > > > > > > > can
> > > > > > > > > > > be
> > > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > > >> > > > > > > > we have some basic
> > functionality
> > > > > > there.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > 15. Today all the admin
> > > operations
> > > > > are
> > > > > > > > > async in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > > >> > > > > > > > return once it is written in
> > ZK,
> > > > and
> > > > > > > that
> > > > > > > > > is why
> > > > > > > > > > > > we
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > > >> > > > > > > > like
> > > > testUtil.waitForTopicCreated()
> > > > > /
> > > > > > > > verify
> > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > > >> > > > > > > > request, etc. With admin
> > > requests
> > > > we
> > > > > > > could
> > > > > > > > > add a
> > > > > > > > > > > > > flag
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > > >> > > > > > > > synchronous requests; when
> it
> > is
> > > > > > turned
> > > > > > > > on,
> > > > > > > > > the
> > > > > > > > > > > > > > response
> > > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > > >> > > > > > > > until the request has been
> > > > > completed.
> > > > > > > And
> > > > > > > > > for
> > > > > > > > > > > > async
> > > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > > >> > > > > > > > "token" field in the
> response,
> > > and
> > > > > > then
> > > > > > > > only
> > > > > > > > > > > need
> > > > > > > > > > > > a
> > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > > >> > > > > > > > verification request" with
> the
> > > > given
> > > > > > > token
> > > > > > > > > to
> > > > > > > > > > > > check
> > > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: I see your point. My idea
> > was
> > > > to
> > > > > > > > provide
> > > > > > > > > > > > specific
> > > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > > >> > > > > > > > long running request, where
> > > > needed.
> > > > > We
> > > > > > > can
> > > > > > > > > do it
> > > > > > > > > > > > the
> > > > > > > > > > > > > > way
> > > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > > >> > > > > > > > concern is that introducing
> a
> > > > token
> > > > > we
> > > > > > > > again
> > > > > > > > > > > will
> > > > > > > > > > > > > make
> > > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > > >> > > > > > > > to do similar thing
> > introducing
> > > > > single
> > > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > > >> > > > > > > > this idea because we wanted
> to
> > > > have
> > > > > > > schema
> > > > > > > > > > > > defined.
> > > > > > > > > > > > > So
> > > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > > > >> > > > > > > > a) have fixed schema but
> > > introduce
> > > > > > each
> > > > > > > > > time new
> > > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > > > > > > > >> > > > > > > > b) use one request for
> > > > verification
> > > > > > but
> > > > > > > > > > > generalize
> > > > > > > > > > > > > it
> > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > > >> > > > > > > > I'm fine with whatever
> > decision
> > > > > > > community
> > > > > > > > > come
> > > > > > > > > > > to.
> > > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > > > > >> > > > > > > > 16. Specifically for
> > ownership,
> > > I
> > > > > > think
> > > > > > > > the
> > > > > > > > > plan
> > > > > > > > > > > > is
> > > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > > >> > > > > > > > like you are describing ACL)
> > via
> > > > an
> > > > > > > > external
> > > > > > > > > > > > system
> > > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > > >> > > > > > > > I remember KIP-11 described
> > > this,
> > > > > but
> > > > > > I
> > > > > > > > > can't
> > > > > > > > > > > find
> > > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no problem. Not
> sure
> > > > though
> > > > > > how
> > > > > > > > we
> > > > > > > > > are
> > > > > > > > > > > > > going
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > > >> > > > > > > > will be committed first and
> > > > include
> > > > > > > > changes
> > > > > > > > > to
> > > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > > >> > > > > > > > Anyway, I added this note to
> > > "Open
> > > > > > > > > Questions"
> > > > > > > > > > > > > section
> > > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at
> 12:34
> > > AM,
> > > > > > Andrii
> > > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > > >> > > > > > > > andrii.biletskyi@stealth.ly
> >
> > > > wrote:
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > > Today I uploaded the patch
> > > that
> > > > > > covers
> > > > > > > > > some of
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf optional
> > > type
> > > > > > > > > > > > > > > > >> > > > > > > > > - switched to java
> protocol
> > > > > > > definitions
> > > > > > > > > > > > > > > > >> > > > > > > > > - simplified messages
> > > > (normalized
> > > > > > > > configs,
> > > > > > > > > > > > removed
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > > I also updated the KIP-4
> > with
> > > > > > > respective
> > > > > > > > > > > changes
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin Operations
> ->
> > > > > updated
> > > > > > > Wire
> > > > > > > > > > > > Protocol
> > > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > > >> > > > > > > > > - Remove ClusterMetadata
> ->
> > > > > changed
> > > > > > to
> > > > > > > > > extend
> > > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > > >> > > > > > > > > - Admin Client -> updated
> my
> > > > > initial
> > > > > > > > > proposal
> > > > > > > > > > > to
> > > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > > >> > > > > > > > > - Error codes -> proposed
> > > > > > fine-grained
> > > > > > > > > error
> > > > > > > > > > > > code
> > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > > I will also send a
> separate
> > > > email
> > > > > to
> > > > > > > > > cover all
> > > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at
> 9:26
> > > PM,
> > > > > > Gwen
> > > > > > > > > Shapira
> > > > > > > > > > > <
> > > > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > > >> > > > > > > > >> It actually specifies
> > changes
> > > > to
> > > > > > the
> > > > > > > > > Metadata
> > > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are consistent
> in
> > > > this
> > > > > > > regard
> > > > > > > > > will
> > > > > > > > > > > be
> > > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at
> > 12:21
> > > > PM,
> > > > > > > Gwen
> > > > > > > > > > > Shapira
> > > > > > > > > > > > <
> > > > > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> > Specifically for
> > > ownership, I
> > > > > > think
> > > > > > > > the
> > > > > > > > > > > plan
> > > > > > > > > > > > is
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > > >> > > > > > > > >> > like you are describing
> > > ACL)
> > > > > via
> > > > > > an
> > > > > > > > > > > external
> > > > > > > > > > > > > > system
> > > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > > >> > > > > > > > >> > I remember KIP-11
> > described
> > > > > this,
> > > > > > > > but I
> > > > > > > > > > > can't
> > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I think
> KIP-4
> > > > > focuses
> > > > > > > on
> > > > > > > > > > > getting
> > > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > > >> > > > > > > > >> > exists from Kafka
> > brokers,
> > > > not
> > > > > on
> > > > > > > > > adding
> > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > > >> > > > > > > > >> > should exist but
> doesn't
> > > yet?
> > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at
> > > 6:37
> > > > > AM,
> > > > > > > > > Guozhang
> > > > > > > > > > > > Wang
> > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> Just want to
> elaborate a
> > > bit
> > > > > > more
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> describe-topic based
> on
> > > > > config /
> > > > > > > > > metadata
> > > > > > > > > > > in
> > > > > > > > > > > > > my
> > > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The
> main
> > > > > > motivation
> > > > > > > > is
> > > > > > > > > to
> > > > > > > > > > > > have
> > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms, which I
> > think
> > > is
> > > > > > quite
> > > > > > > > > > > important
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> architecture: today
> > anyone
> > > > can
> > > > > > > > create
> > > > > > > > > > > topics
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> there is no concept or
> > > > > > "ownership"
> > > > > > > > of
> > > > > > > > > > > topics
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> users. For example, at
> > > > > LinkedIn
> > > > > > we
> > > > > > > > > > > basically
> > > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> some casual topic name
> > > > prefix,
> > > > > > > which
> > > > > > > > > is a
> > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale our
> customers.
> > It
> > > > > would
> > > > > > > be
> > > > > > > > > great
> > > > > > > > > > > to
> > > > > > > > > > > > > use
> > > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics
> that
> > > is
> > > > > > > created
> > > > > > > > > by me.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics
> > whose
> > > > > > > retention
> > > > > > > > > time
> > > > > > > > > > > is
> > > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics
> > whose
> > > > > > writable
> > > > > > > > > group
> > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> One possible way to
> > > achieve
> > > > > this
> > > > > > > is
> > > > > > > > to
> > > > > > > > > > > add a
> > > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> create-topic request,
> > > whose
> > > > > > value
> > > > > > > > will
> > > > > > > > > > > also
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> topic; then
> > > describe-topics
> > > > > can
> > > > > > > > > choose to
> > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2) config K-V
> > > > matching,
> > > > > > 3)
> > > > > > > > > metadata
> > > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at
> > > 4:37
> > > > > PM,
> > > > > > > > > Guozhang
> > > > > > > > > > > > Wang
> > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for the
> updated
> > > > wiki.
> > > > > A
> > > > > > > few
> > > > > > > > > > > comments
> > > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error description
> in
> > > > > > > response: I
> > > > > > > > > think
> > > > > > > > > > > > if
> > > > > > > > > > > > > > some
> > > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> several different
> error
> > > > cases
> > > > > > > then
> > > > > > > > we
> > > > > > > > > > > > should
> > > > > > > > > > > > > > really
> > > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In general the
> > > > > errorCode
> > > > > > > > > itself
> > > > > > > > > > > > would
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> describing the server
> > > side
> > > > > > > errors.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic
> > > request:
> > > > it
> > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > > great
> > > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex for
> > this
> > > > > > > request.
> > > > > > > > > For
> > > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic command is
> to
> > > > list
> > > > > > all
> > > > > > > > > topics
> > > > > > > > > > > > whose
> > > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex then
> > we
> > > > have
> > > > > > to
> > > > > > > > > first
> > > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> description info and
> > then
> > > > > > filter
> > > > > > > at
> > > > > > > > > the
> > > > > > > > > > > > > client
> > > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in
> > create
> > > > > topic:
> > > > > > > > this
> > > > > > > > > is
> > > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can add
> > another
> > > > > > metadata
> > > > > > > > > K-V or
> > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> with config K-V in
> > create
> > > > > topic
> > > > > > > > like
> > > > > > > > > we
> > > > > > > > > > > did
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> field can be quite
> > useful
> > > > in
> > > > > > > > storing
> > > > > > > > > > > > > > information
> > > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue the create
> > > > command,
> > > > > > > etc,
> > > > > > > > > which
> > > > > > > > > > > is
> > > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant setting.
> > > Then
> > > > in
> > > > > > the
> > > > > > > > > > > describe
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of the
> > metadata
> > > > > field.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all the
> admin
> > > > > > operations
> > > > > > > > are
> > > > > > > > > > > async
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> return once it is
> > written
> > > > in
> > > > > > ZK,
> > > > > > > > and
> > > > > > > > > that
> > > > > > > > > > > > is
> > > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > > verify
> > > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> request, etc. With
> > admin
> > > > > > requests
> > > > > > > > we
> > > > > > > > > > > could
> > > > > > > > > > > > > add
> > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous requests;
> > > when
> > > > it
> > > > > > is
> > > > > > > > > turned
> > > > > > > > > > > on,
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> until the request has
> > > been
> > > > > > > > > completed. And
> > > > > > > > > > > > for
> > > > > > > > > > > > > > async
> > > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token" field in the
> > > > > response,
> > > > > > > and
> > > > > > > > > then
> > > > > > > > > > > > only
> > > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> verification request"
> > > with
> > > > > the
> > > > > > > > given
> > > > > > > > > > > token
> > > > > > > > > > > > to
> > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for extending
> > > > Metadata
> > > > > > > > request
> > > > > > > > > to
> > > > > > > > > > > > > include
> > > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> information, and then
> > we
> > > > can
> > > > > > > remove
> > > > > > > > > the
> > > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015
> at
> > > > 10:23
> > > > > > AM,
> > > > > > > > Joel
> > > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for sending
> > that
> > > > out
> > > > > > Joe
> > > > > > > -
> > > > > > > > I
> > > > > > > > > > > don't
> > > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so if
> notes
> > > can
> > > > be
> > > > > > > sent
> > > > > > > > > out
> > > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015
> > at
> > > > > > > 09:16:13AM
> > > > > > > > > > > -0800,
> > > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for sending
> > > this
> > > > > out
> > > > > > > Joe.
> > > > > > > > > > > Looking
> > > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2,
> 2015
> > at
> > > > > 6:46
> > > > > > > AM,
> > > > > > > > > Joe
> > > > > > > > > > > > Stein
> > > > > > > > > > > > > <
> > > > > > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just sent
> > > out a
> > > > > > > google
> > > > > > > > > > > hangout
> > > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I found
> > > > working
> > > > > > on
> > > > > > > a
> > > > > > > > > KIP.
> > > > > > > > > > > If
> > > > > > > > > > > > I
> > > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me know and
> > can
> > > > > update
> > > > > > > it,
> > > > > > > > > np.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We should do
> this
> > > > every
> > > > > > > > Tuesday
> > > > > > > > > @
> > > > > > > > > > > 2pm
> > > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to make a
> > > google
> > > > > > > account
> > > > > > > > > so we
> > > > > > > > > > > > can
> > > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in progress and
> > > > related
> > > > > > JIRA
> > > > > > > > > that
> > > > > > > > > > > are
> > > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24,
> > 2015
> > > > at
> > > > > > 2:59
> > > > > > > > > PM, Jay
> > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on
> > > Google
> > > > > > > hangouts
> > > > > > > > > that
> > > > > > > > > > > > will
> > > > > > > > > > > > > > also
> > > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> available on
> > > youtube.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24,
> > > 2015
> > > > at
> > > > > > > 11:49
> > > > > > > > > AM,
> > > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> jholoman@cloudera.com
> > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're happy
> to
> > > send
> > > > > > out a
> > > > > > > > > Webex
> > > > > > > > > > > for
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions if
> > there
> > > > is
> > > > > > > > > interest and
> > > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb
> 24,
> > > > 2015
> > > > > at
> > > > > > > > > 11:28 AM,
> > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try
> to
> > > get
> > > > > the
> > > > > > > > > technical
> > > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there is
> some
> > > > > benefit
> > > > > > > to
> > > > > > > > > live
> > > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post
> > > > > instructions
> > > > > > > and
> > > > > > > > > give
> > > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at
> > that
> > > > > time
> > > > > > > > would
> > > > > > > > > work
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb
> > 24,
> > > > > 2015
> > > > > > at
> > > > > > > > > 8:18
> > > > > > > > > > > AM,
> > > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly
> > would
> > > be
> > > > > > great
> > > > > > > > > maybe
> > > > > > > > > > > > like
> > > > > > > > > > > > > > every
> > > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't
> > mind
> > > > > google
> > > > > > > > > hangout
> > > > > > > > > > > but
> > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we know
> the
> > > > > apache
> > > > > > > irc
> > > > > > > > > > > channel
> > > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes? We
> > can
> > > > pull
> > > > > > > > > transcripts
> > > > > > > > > > > > too
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it
> helpful
> > > for
> > > > > > > things.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~
> Joestein
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue,
> Feb
> > > 24,
> > > > > > 2015
> > > > > > > at
> > > > > > > > > 11:10
> > > > > > > > > > > > AM,
> > > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com
> >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd
> > talked
> > > > > about
> > > > > > > > > doing a
> > > > > > > > > > > > > Google
> > > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > generalizing
> > > > > > that a
> > > > > > > > > little
> > > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> everyone
> > > > > > spending a
> > > > > > > > > > > > reasonable
> > > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync up
> > > once
> > > > a
> > > > > > > week.
> > > > > > > > I
> > > > > > > > > > > think
> > > > > > > > > > > > we
> > > > > > > > > > > > > > could
> > > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff,
> > make
> > > > > sure
> > > > > > we
> > > > > > > > > are on
> > > > > > > > > > > > top
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > issues,
> > > etc.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We can
> > make
> > > > it
> > > > > > > > publicly
> > > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any
> > > interest
> > > > in
> > > > > > > doing
> > > > > > > > > this?
> > > > > > > > > > > > If
> > > > > > > > > > > > > so
> > > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue,
> > Feb
> > > > 24,
> > > > > > > 2015
> > > > > > > > at
> > > > > > > > > > > 3:57
> > > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi
> all,
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > I've
> > > > updated
> > > > > > KIP
> > > > > > > > > page,
> > > > > > > > > > > > fixed
> > > > > > > > > > > > > /
> > > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > > > > >
> > > > > > ...
> > > > > >
> > > > > > [Message clipped]
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Jun,

I see your point. But wouldn't that lead to a "fat" client implementations?
Suppose someone would like to implement client for Admin Wire protocol.
Not only people will have to code quite complicated logic like "send
describe
request to each broker" (again state machin?) but it will also mean people
must understand internal kafka logic related to topic storage and how
information is propageted from the controller to brokers.
I see this like a dilemma between having a concise Wire Protocol and
self-sufficient API to make client implementations simple.
I don't have a win-win solution though.

Thanks,
Andrii Biletskyi


On Fri, Mar 20, 2015 at 11:19 PM, Jun Rao <ju...@confluent.io> wrote:

> For 1), 2) and 3), blocking would probably mean that the new metadata is
> propagated to every broker. To achieve that, the client can keep issuing
> the describe topic request to every broker until it sees the new metadata
> in the response.
>
> Thanks,
>
> Jun
>
> On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Hm, actually the ticket you linked, Guozhang, brings as back
> > to the problem what should be considered a post-condition for
> > each of the admin commands.
> > In my understanding:
> >
> > 1) CreateTopic - broker created /brokers/topics/<topic>
> > (Not the controller picked up changes from zk and broadcasted
> > LeaderAndIsr and UpdateMetadata)
> >
> > 2) AlterTopic - same as 1) - broker changed assignment data
> > in zookeeper or created admin path for topic config change
> >
> > 3) DeleteTopic - admin path /admin/delete_topics is created
> >
> > 4) ReassignPartitions and PreferredReplica - corresponding admin
> > path is created
> >
> > Now what can be considered a completed operation from the client's
> > perspective?
> > 1) Topic is created once corresponding data is in zk
> > (I remember there were some thoughts that it'd be good to consider
> > topic created once all replicas receive information about it and thus
> > clients can produce/consume from it, but as was discussed this seems
> > to be a hard thing to do)
> >
> > 2) Probably same as 1), so right after AlterTopic is issued
> >
> > 3) The topic has been removed from /brokers/topics
> >
> > 4) ReassignPartitions and PrefferedReplica were discussed earlier -
> > in short the former is completed once partition state info in zk matches
> > reassignment request and admin path is empty, the latter - once data
> > in zk shows that head of assignned replicas of the partition and leader
> > is the same replica
> >
> > Thoughts?
> >
> > Thanks,
> > Andrii Biletskyi
> >
> > On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > I think while loop is fine for supporting blocking, just that we need
> to
> > > add back off to avoid bombarding brokers with DescribeTopic requests.
> > >
> > > Also I have linked KAFKA-1125
> > > <https://issues.apache.org/jira/browse/KAFKA-1125> to your proposal,
> and
> > > when KAFKA-1694 is done this ticket can also be closed.
> > >
> > > Guozhang
> > >
> > > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Great.
> > > > I want to elaborate this a bit more, to see we are on the same page
> > > > concerning the client code.
> > > >
> > > > So with all topic commands being async a client (AdminClient in our
> > > > case or any other other client people would like to implement) to
> > support
> > > > a blocking operation (which seems to be a natural use-case e.g. for
> > topic
> > > > creation): would have to do:
> > > > 1. issue CreateTopicRequest
> > > > 2. if successful, in a "while" loop send DescribeTopicRequest and
> > > > break the loop once all topics are returned in response (or upon
> > > timeout).
> > > > 3. if unsuccessful throw exception
> > > > Would it be okay?
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > >
> > > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Andrii,
> > > > >
> > > > > I think you are right. It seems that only ReassignPartitions needs
> a
> > > > > separate verification request.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Guys,
> > > > > > I like this idea too. Let's stick with that. I'll update KIP
> > > > accordingly.
> > > > > >
> > > > > > I was also thinking we can avoid adding dedicated status check
> > > > > > requests for topic commands. - We have everything in
> DescribeTopic
> > > > > > for that! E.g.:
> > > > > > User issued CreateTopic - to check the status client sends
> > > > DescribeTopic
> > > > > > and checks whether is something returned for that topic. The same
> > for
> > > > > > alteration, deletion.
> > > > > > Btw, PreferredReplica status can be also checked with
> > > > > DescribeTopicRequest
> > > > > > (head of assigned replicas list == leader).
> > > > > > For ReassignPartitions as discussed we'll need to have a separate
> > > > > Verify...
> > > > > > request.
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <
> wangguoz@gmail.com
> > >
> > > > > wrote:
> > > > > >
> > > > > > > +1 on broker writing to ZK for async handling. I was thinking
> > that
> > > in
> > > > > the
> > > > > > > end state the admin requests would be eventually sent to
> > controller
> > > > > > either
> > > > > > > through re-routing or clients discovering them, instead of
> > letting
> > > > > > > controller listen on ZK admin path. But thinking about it a
> > second
> > > > > time,
> > > > > > I
> > > > > > > think it is actually simpler to let controller manage
> > > > > > > incoming queued-up admin requests through ZK.
> > > > > > >
> > > > > > > Guozhang
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <
> jjkoshy.w@gmail.com
> > >
> > > > > wrote:
> > > > > > >
> > > > > > > > +1 as well. I think it helps to keep the rerouting approach
> > > > > orthogonal
> > > > > > > > to this KIP.
> > > > > > > >
> > > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps wrote:
> > > > > > > > > I'm +1 on Jun's suggestion as long as it can work for all
> the
> > > > > > requests.
> > > > > > > > >
> > > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > > > > >
> > > > > > > > > > Andrii,
> > > > > > > > > >
> > > > > > > > > > I think we agreed on the following.
> > > > > > > > > >
> > > > > > > > > > (a) Admin requests can be sent to and handled by any
> > broker.
> > > > > > > > > > (b) Admin requests are processed asynchronously, at least
> > for
> > > > > now.
> > > > > > > > That is,
> > > > > > > > > > when the client gets a response, it just means that the
> > > request
> > > > > is
> > > > > > > > > > initiated, but not necessarily completed. Then, it's up
> to
> > > the
> > > > > > client
> > > > > > > > to
> > > > > > > > > > issue another request to check the status for completion.
> > > > > > > > > >
> > > > > > > > > > To support (a), we were thinking of doing request
> > forwarding
> > > to
> > > > > the
> > > > > > > > > > controller (utilizing KAFKA-1912). I am making an
> > alternative
> > > > > > > proposal.
> > > > > > > > > > Basically, the broker can just write to ZooKeeper to
> inform
> > > the
> > > > > > > > controller
> > > > > > > > > > about the request. For example, to handle
> > > > partitionReassignment,
> > > > > > the
> > > > > > > > broker
> > > > > > > > > > will just write the requested partitions to
> > > > > > > /admin/reassign_partitions
> > > > > > > > > > (like what AdminUtils currently does) and then send a
> > > response
> > > > to
> > > > > > the
> > > > > > > > > > client. This shouldn't take long and the implementation
> > will
> > > be
> > > > > > > simpler
> > > > > > > > > > than forwarding the requests to the controller through
> RPC.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > >
> > > > > > > > > > > Jun,
> > > > > > > > > > >
> > > > > > > > > > > I might be wrong but didn't we agree we will let any
> > broker
> > > > > from
> > > > > > > the
> > > > > > > > > > > cluster handle *long-running* admin requests (at this
> > time
> > > > > > > > > > preferredReplica
> > > > > > > > > > > and
> > > > > > > > > > > reassignPartitions), via zk admin path. Thus
> CreateTopics
> > > etc
> > > > > > > should
> > > > > > > > be
> > > > > > > > > > > sent
> > > > > > > > > > > only to the controller.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> > > jun@confluent.io>
> > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Joel, Andril,
> > > > > > > > > > > >
> > > > > > > > > > > > I think we agreed that those admin requests can be
> > issued
> > > > to
> > > > > > any
> > > > > > > > > > broker.
> > > > > > > > > > > > Because of that, there doesn't seem to be a strong
> need
> > > to
> > > > > know
> > > > > > > the
> > > > > > > > > > > > controller. So, perhaps we can proceed by not making
> > any
> > > > > change
> > > > > > > to
> > > > > > > > the
> > > > > > > > > > > > format of TMR right now. When we start using create
> > topic
> > > > > > request
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > producer, we will need a new version of TMR that
> > doesn't
> > > > > > trigger
> > > > > > > > auto
> > > > > > > > > > > topic
> > > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > > >
> > > > > > > > > > > > As a first cut implementation, I think the broker can
> > > just
> > > > > > write
> > > > > > > > to ZK
> > > > > > > > > > > > directly for
> > > > > > > > > > > >
> > > > > > >
> > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > > requests, instead of forwarding them to the
> controller.
> > > > This
> > > > > > will
> > > > > > > > > > > simplify
> > > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> > > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > > >
> > > > > > > > > > > > > For (1) yes we will circle back on that shortly
> after
> > > > > syncing
> > > > > > > up
> > > > > > > > in
> > > > > > > > > > > > > person. I think it is close to getting committed
> > > although
> > > > > > > > development
> > > > > > > > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > > > > > > > >
> > > > > > > > > > > > > There is one more item we covered at the hangout.
> > i.e.,
> > > > > > whether
> > > > > > > > we
> > > > > > > > > > > > > want to add the coordinator to the topic metadata
> > > > response
> > > > > or
> > > > > > > > provide
> > > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > > >
> > > > > > > > > > > > > There are two reasons I think we should try and
> avoid
> > > > > adding
> > > > > > > the
> > > > > > > > > > > > > field:
> > > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > > - If we finally do request rerouting in Kafka then
> > the
> > > > > field
> > > > > > > > would
> > > > > > > > > > add
> > > > > > > > > > > > >   little to no value. (It still helps to have a
> > > separate
> > > > > > > > > > > > >   ClusterMetadataRequest to query for cluster-wide
> > > > > > information
> > > > > > > > such
> > > > > > > > > > as
> > > > > > > > > > > > >   'which broker is the controller?' as Joe
> > mentioned.)
> > > > > > > > > > > > >
> > > > > > > > > > > > > I think it would be cleaner to have an explicit
> > > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > > that you can send to any broker in order to obtain
> > the
> > > > > > > controller
> > > > > > > > > > (and
> > > > > > > > > > > > > in the future possibly other cluster-wide
> > > information). I
> > > > > > think
> > > > > > > > the
> > > > > > > > > > > > > main argument against doing this and instead adding
> > it
> > > to
> > > > > the
> > > > > > > > topic
> > > > > > > > > > > > > metadata response was convenience - i.e., you don't
> > > have
> > > > to
> > > > > > > > discover
> > > > > > > > > > > > > the controller in advance. However, I don't see
> much
> > > > actual
> > > > > > > > > > > > > benefit/convenience in this and in fact think it
> is a
> > > > > > > non-issue.
> > > > > > > > Let
> > > > > > > > > > > > > me know if I'm overlooking something here.
> > > > > > > > > > > > >
> > > > > > > > > > > > > As an example, say we need to initiate partition
> > > > > reassignment
> > > > > > > by
> > > > > > > > > > > > > issuing the new ReassignPartitionsRequest to the
> > > > controller
> > > > > > > > (assume
> > > > > > > > > > we
> > > > > > > > > > > > > already have the desired manual partition
> > assignment).
> > > > If
> > > > > we
> > > > > > > > are to
> > > > > > > > > > > > > augment topic metadata response then the flow be
> > > > something
> > > > > > like
> > > > > > > > this
> > > > > > > > > > :
> > > > > > > > > > > > >
> > > > > > > > > > > > > - Issue topic metadata request to any broker (and
> > > > discover
> > > > > > the
> > > > > > > > > > > > >   controller
> > > > > > > > > > > > > - Connect to controller if required (i.e., if the
> > > broker
> > > > > > above
> > > > > > > !=
> > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > - Issue the partition reassignment request to the
> > > > > controller.
> > > > > > > > > > > > >
> > > > > > > > > > > > > With an explicit cluster metadata request it would
> > be:
> > > > > > > > > > > > > - Issue cluster metadata request to any broker
> > > > > > > > > > > > > - Connect to controller if required (i.e., if the
> > > broker
> > > > > > above
> > > > > > > !=
> > > > > > > > > > > > >   controller)
> > > > > > > > > > > > > - Issue the partition reassignment request
> > > > > > > > > > > > >
> > > > > > > > > > > > > So it seems to add little practical value and
> bloats
> > > > topic
> > > > > > > > metadata
> > > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > > >
> > > > > > > > > > > > > The other angle to this is the following - is it a
> > > matter
> > > > > of
> > > > > > > > naming?
> > > > > > > > > > > > > Should we just rename topic metadata
> request/response
> > > to
> > > > > just
> > > > > > > > > > > > > MetadataRequest/Response and add cluster metadata
> to
> > > it?
> > > > By
> > > > > > > that
> > > > > > > > same
> > > > > > > > > > > > > token should we also allow querying for the
> consumer
> > > > > > > coordinator
> > > > > > > > (and
> > > > > > > > > > > > > in future transaction coordinator) as well? This
> > leads
> > > > to a
> > > > > > > > bloated
> > > > > > > > > > > > > request which isn't very appealing and altogether
> > > > > confusing.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Joel
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao
> > > wrote:
> > > > > > > > > > > > > > Andri,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1. I just realized that in order to start working
> > on
> > > > > > > > KAFKA-1927, we
> > > > > > > > > > > > will
> > > > > > > > > > > > > > need to merge the changes to OffsetCommitRequest
> > > (from
> > > > > > 0.8.2)
> > > > > > > > to
> > > > > > > > > > > trunk.
> > > > > > > > > > > > > > This is planned to be done as part of KAFKA-1634.
> > So,
> > > > we
> > > > > > will
> > > > > > > > need
> > > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 2. Thinking about this a bit more, if the
> semantic
> > of
> > > > > those
> > > > > > > > "write"
> > > > > > > > > > > > > > requests is async (i.e., after the client gets a
> > > > > response,
> > > > > > it
> > > > > > > > just
> > > > > > > > > > > > means
> > > > > > > > > > > > > > that the operation is initiated, but not
> > necessarily
> > > > > > > > completed), we
> > > > > > > > > > > > don't
> > > > > > > > > > > > > > really need to forward the requests to the
> > > controller.
> > > > > > > > Instead, the
> > > > > > > > > > > > > > receiving broker can just write the operation to
> ZK
> > > as
> > > > > the
> > > > > > > > admin
> > > > > > > > > > > > command
> > > > > > > > > > > > > > line tool previously does. This will simplify the
> > > > > > > > implementation.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 8. There is another implementation detail for
> > > describe
> > > > > > topic.
> > > > > > > > > > > Ideally,
> > > > > > > > > > > > we
> > > > > > > > > > > > > > want to read the topic config from the broker
> > cache,
> > > > > > instead
> > > > > > > of
> > > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > > Currently, every broker reads the topic-level
> > config
> > > > for
> > > > > > all
> > > > > > > > > > topics.
> > > > > > > > > > > > > > However, it ignores those for topics not hosted
> on
> > > > > itself.
> > > > > > > So,
> > > > > > > > we
> > > > > > > > > > may
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > to change TopicConfigManager a bit so that it
> > caches
> > > > the
> > > > > > > > configs
> > > > > > > > > > for
> > > > > > > > > > > > all
> > > > > > > > > > > > > > topics.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Jun
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii
> Biletskyi <
> > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 1. Q: Get rid of all scala requests objects,
> use
> > > java
> > > > > > > > protocol
> > > > > > > > > > > > > definitions.
> > > > > > > > > > > > > > >     A: Gwen kindly took that (KAFKA-1927). It's
> > > > > important
> > > > > > > to
> > > > > > > > > > speed
> > > > > > > > > > > up
> > > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > > >          there since this ticket blocks other
> > > > important
> > > > > > > > changes.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 2. Q: Generic re-reroute facility vs client
> > > > maintaining
> > > > > > > > cluster
> > > > > > > > > > > > state.
> > > > > > > > > > > > > > >     A: Jay has added pseudo code to KAFKA-1912
> -
> > > need
> > > > > to
> > > > > > > > consider
> > > > > > > > > > > > > whether
> > > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > > >         easy to implement as a server-side
> > feature
> > > > > > > (comments
> > > > > > > > are
> > > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > > > > > > > >     A: This might be useful for clients, add
> this
> > > to
> > > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > > > > > > > >     A: I will add proposed by Jun solution
> (using
> > > > > > clientId
> > > > > > > > for
> > > > > > > > > > > that)
> > > > > > > > > > > > > to the
> > > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs grabbing
> all
> > > > > > protocol
> > > > > > > > > > changes
> > > > > > > > > > > in
> > > > > > > > > > > > > one
> > > > > > > > > > > > > > > version.
> > > > > > > > > > > > > > >     A: It was decided to try to gather all
> > changes
> > > to
> > > > > > > > protocol
> > > > > > > > > > > > (before
> > > > > > > > > > > > > > > release).
> > > > > > > > > > > > > > >         In case of TMR it worth checking:
> > > KAFKA-2020
> > > > > and
> > > > > > > > KIP-13
> > > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 6. Q: JSON lib is needed to deserialize user's
> > > input
> > > > in
> > > > > > CLI
> > > > > > > > tool.
> > > > > > > > > > > > > > >     A: Use jackson for that, /tools project is
> a
> > > > > separate
> > > > > > > > jar so
> > > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs generic
> status
> > > > check
> > > > > > > > command.
> > > > > > > > > > > > > > >      A: For long-running requests like reassign
> > > > > > partitions
> > > > > > > > > > > *progress*
> > > > > > > > > > > > > check
> > > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > > >          it makes sense to introduce it.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >  Please add, correct me if I missed something.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii
> > Biletskyi <
> > > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > You are right, I removed ClusterMetadata
> > because
> > > we
> > > > > > have
> > > > > > > > > > > partially
> > > > > > > > > > > > > > > > what we need in TopicMetadata. Also, as Jay
> > > pointed
> > > > > out
> > > > > > > > > > earlier,
> > > > > > > > > > > we
> > > > > > > > > > > > > > > > would like to have "orthogonal" API, but at
> the
> > > > same
> > > > > > time
> > > > > > > > we
> > > > > > > > > > need
> > > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > But I like your idea and even have some other
> > > > > arguments
> > > > > > > for
> > > > > > > > > > this
> > > > > > > > > > > > > option:
> > > > > > > > > > > > > > > > There is also DescribeTopicRequest which was
> > > > proposed
> > > > > > in
> > > > > > > > this
> > > > > > > > > > > KIP,
> > > > > > > > > > > > > > > > it returns topic configs, partitions,
> > replication
> > > > > > factor
> > > > > > > > plus
> > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > > leader replica. The later part is really
> > already
> > > > > there
> > > > > > in
> > > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > > So again we'll have to add stuff to TMR, not
> to
> > > > > > duplicate
> > > > > > > > some
> > > > > > > > > > > info
> > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > newly added requests. However, this way we'll
> > end
> > > > up
> > > > > > with
> > > > > > > > > > > "monster"
> > > > > > > > > > > > > > > > request which returns cluster metadata, topic
> > > > > > replication
> > > > > > > > and
> > > > > > > > > > > > config
> > > > > > > > > > > > > info
> > > > > > > > > > > > > > > > plus partition replication data. Seems
> logical
> > to
> > > > > split
> > > > > > > > TMR to
> > > > > > > > > > > > > > > > - ClusterMetadata (brokers + controller,
> maybe
> > > smth
> > > > > > else)
> > > > > > > > > > > > > > > > - TopicMetadata (topic info + partition
> > details)
> > > > > > > > > > > > > > > > But since current TMR is involved in lots of
> > > places
> > > > > > > > (including
> > > > > > > > > > > > > network
> > > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > > as I understand) this might be very serious
> > > change
> > > > > and
> > > > > > it
> > > > > > > > > > > probably
> > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <
> > > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >> I may be missing some context but hopefully
> > this
> > > > > will
> > > > > > > > also be
> > > > > > > > > > > > > covered
> > > > > > > > > > > > > > > >> today: I thought the earlier proposal where
> > > there
> > > > > was
> > > > > > an
> > > > > > > > > > > explicit
> > > > > > > > > > > > > > > >> ClusterMetadata request was clearer and
> > > explicit.
> > > > > > During
> > > > > > > > the
> > > > > > > > > > > > course
> > > > > > > > > > > > > of
> > > > > > > > > > > > > > > >> this thread I think the conclusion was that
> > the
> > > > main
> > > > > > > need
> > > > > > > > was
> > > > > > > > > > > for
> > > > > > > > > > > > > > > >> controller information and that can be
> rolled
> > > into
> > > > > the
> > > > > > > > topic
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > >> response but that seems a bit irrelevant to
> > > topic
> > > > > > > > metadata.
> > > > > > > > > > > FWIW I
> > > > > > > > > > > > > > > >> think the full broker-list is also
> irrelevant
> > to
> > > > > topic
> > > > > > > > > > metadata,
> > > > > > > > > > > > but
> > > > > > > > > > > > > > > >> it is already there and in use. I think
> there
> > is
> > > > > still
> > > > > > > > room
> > > > > > > > > > for
> > > > > > > > > > > an
> > > > > > > > > > > > > > > >> explicit ClusterMetadata request since there
> > may
> > > > be
> > > > > > > other
> > > > > > > > > > > > > > > >> cluster-level information that we may want
> to
> > > add
> > > > > over
> > > > > > > > time
> > > > > > > > > > (and
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > >> have nothing to do with topic metadata).
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200,
> > Andrii
> > > > > > > Biletskyi
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >> > 101. Okay, if you say that such use case
> is
> > > > > > > important. I
> > > > > > > > > > also
> > > > > > > > > > > > > think
> > > > > > > > > > > > > > > >> > using clientId for these purposes is fine
> -
> > if
> > > > we
> > > > > > > > already
> > > > > > > > > > have
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > > >> > as part of all Wire protocol messages, why
> > not
> > > > use
> > > > > > > that.
> > > > > > > > > > > > > > > >> > I will update KIP-4 page if nobody has
> other
> > > > ideas
> > > > > > > > (which
> > > > > > > > > > may
> > > > > > > > > > > > > come up
> > > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP
> > accordingly.
> > > I
> > > > > > think
> > > > > > > > we can
> > > > > > > > > > > add
> > > > > > > > > > > > > new,
> > > > > > > > > > > > > > > >> > fine-grained error codes if some error
> code
> > > > > received
> > > > > > > in
> > > > > > > > > > > specific
> > > > > > > > > > > > > case
> > > > > > > > > > > > > > > >> > won't give enough context to return a
> > > > descriptive
> > > > > > > error
> > > > > > > > > > > message
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >> > Look forward to discussing all outstanding
> > > > issues
> > > > > in
> > > > > > > > detail
> > > > > > > > > > > > today
> > > > > > > > > > > > > > > during
> > > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao
> <
> > > > > > > > jun@confluent.io
> > > > > > > > > > >
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > > >> > > 101. There may be a use case where you
> > only
> > > > want
> > > > > > the
> > > > > > > > > > topics
> > > > > > > > > > > to
> > > > > > > > > > > > > be
> > > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > > >> > > manually by admins. Currently, you can
> do
> > > that
> > > > > by
> > > > > > > > > > disabling
> > > > > > > > > > > > auto
> > > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > >> > > creation and issue topic creation from
> the
> > > > > > > > TopicCommand.
> > > > > > > > > > If
> > > > > > > > > > > we
> > > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > > >> > > topic creation completely on the broker
> > and
> > > > > don't
> > > > > > > > have a
> > > > > > > > > > way
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > >> > > between topic creation requests from the
> > > > regular
> > > > > > > > clients
> > > > > > > > > > and
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > > >> > > can't support manual topic creation any
> > > more.
> > > > I
> > > > > > was
> > > > > > > > > > thinking
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > > >> > > way of distinguishing the clients making
> > the
> > > > > topic
> > > > > > > > > > creation
> > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > >> > > using clientId. For example, the admin
> > tool
> > > > can
> > > > > > set
> > > > > > > > it to
> > > > > > > > > > > > > something
> > > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > > >> > > admin and the broker can treat that
> > clientId
> > > > > > > > specially.
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > Also, there is a related discussion in
> > > > > KAFKA-2020.
> > > > > > > > > > > Currently,
> > > > > > > > > > > > > we do
> > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > 1. If leader is not available, we set
> the
> > > > > > partition
> > > > > > > > level
> > > > > > > > > > > > error
> > > > > > > > > > > > > code
> > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > > >> > > 2. If a non-leader replica is not
> > available,
> > > > we
> > > > > > take
> > > > > > > > that
> > > > > > > > > > > > > replica
> > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > >> > > the assigned replica list and isr in the
> > > > > response.
> > > > > > > As
> > > > > > > > an
> > > > > > > > > > > > > indication
> > > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > > >> > > doing that, we set the partition level
> > error
> > > > > code
> > > > > > to
> > > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > This has a few problems. First,
> > > > > > ReplicaNotAvailable
> > > > > > > > > > probably
> > > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > > >> > > an error, at least for the normal
> > > > > > producer/consumer
> > > > > > > > > > clients
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > just
> > > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > > >> > > to find out the leader. Second, it can
> > > happen
> > > > > that
> > > > > > > > both
> > > > > > > > > > the
> > > > > > > > > > > > > leader
> > > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > >> > > another replica are not available at the
> > > same
> > > > > > time.
> > > > > > > > There
> > > > > > > > > > is
> > > > > > > > > > > > no
> > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > >> > > to indicate both. Third, even if a
> replica
> > > is
> > > > > not
> > > > > > > > > > available,
> > > > > > > > > > > > > it's
> > > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > > >> > > useful to return its replica id since
> some
> > > > > clients
> > > > > > > > (e.g.
> > > > > > > > > > > admin
> > > > > > > > > > > > > tool)
> > > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > One way to address this issue is to
> always
> > > > > return
> > > > > > > the
> > > > > > > > > > > replica
> > > > > > > > > > > > > id for
> > > > > > > > > > > > > > > >> > > leader, assigned replicas, and isr
> > > regardless
> > > > of
> > > > > > > > whether
> > > > > > > > > > the
> > > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > > >> > > broker is live or not. Since we also
> > return
> > > > the
> > > > > > list
> > > > > > > > of
> > > > > > > > > > live
> > > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > >> > > client can figure out whether a leader
> or
> > a
> > > > > > replica
> > > > > > > is
> > > > > > > > > > live
> > > > > > > > > > > or
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > > >> > > accordingly. This way, we don't need to
> > set
> > > > the
> > > > > > > > partition
> > > > > > > > > > > > level
> > > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > > >> > > when the leader or a replica is not
> > > available.
> > > > > > This
> > > > > > > > > > doesn't
> > > > > > > > > > > > > change
> > > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > > >> > > protocol, but does change the semantics.
> > > Since
> > > > > we
> > > > > > > are
> > > > > > > > > > > evolving
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > > >> > > of TopicMetadataRequest here, we can
> > > > potentially
> > > > > > > > piggyback
> > > > > > > > > > > the
> > > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > 102.1 For those types of errors due to
> > > invalid
> > > > > > > input,
> > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > >> > > guard it at parameter validation time
> and
> > > > throw
> > > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > > >> > > without even sending the request to the
> > > > broker?
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii
> > > > > > Biletskyi <
> > > > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > > > 101. If I understand you correctly,
> you
> > > are
> > > > > > saying
> > > > > > > > > > future
> > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > > >> > > > will be ported to TMR_V1) won't be
> able
> > to
> > > > > > > > automatically
> > > > > > > > > > > > > create
> > > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > >> > > > unconditionally remove topic creation
> > from
> > > > > > there).
> > > > > > > > But
> > > > > > > > > > we
> > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > > >> > > > Ok, about your proposal: I'm not a big
> > fan
> > > > > too,
> > > > > > > > when it
> > > > > > > > > > > > comes
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > > >> > > > clients directly in protocol schema.
> And
> > > > also
> > > > > > I'm
> > > > > > > > not
> > > > > > > > > > > sure I
> > > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > > >> > > > auto.create.topics.enable is a server
> > side
> > > > > > > > > > configuration.
> > > > > > > > > > > > Can
> > > > > > > > > > > > > we
> > > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > > >> > > > in future versions, add this setting
> to
> > > > > producer
> > > > > > > and
> > > > > > > > > > based
> > > > > > > > > > > > on
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > > >> > > > UnknownTopic create topic explicitly
> by
> > a
> > > > > > separate
> > > > > > > > > > > producer
> > > > > > > > > > > > > call
> > > > > > > > > > > > > > > via
> > > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because we want
> to
> > > > > support
> > > > > > > > batching
> > > > > > > > > > > and
> > > > > > > > > > > > > at
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > > >> > > > want to give descriptive error
> messages
> > > for
> > > > > > > clients.
> > > > > > > > > > Since
> > > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > > >> > > > to construct such messages (e.g.
> > > AdminClient
> > > > > > layer
> > > > > > > > can
> > > > > > > > > > > know
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > > >> > > > means two cases: either invalid
> number -
> > > > e.g.
> > > > > > -1;
> > > > > > > or
> > > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > > >> > > > partitions argument wasn't) - I
> wrapped
> > > > > > responses
> > > > > > > in
> > > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > > >> > > > other ideas, this was just initial
> > > version.
> > > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change that
> to
> > > > > > probably
> > > > > > > > some
> > > > > > > > > > > other
> > > > > > > > > > > > > dto.
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun
> > Rao <
> > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > >> > > > > 101. That's what I was thinking too,
> > but
> > > > it
> > > > > > may
> > > > > > > > not be
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > > >> > > > > we can let it not trigger auto topic
> > > > > creation.
> > > > > > > > Then,
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > > >> > > > > if it gets an UnknownTopicException,
> > it
> > > > can
> > > > > > > > explicitly
> > > > > > > > > > > > > issue a
> > > > > > > > > > > > > > > >> > > > > createTopicRequest for auto topic
> > > > creation.
> > > > > On
> > > > > > > the
> > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > > >> > > > > never issue createTopicRequest. This
> > > works
> > > > > > when
> > > > > > > > auto
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > > >> > > > > enabled on the broker side.
> However, I
> > > am
> > > > > not
> > > > > > > > sure how
> > > > > > > > > > > > > things
> > > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > > >> > > > > when auto topic creation is disabled
> > on
> > > > the
> > > > > > > broker
> > > > > > > > > > side.
> > > > > > > > > > > > In
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > >> > > > > want to have a way to manually
> create
> > a
> > > > > topic,
> > > > > > > > > > > potentially
> > > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > >> > > > > commands. However, then we need a
> way
> > to
> > > > > > > > distinguish
> > > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > > >> > > > > issued from the producer clients and
> > the
> > > > > admin
> > > > > > > > tools.
> > > > > > > > > > > May
> > > > > > > > > > > > > be we
> > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > >> > > > > new field in createTopicRequest and
> > set
> > > it
> > > > > > > > differently
> > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > > >> > > > > client and the admin client.
> However,
> > I
> > > am
> > > > > not
> > > > > > > > sure if
> > > > > > > > > > > > > that's
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring existing
> requests
> > > is a
> > > > > > > > non-trivial
> > > > > > > > > > > > > amount of
> > > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > > >> > > > > posted some comments in KAFKA-1927.
> We
> > > > will
> > > > > > > > probably
> > > > > > > > > > > have
> > > > > > > > > > > > > to fix
> > > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > > >> > > > > first, before adding the new logic
> in
> > > > > > > KAFKA-1694.
> > > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird that we
> return
> > > > > > exception
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > api. It
> > > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > >> > > > > we should either return error code
> or
> > > > throw
> > > > > an
> > > > > > > > > > exception
> > > > > > > > > > > > > when
> > > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > > >> > > > > 102.2. We probably shouldn't
> > explicitly
> > > > use
> > > > > > the
> > > > > > > > > > request
> > > > > > > > > > > > > object
> > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > > >> > > > > Not every request evolution requires
> > an
> > > > api
> > > > > > > > change.
> > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM,
> > Andrii
> > > > > > > Biletskyi
> > > > > > > > <
> > > > > > > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > > Thanks for you comments. Answers
> > > inline:
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > > 100. There are a few fields such
> as
> > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized that
> are
> > > > > > > represented
> > > > > > > > as a
> > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > >> > > > > > > composite structures in json.
> > Could
> > > we
> > > > > > > flatten
> > > > > > > > > > them
> > > > > > > > > > > > out
> > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > arrays/records?
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > > Yes, now with Admin Client this
> > looks
> > > a
> > > > > bit
> > > > > > > > weird.
> > > > > > > > > > My
> > > > > > > > > > > > > initial
> > > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > > >> > > > > > ReassignPartitionCommand accepts
> > input
> > > > in
> > > > > > > json,
> > > > > > > > we
> > > > > > > > > > > want
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > > >> > > > > > interfaces unchanged, where
> > possible.
> > > > > > > > > > > > > > > >> > > > > > If we port it to deserialized
> > format,
> > > in
> > > > > CLI
> > > > > > > > (/tools
> > > > > > > > > > > > > project)
> > > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > > >> > > > > > json library since /tools is
> written
> > > in
> > > > > java
> > > > > > > and
> > > > > > > > > > we'll
> > > > > > > > > > > > > need to
> > > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > > >> > > > > > provided by a user. Can we quickly
> > > agree
> > > > > on
> > > > > > > what
> > > > > > > > > > this
> > > > > > > > > > > > > library
> > > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > > 101. Does TopicMetadataRequest v1
> > > still
> > > > > > > trigger
> > > > > > > > auto
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now that we
> > > have a
> > > > > > > > separate
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > > > > > createTopicRequest
> > > > > > > > > > and
> > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > producer/consumer
> > > > > > > > client,
> > > > > > > > > > in
> > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > >> > > > > > > tools? For example, ideally, we
> > > don't
> > > > > want
> > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto topic
> > > > creation.
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > > I agree, this strange logic should
> > be
> > > > > fixed.
> > > > > > > > I'm not
> > > > > > > > > > > > > confident
> > > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > > >> > > > > > correct me if I'm wrong, but it
> > > doesn't
> > > > > look
> > > > > > > > like a
> > > > > > > > > > > hard
> > > > > > > > > > > > > thing
> > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > > >> > > > > > leverage AdminClient for that in
> > > > Producer
> > > > > > and
> > > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > >> > > > > > creation from the
> > > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant getting rid
> of
> > > > scala
> > > > > > > > classes
> > > > > > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader
> and
> > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when adding
> > the
> > > > new
> > > > > > > > requests
> > > > > > > > > > for
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > >> > > > > > > However, the long term plan is
> to
> > > get
> > > > > rid
> > > > > > of
> > > > > > > > all
> > > > > > > > > > > those
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > >> > > > > > > java request/response in the
> > client.
> > > > > Since
> > > > > > > > this
> > > > > > > > > > KIP
> > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > requests,
> > > > > > perhaps
> > > > > > > we
> > > > > > > > > > > should
> > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > >> > > > > > > clean up the existing scala
> > requests
> > > > > first
> > > > > > > > before
> > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > > Yes, looks like I misunderstood
> the
> > > > point
> > > > > of
> > > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > > >> > > > > > rework that. The only thing is
> that
> > I
> > > > > don't
> > > > > > > see
> > > > > > > > any
> > > > > > > > > > > > > example
> > > > > > > > > > > > > > > how
> > > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > > >> > > > > > least one existing protocol
> message.
> > > > Thus,
> > > > > > as
> > > > > > > I
> > > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > > >> > > > > > Re porting all existing RQ/RP in
> > this
> > > > > patch.
> > > > > > > > Sounds
> > > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > > >> > > > > > requirement to have Admin KIP
> done,
> > > I'm
> > > > > > afraid
> > > > > > > > this
> > > > > > > > > > > can
> > > > > > > > > > > > > be a
> > > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > > >> > > > > > There are 13 protocol messages and
> > all
> > > > > that
> > > > > > > > would
> > > > > > > > > > > > require
> > > > > > > > > > > > > not
> > > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > > >> > > > > > intensive manual testing, no? I'm
> > > afraid
> > > > > I'm
> > > > > > > > not the
> > > > > > > > > > > > > right guy
> > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > > >> > > > > > Kafka core internals :). Let me
> know
> > > > your
> > > > > > > > thoughts
> > > > > > > > > > on
> > > > > > > > > > > > this
> > > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > > > >> > >
> > > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM,
> Jun
> > > > Rao <
> > > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > 100. There are a few fields such
> > as
> > > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized that
> are
> > > > > > > represented
> > > > > > > > as a
> > > > > > > > > > > > > string,
> > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > > >> > > > > > > composite structures in json.
> > Could
> > > we
> > > > > > > flatten
> > > > > > > > > > them
> > > > > > > > > > > > out
> > > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > > arrays/records?
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > 101. Does TopicMetadataRequest
> v1
> > > > still
> > > > > > > > trigger
> > > > > > > > > > auto
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > > >> > > > > > > will be a bit weird now that we
> > > have a
> > > > > > > > separate
> > > > > > > > > > > topic
> > > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > > > > > createTopicRequest
> > > > > > > > > > and
> > > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > > producer/consumer
> > > > > > > > client,
> > > > > > > > > > in
> > > > > > > > > > > > > addition
> > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > > >> > > > > > > tools? For example, ideally, we
> > > don't
> > > > > want
> > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto topic
> > > > creation.
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant getting rid
> > of
> > > > > scala
> > > > > > > > classes
> > > > > > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader
> and
> > > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when adding
> > the
> > > > new
> > > > > > > > requests
> > > > > > > > > > for
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > > >> > > > > > > However, the long term plan is
> to
> > > get
> > > > > rid
> > > > > > of
> > > > > > > > all
> > > > > > > > > > > those
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > >> > > > > > > java request/response in the
> > client.
> > > > > Since
> > > > > > > > this
> > > > > > > > > > KIP
> > > > > > > > > > > > > proposes
> > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > > >> > > > > > > significant number of new
> > requests,
> > > > > > perhaps
> > > > > > > we
> > > > > > > > > > > should
> > > > > > > > > > > > > bite
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > >> > > > > > > clean up the existing scala
> > requests
> > > > > first
> > > > > > > > before
> > > > > > > > > > > > > adding new
> > > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM,
> > > > Andrii
> > > > > > > > Biletskyi
> > > > > > > > > > <
> > > > > > > > > > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly>
> > wrote:
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > As said above - I list again
> all
> > > > > > comments
> > > > > > > > from
> > > > > > > > > > > this
> > > > > > > > > > > > > thread
> > > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > > >> > > > > > > > can see what's left and
> finalize
> > > all
> > > > > > > pending
> > > > > > > > > > > issues.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > > > >> > > > > > > > 1. This is much needed
> > > > functionality,
> > > > > > but
> > > > > > > > there
> > > > > > > > > > > are
> > > > > > > > > > > > a
> > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > > >> > > > > > > > really think these protocols
> > > > through.
> > > > > We
> > > > > > > > really
> > > > > > > > > > > want
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > end
> > > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > > >> > > > > > > > of well thought-out,
> orthoganol
> > > > apis.
> > > > > > For
> > > > > > > > this
> > > > > > > > > > > > reason
> > > > > > > > > > > > > I
> > > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > > >> > > > > > > > important to think through the
> > end
> > > > > state
> > > > > > > > even if
> > > > > > > > > > > > that
> > > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > >> > > > > > > > won't implement in the first
> > > phase.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: Definitely behind this.
> Would
> > > > > > > appreciate
> > > > > > > > if
> > > > > > > > > > > there
> > > > > > > > > > > > > are
> > > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 2. Let's please please please
> > wait
> > > > > until
> > > > > > > we
> > > > > > > > have
> > > > > > > > > > > > > switched
> > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > > >> > > > > > > > to the new java protocol
> > > > definitions.
> > > > > If
> > > > > > > we
> > > > > > > > add
> > > > > > > > > > > > upteen
> > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > > >> > > > > > > > objects that is just
> generating
> > > more
> > > > > > work
> > > > > > > > for
> > > > > > > > > > the
> > > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > > > removed
> > > > > > > scala
> > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 3. This proposal introduces a
> > new
> > > > type
> > > > > > of
> > > > > > > > > > optional
> > > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > > >> > > > > > > > inconsistent with everything
> > else
> > > in
> > > > > the
> > > > > > > > > > protocol
> > > > > > > > > > > > > where we
> > > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > > >> > > > > > > > other marker value. You could
> > > argue
> > > > > > either
> > > > > > > > way
> > > > > > > > > > but
> > > > > > > > > > > > > let's
> > > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > >> > > > > > > > for consistency. For clients
> > that
> > > > > > > > implemented
> > > > > > > > > > the
> > > > > > > > > > > > > protocol
> > > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > > >> > > > > > > > than our scala code these
> basic
> > > > > > primitives
> > > > > > > > are
> > > > > > > > > > > hard
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > > > removed
> > > > > > > > MaybeOf
> > > > > > > > > > > type
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This seems
> > to
> > > > > > > duplicate
> > > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > >> > > > > > > > brokers, topics, and
> > partitions. I
> > > > > think
> > > > > > > we
> > > > > > > > > > should
> > > > > > > > > > > > > rename
> > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > > >> > > > > > > > ClusterMetadataRequest (or
> just
> > > > > > > > MetadataRequest)
> > > > > > > > > > > and
> > > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > >> > > > > > > > controller. Or are there other
> > > > things
> > > > > we
> > > > > > > > could
> > > > > > > > > > add
> > > > > > > > > > > > > here?
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: I agree. Updated the KIP.
> > Let's
> > > > > > extends
> > > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 5. We have a tendency to try
> to
> > > > make a
> > > > > > lot
> > > > > > > > of
> > > > > > > > > > > > requests
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > > >> > > > > > > > particular nodes. This adds a
> > lot
> > > of
> > > > > > > burden
> > > > > > > > for
> > > > > > > > > > > > client
> > > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > > >> > > > > > > > sounds easy but each discovery
> > can
> > > > > fail
> > > > > > in
> > > > > > > > many
> > > > > > > > > > > > parts
> > > > > > > > > > > > > so
> > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > > >> > > > > > > > full state machine to do
> > right). I
> > > > > think
> > > > > > > we
> > > > > > > > > > should
> > > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > > >> > > > > > > > commands and ideally as many
> of
> > > the
> > > > > > other
> > > > > > > > apis
> > > > > > > > > > as
> > > > > > > > > > > > > possible
> > > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > > >> > > > > > > > brokers and just redirect to
> the
> > > > > > > controller
> > > > > > > > on
> > > > > > > > > > the
> > > > > > > > > > > > > broker
> > > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > > >> > > > > > > > there would be a general way
> to
> > > > > > > encapsulate
> > > > > > > > this
> > > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: It's a very interesting
> idea,
> > > but
> > > > > > seems
> > > > > > > > there
> > > > > > > > > > > are
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > > >> > > > > > > > feature (like performance
> > > > > > considerations,
> > > > > > > > how
> > > > > > > > > > this
> > > > > > > > > > > > > will
> > > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > > >> > > > > > > > I believe this shouldn't be a
> > > > blocker.
> > > > > > If
> > > > > > > > this
> > > > > > > > > > > > > feature is
> > > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > > >> > > > > > > > point it won't affect Admin
> > > changes
> > > > -
> > > > > at
> > > > > > > > least
> > > > > > > > > > no
> > > > > > > > > > > > > changes
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 6. We should probably
> normalize
> > > the
> > > > > key
> > > > > > > > value
> > > > > > > > > > > pairs
> > > > > > > > > > > > > used
> > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > >> > > > > > > > than embedding a new
> formatting.
> > > So
> > > > > two
> > > > > > > > strings
> > > > > > > > > > > > rather
> > > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > > > > > normalized
> > > > > > > > > > configs
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 7. Is the postcondition of
> these
> > > > APIs
> > > > > > that
> > > > > > > > the
> > > > > > > > > > > > > command has
> > > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > > >> > > > > > > > the command has been
> completed?
> > It
> > > > is
> > > > > a
> > > > > > > lot
> > > > > > > > more
> > > > > > > > > > > > > usable if
> > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > > >> > > > > > > > been completed so you know
> that
> > if
> > > > you
> > > > > > > > create a
> > > > > > > > > > > > topic
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > >> > > > > > > > it you won't get an exception
> > > about
> > > > > > there
> > > > > > > > being
> > > > > > > > > > no
> > > > > > > > > > > > > such
> > > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: For long running requests
> > (like
> > > > > > > reassign
> > > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > > >> > > > > > > > command has begun - so we
> don't
> > > > block
> > > > > > the
> > > > > > > > > > client.
> > > > > > > > > > > In
> > > > > > > > > > > > > case
> > > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > > >> > > > > > > > topic commands, this will be
> > > > > refactored
> > > > > > > and
> > > > > > > > > > topic
> > > > > > > > > > > > > commands
> > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > > >> > > > > > > > immediately, since the
> > Controller
> > > > will
> > > > > > > serve
> > > > > > > > > > Admin
> > > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 8. Describe topic and list
> > topics
> > > > > > > duplicate
> > > > > > > > a
> > > > > > > > > > lot
> > > > > > > > > > > of
> > > > > > > > > > > > > stuff
> > > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > > >> > > > > > > > request. Is there a reason to
> > give
> > > > > back
> > > > > > > > topics
> > > > > > > > > > > > marked
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > > >> > > > > > > > like if we just make the
> > > > > post-condition
> > > > > > of
> > > > > > > > the
> > > > > > > > > > > > delete
> > > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > >> > > > > > > > topic is deleted that will get
> > rid
> > > > of
> > > > > > the
> > > > > > > > need
> > > > > > > > > > for
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > > > removed
> > > > > > > > topics
> > > > > > > > > > > marked
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 9. Should we consider batching
> > > these
> > > > > > > > requests?
> > > > > > > > > > We
> > > > > > > > > > > > have
> > > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > >> > > > > > > > allow multiple operations to
> be
> > > > > batched.
> > > > > > > My
> > > > > > > > > > > > suspicion
> > > > > > > > > > > > > is
> > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > >> > > > > > > > we will get a lot of code that
> > > does
> > > > > > > > something
> > > > > > > > > > like
> > > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > > adminClient.listTopics())
> > > > > > > > > > > > > > > >> > > > > > > >
> > > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > > >> > > > > > > > this code will work great when
> > you
> > > > > test
> > > > > > > on 5
> > > > > > > > > > > topics
> > > > > > > > > > > > > but
> > > > > > > > > > > > > > > not
> > > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please
> > check
> > > > > "Topic
> > > > > > > > Admin
> > > > > > > > > > > > Schema"
> > > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 10. I think we should also
> > discuss
> > > > how
> > > > > > we
> > > > > > > > want
> > > > > > > > > > to
> > > > > > > > > > > > > expose a
> > > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > > >> > > > > > > > client api for these
> operations.
> > > > > > Currently
> > > > > > > > > > people
> > > > > > > > > > > > > rely on
> > > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > > >> > > > > > > > is totally sketchy. I think we
> > > > > probably
> > > > > > > need
> > > > > > > > > > > another
> > > > > > > > > > > > > > > client
> > > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > > >> > > > > > > > that exposes administrative
> > > > > > functionality.
> > > > > > > > We
> > > > > > > > > > will
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > >> > > > > > > > properly test the new apis, I
> > > > suspect.
> > > > > > We
> > > > > > > > should
> > > > > > > > > > > > > figure
> > > > > > > > > > > > > > > out
> > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please
> > check
> > > > > "Admin
> > > > > > > > Client"
> > > > > > > > > > > > > section
> > > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 11. The other information that
> > > would
> > > > > be
> > > > > > > > really
> > > > > > > > > > > > useful
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > get
> > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > >> > > > > > > > information about
> > partitions--how
> > > > much
> > > > > > > data
> > > > > > > > is
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > > >> > > > > > > > the segment offsets, what is
> the
> > > > > log-end
> > > > > > > > offset
> > > > > > > > > > > > (i.e.
> > > > > > > > > > > > > last
> > > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > > >> > > > > > > > the compaction point, etc. I
> > think
> > > > > that
> > > > > > > done
> > > > > > > > > > right
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > >> > > > > > > > successor to the very awkward
> > > > > > > OffsetRequest
> > > > > > > > we
> > > > > > > > > > > have
> > > > > > > > > > > > > today.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > > ConsumerGroupOffsetsRequest
> > > > > > > in
> > > > > > > > the
> > > > > > > > > > > > latest
> > > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > > >> > > > > > > > be resolved in a separate KIP
> /
> > > jira
> > > > > > > ticket.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 12. Generally we can do good
> > error
> > > > > > > handling
> > > > > > > > > > > without
> > > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > > >> > > > > > > > messages. I.e. generally the
> > > client
> > > > > has
> > > > > > > the
> > > > > > > > > > > context
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > > >> > > > > > > > an error that the topic
> doesn't
> > > > exist
> > > > > to
> > > > > > > say
> > > > > > > > > > > "Topic
> > > > > > > > > > > > X
> > > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > > >> > > > > > > > than "error code 14" (or
> > > whatever).
> > > > > > Maybe
> > > > > > > > there
> > > > > > > > > > > are
> > > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > > >> > > > > > > > this is hard? If we want to
> add
> > > > > > > server-side
> > > > > > > > > > error
> > > > > > > > > > > > > messages
> > > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > > >> > > > > > > > need to do this in a
> consistent
> > > way
> > > > > > across
> > > > > > > > the
> > > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please
> > check
> > > > > > > "Protocol
> > > > > > > > > > > Errors"
> > > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > >> > > > > > > > comprehensive, fine-grained
> list
> > > of
> > > > > > error
> > > > > > > > codes.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > > > > > > > >> > > > > > > > 13. Describe topic request: it
> > > would
> > > > > be
> > > > > > > > great to
> > > > > > > > > > > go
> > > > > > > > > > > > > beyond
> > > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > >> > > > > > > > topic name regex for this
> > request.
> > > > For
> > > > > > > > example,
> > > > > > > > > > a
> > > > > > > > > > > > very
> > > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > > >> > > > > > > > the topic command is to list
> all
> > > > > topics
> > > > > > > > whose
> > > > > > > > > > > config
> > > > > > > > > > > > > A's
> > > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > > >> > > > > > > > topic name regex then we have
> to
> > > > first
> > > > > > > > retrieve
> > > > > > > > > > > > > __all__
> > > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > > >> > > > > > > > description info and then
> filter
> > > at
> > > > > the
> > > > > > > > client
> > > > > > > > > > > end,
> > > > > > > > > > > > > which
> > > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in create
> topic:
> > > > this
> > > > > is
> > > > > > > > related
> > > > > > > > > > > to
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > > >> > > > > > > > maybe we can add another
> > metadata
> > > > K-V
> > > > > or
> > > > > > > > just a
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > > >> > > > > > > > with config K-V in create
> topic
> > > like
> > > > > we
> > > > > > > did
> > > > > > > > for
> > > > > > > > > > > > offset
> > > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > > >> > > > > > > > field can be quite useful in
> > > storing
> > > > > > > > information
> > > > > > > > > > > > like
> > > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > > >> > > > > > > > who issue the create command,
> > etc,
> > > > > which
> > > > > > > is
> > > > > > > > > > quite
> > > > > > > > > > > > > > > important
> > > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > > >> > > > > > > > multi-tenant setting. Then in
> > the
> > > > > > describe
> > > > > > > > topic
> > > > > > > > > > > > > request
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > > >> > > > > > > > on regex of the metadata
> field.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: As discussed it is very
> > > > interesting
> > > > > > but
> > > > > > > > can
> > > > > > > > > > be
> > > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > > >> > > > > > > > we have some basic
> functionality
> > > > > there.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > 15. Today all the admin
> > operations
> > > > are
> > > > > > > > async in
> > > > > > > > > > > the
> > > > > > > > > > > > > sense
> > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > > >> > > > > > > > return once it is written in
> ZK,
> > > and
> > > > > > that
> > > > > > > > is why
> > > > > > > > > > > we
> > > > > > > > > > > > > need
> > > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > > >> > > > > > > > like
> > > testUtil.waitForTopicCreated()
> > > > /
> > > > > > > verify
> > > > > > > > > > > > partition
> > > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > > >> > > > > > > > request, etc. With admin
> > requests
> > > we
> > > > > > could
> > > > > > > > add a
> > > > > > > > > > > > flag
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > > >> > > > > > > > synchronous requests; when it
> is
> > > > > turned
> > > > > > > on,
> > > > > > > > the
> > > > > > > > > > > > > response
> > > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > > >> > > > > > > > until the request has been
> > > > completed.
> > > > > > And
> > > > > > > > for
> > > > > > > > > > > async
> > > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > > >> > > > > > > > "token" field in the response,
> > and
> > > > > then
> > > > > > > only
> > > > > > > > > > need
> > > > > > > > > > > a
> > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > > >> > > > > > > > verification request" with the
> > > given
> > > > > > token
> > > > > > > > to
> > > > > > > > > > > check
> > > > > > > > > > > > > if the
> > > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: I see your point. My idea
> was
> > > to
> > > > > > > provide
> > > > > > > > > > > specific
> > > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > > >> > > > > > > > long running request, where
> > > needed.
> > > > We
> > > > > > can
> > > > > > > > do it
> > > > > > > > > > > the
> > > > > > > > > > > > > way
> > > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > > >> > > > > > > > concern is that introducing a
> > > token
> > > > we
> > > > > > > again
> > > > > > > > > > will
> > > > > > > > > > > > make
> > > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > > >> > > > > > > > to do similar thing
> introducing
> > > > single
> > > > > > > > > > > AdminRequest
> > > > > > > > > > > > > for
> > > > > > > > > > > > > > > all
> > > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > > >> > > > > > > > this idea because we wanted to
> > > have
> > > > > > schema
> > > > > > > > > > > defined.
> > > > > > > > > > > > So
> > > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > > >> > > > > > > > a) have fixed schema but
> > introduce
> > > > > each
> > > > > > > > time new
> > > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > > > > > > >> > > > > > > > b) use one request for
> > > verification
> > > > > but
> > > > > > > > > > generalize
> > > > > > > > > > > > it
> > > > > > > > > > > > > with
> > > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > > >> > > > > > > > I'm fine with whatever
> decision
> > > > > > community
> > > > > > > > come
> > > > > > > > > > to.
> > > > > > > > > > > > > Just
> > > > > > > > > > > > > > > let
> > > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > > > >> > > > > > > > 16. Specifically for
> ownership,
> > I
> > > > > think
> > > > > > > the
> > > > > > > > plan
> > > > > > > > > > > is
> > > > > > > > > > > > > to add
> > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > > >> > > > > > > > like you are describing ACL)
> via
> > > an
> > > > > > > external
> > > > > > > > > > > system
> > > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > > >> > > > > > > > I remember KIP-11 described
> > this,
> > > > but
> > > > > I
> > > > > > > > can't
> > > > > > > > > > find
> > > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > A: Okay, no problem. Not sure
> > > though
> > > > > how
> > > > > > > we
> > > > > > > > are
> > > > > > > > > > > > going
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > > >> > > > > > > > will be committed first and
> > > include
> > > > > > > changes
> > > > > > > > to
> > > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > > >> > > > > > > > Anyway, I added this note to
> > "Open
> > > > > > > > Questions"
> > > > > > > > > > > > section
> > > > > > > > > > > > > so
> > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34
> > AM,
> > > > > Andrii
> > > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > > >> > > > > > > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > > Today I uploaded the patch
> > that
> > > > > covers
> > > > > > > > some of
> > > > > > > > > > > the
> > > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf optional
> > type
> > > > > > > > > > > > > > > >> > > > > > > > > - switched to java protocol
> > > > > > definitions
> > > > > > > > > > > > > > > >> > > > > > > > > - simplified messages
> > > (normalized
> > > > > > > configs,
> > > > > > > > > > > removed
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > > I also updated the KIP-4
> with
> > > > > > respective
> > > > > > > > > > changes
> > > > > > > > > > > > and
> > > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin Operations ->
> > > > updated
> > > > > > Wire
> > > > > > > > > > > Protocol
> > > > > > > > > > > > > schema
> > > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > > >> > > > > > > > > - Remove ClusterMetadata ->
> > > > changed
> > > > > to
> > > > > > > > extend
> > > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > > >> > > > > > > > > - Admin Client -> updated my
> > > > initial
> > > > > > > > proposal
> > > > > > > > > > to
> > > > > > > > > > > > > reflect
> > > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > > >> > > > > > > > > - Error codes -> proposed
> > > > > fine-grained
> > > > > > > > error
> > > > > > > > > > > code
> > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > > I will also send a separate
> > > email
> > > > to
> > > > > > > > cover all
> > > > > > > > > > > > > comments
> > > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26
> > PM,
> > > > > Gwen
> > > > > > > > Shapira
> > > > > > > > > > <
> > > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > > >> > > > > > > > >> It actually specifies
> changes
> > > to
> > > > > the
> > > > > > > > Metadata
> > > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are consistent in
> > > this
> > > > > > regard
> > > > > > > > will
> > > > > > > > > > be
> > > > > > > > > > > > > good.
> > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at
> 12:21
> > > PM,
> > > > > > Gwen
> > > > > > > > > > Shapira
> > > > > > > > > > > <
> > > > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> > Specifically for
> > ownership, I
> > > > > think
> > > > > > > the
> > > > > > > > > > plan
> > > > > > > > > > > is
> > > > > > > > > > > > > to
> > > > > > > > > > > > > > > add
> > > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > > >> > > > > > > > >> > like you are describing
> > ACL)
> > > > via
> > > > > an
> > > > > > > > > > external
> > > > > > > > > > > > > system
> > > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > > >> > > > > > > > >> > I remember KIP-11
> described
> > > > this,
> > > > > > > but I
> > > > > > > > > > can't
> > > > > > > > > > > > > find
> > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I think KIP-4
> > > > focuses
> > > > > > on
> > > > > > > > > > getting
> > > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > > >> > > > > > > > >> > exists from Kafka
> brokers,
> > > not
> > > > on
> > > > > > > > adding
> > > > > > > > > > > > > information
> > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > > >> > > > > > > > >> > should exist but doesn't
> > yet?
> > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at
> > 6:37
> > > > AM,
> > > > > > > > Guozhang
> > > > > > > > > > > Wang
> > > > > > > > > > > > <
> > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >> Just want to elaborate a
> > bit
> > > > > more
> > > > > > on
> > > > > > > > the
> > > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > > >> > > > > > > > >> >> describe-topic based on
> > > > config /
> > > > > > > > metadata
> > > > > > > > > > in
> > > > > > > > > > > > my
> > > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The main
> > > > > motivation
> > > > > > > is
> > > > > > > > to
> > > > > > > > > > > have
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms, which I
> think
> > is
> > > > > quite
> > > > > > > > > > important
> > > > > > > > > > > > in
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > > >> > > > > > > > >> >> architecture: today
> anyone
> > > can
> > > > > > > create
> > > > > > > > > > topics
> > > > > > > > > > > > in
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > > >> > > > > > > > >> >> there is no concept or
> > > > > "ownership"
> > > > > > > of
> > > > > > > > > > topics
> > > > > > > > > > > > > that
> > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > > >> > > > > > > > >> >> users. For example, at
> > > > LinkedIn
> > > > > we
> > > > > > > > > > basically
> > > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > > >> > > > > > > > >> >> some casual topic name
> > > prefix,
> > > > > > which
> > > > > > > > is a
> > > > > > > > > > > bit
> > > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > > >> > > > > > > > >> >> we scale our customers.
> It
> > > > would
> > > > > > be
> > > > > > > > great
> > > > > > > > > > to
> > > > > > > > > > > > use
> > > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics that
> > is
> > > > > > created
> > > > > > > > by me.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics
> whose
> > > > > > retention
> > > > > > > > time
> > > > > > > > > > is
> > > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics
> whose
> > > > > writable
> > > > > > > > group
> > > > > > > > > > > > include
> > > > > > > > > > > > > > > user
> > > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >> One possible way to
> > achieve
> > > > this
> > > > > > is
> > > > > > > to
> > > > > > > > > > add a
> > > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > > >> > > > > > > > >> >> create-topic request,
> > whose
> > > > > value
> > > > > > > will
> > > > > > > > > > also
> > > > > > > > > > > be
> > > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > > >> > > > > > > > >> >> topic; then
> > describe-topics
> > > > can
> > > > > > > > choose to
> > > > > > > > > > > > batch
> > > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2) config K-V
> > > matching,
> > > > > 3)
> > > > > > > > metadata
> > > > > > > > > > > > > regex,
> > > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at
> > 4:37
> > > > PM,
> > > > > > > > Guozhang
> > > > > > > > > > > Wang
> > > > > > > > > > > > <
> > > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for the updated
> > > wiki.
> > > > A
> > > > > > few
> > > > > > > > > > comments
> > > > > > > > > > > > > below:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error description in
> > > > > > response: I
> > > > > > > > think
> > > > > > > > > > > if
> > > > > > > > > > > > > some
> > > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> several different error
> > > cases
> > > > > > then
> > > > > > > we
> > > > > > > > > > > should
> > > > > > > > > > > > > really
> > > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In general the
> > > > errorCode
> > > > > > > > itself
> > > > > > > > > > > would
> > > > > > > > > > > > be
> > > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> describing the server
> > side
> > > > > > errors.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic
> > request:
> > > it
> > > > > > would
> > > > > > > > be
> > > > > > > > > > > great
> > > > > > > > > > > > > to go
> > > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex for
> this
> > > > > > request.
> > > > > > > > For
> > > > > > > > > > > > > example, a
> > > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic command is to
> > > list
> > > > > all
> > > > > > > > topics
> > > > > > > > > > > whose
> > > > > > > > > > > > > > > config
> > > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex then
> we
> > > have
> > > > > to
> > > > > > > > first
> > > > > > > > > > > > retrieve
> > > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> description info and
> then
> > > > > filter
> > > > > > at
> > > > > > > > the
> > > > > > > > > > > > client
> > > > > > > > > > > > > end,
> > > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in
> create
> > > > topic:
> > > > > > > this
> > > > > > > > is
> > > > > > > > > > > > > related to
> > > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can add
> another
> > > > > metadata
> > > > > > > > K-V or
> > > > > > > > > > > > just a
> > > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> with config K-V in
> create
> > > > topic
> > > > > > > like
> > > > > > > > we
> > > > > > > > > > did
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> field can be quite
> useful
> > > in
> > > > > > > storing
> > > > > > > > > > > > > information
> > > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue the create
> > > command,
> > > > > > etc,
> > > > > > > > which
> > > > > > > > > > is
> > > > > > > > > > > > > quite
> > > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant setting.
> > Then
> > > in
> > > > > the
> > > > > > > > > > describe
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of the
> metadata
> > > > field.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all the admin
> > > > > operations
> > > > > > > are
> > > > > > > > > > async
> > > > > > > > > > > > in
> > > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> return once it is
> written
> > > in
> > > > > ZK,
> > > > > > > and
> > > > > > > > that
> > > > > > > > > > > is
> > > > > > > > > > > > > why we
> > > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > > verify
> > > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> request, etc. With
> admin
> > > > > requests
> > > > > > > we
> > > > > > > > > > could
> > > > > > > > > > > > add
> > > > > > > > > > > > > a
> > > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous requests;
> > when
> > > it
> > > > > is
> > > > > > > > turned
> > > > > > > > > > on,
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> until the request has
> > been
> > > > > > > > completed. And
> > > > > > > > > > > for
> > > > > > > > > > > > > async
> > > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> "token" field in the
> > > > response,
> > > > > > and
> > > > > > > > then
> > > > > > > > > > > only
> > > > > > > > > > > > > need a
> > > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> verification request"
> > with
> > > > the
> > > > > > > given
> > > > > > > > > > token
> > > > > > > > > > > to
> > > > > > > > > > > > > check
> > > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for extending
> > > Metadata
> > > > > > > request
> > > > > > > > to
> > > > > > > > > > > > include
> > > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> information, and then
> we
> > > can
> > > > > > remove
> > > > > > > > the
> > > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at
> > > 10:23
> > > > > AM,
> > > > > > > Joel
> > > > > > > > > > > Koshy <
> > > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for sending
> that
> > > out
> > > > > Joe
> > > > > > -
> > > > > > > I
> > > > > > > > > > don't
> > > > > > > > > > > > > think I
> > > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so if notes
> > can
> > > be
> > > > > > sent
> > > > > > > > out
> > > > > > > > > > > > > afterward
> > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015
> at
> > > > > > 09:16:13AM
> > > > > > > > > > -0800,
> > > > > > > > > > > > Gwen
> > > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for sending
> > this
> > > > out
> > > > > > Joe.
> > > > > > > > > > Looking
> > > > > > > > > > > > > forward
> > > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015
> at
> > > > 6:46
> > > > > > AM,
> > > > > > > > Joe
> > > > > > > > > > > Stein
> > > > > > > > > > > > <
> > > > > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just sent
> > out a
> > > > > > google
> > > > > > > > > > hangout
> > > > > > > > > > > > > invite
> > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I found
> > > working
> > > > > on
> > > > > > a
> > > > > > > > KIP.
> > > > > > > > > > If
> > > > > > > > > > > I
> > > > > > > > > > > > > missed
> > > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me know and
> can
> > > > update
> > > > > > it,
> > > > > > > > np.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We should do this
> > > every
> > > > > > > Tuesday
> > > > > > > > @
> > > > > > > > > > 2pm
> > > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to make a
> > google
> > > > > > account
> > > > > > > > so we
> > > > > > > > > > > can
> > > > > > > > > > > > > manage
> > > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in progress and
> > > related
> > > > > JIRA
> > > > > > > > that
> > > > > > > > > > are
> > > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24,
> 2015
> > > at
> > > > > 2:59
> > > > > > > > PM, Jay
> > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on
> > Google
> > > > > > hangouts
> > > > > > > > that
> > > > > > > > > > > will
> > > > > > > > > > > > > also
> > > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> available on
> > youtube.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24,
> > 2015
> > > at
> > > > > > 11:49
> > > > > > > > AM,
> > > > > > > > > > > Jeff
> > > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> jholoman@cloudera.com
> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're happy to
> > send
> > > > > out a
> > > > > > > > Webex
> > > > > > > > > > for
> > > > > > > > > > > > > this
> > > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions if
> there
> > > is
> > > > > > > > interest and
> > > > > > > > > > > > > publish
> > > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24,
> > > 2015
> > > > at
> > > > > > > > 11:28 AM,
> > > > > > > > > > > Jay
> > > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try to
> > get
> > > > the
> > > > > > > > technical
> > > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there is some
> > > > benefit
> > > > > > to
> > > > > > > > live
> > > > > > > > > > > > > discussion
> > > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post
> > > > instructions
> > > > > > and
> > > > > > > > give
> > > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at
> that
> > > > time
> > > > > > > would
> > > > > > > > work
> > > > > > > > > > > for
> > > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb
> 24,
> > > > 2015
> > > > > at
> > > > > > > > 8:18
> > > > > > > > > > AM,
> > > > > > > > > > > > Joe
> > > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly
> would
> > be
> > > > > great
> > > > > > > > maybe
> > > > > > > > > > > like
> > > > > > > > > > > > > every
> > > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't
> mind
> > > > google
> > > > > > > > hangout
> > > > > > > > > > but
> > > > > > > > > > > > > there
> > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we know the
> > > > apache
> > > > > > irc
> > > > > > > > > > channel
> > > > > > > > > > > > > works.
> > > > > > > > > > > > > > > We
> > > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes? We
> can
> > > pull
> > > > > > > > transcripts
> > > > > > > > > > > too
> > > > > > > > > > > > > and
> > > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it helpful
> > for
> > > > > > things.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb
> > 24,
> > > > > 2015
> > > > > > at
> > > > > > > > 11:10
> > > > > > > > > > > AM,
> > > > > > > > > > > > > Jay
> > > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd
> talked
> > > > about
> > > > > > > > doing a
> > > > > > > > > > > > Google
> > > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > generalizing
> > > > > that a
> > > > > > > > little
> > > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > everyone
> > > > > spending a
> > > > > > > > > > > reasonable
> > > > > > > > > > > > > chunk
> > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync up
> > once
> > > a
> > > > > > week.
> > > > > > > I
> > > > > > > > > > think
> > > > > > > > > > > we
> > > > > > > > > > > > > could
> > > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff,
> make
> > > > sure
> > > > > we
> > > > > > > > are on
> > > > > > > > > > > top
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > issues,
> > etc.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We can
> make
> > > it
> > > > > > > publicly
> > > > > > > > > > > > > available so
> > > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any
> > interest
> > > in
> > > > > > doing
> > > > > > > > this?
> > > > > > > > > > > If
> > > > > > > > > > > > so
> > > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue,
> Feb
> > > 24,
> > > > > > 2015
> > > > > > > at
> > > > > > > > > > 3:57
> > > > > > > > > > > > AM,
> > > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > I've
> > > updated
> > > > > KIP
> > > > > > > > page,
> > > > > > > > > > > fixed
> > > > > > > > > > > > /
> > > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > > > >
> > > > > ...
> > > > >
> > > > > [Message clipped]
> > > >
> > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
For 1), 2) and 3), blocking would probably mean that the new metadata is
propagated to every broker. To achieve that, the client can keep issuing
the describe topic request to every broker until it sees the new metadata
in the response.

Thanks,

Jun

On Fri, Mar 20, 2015 at 12:16 PM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Hm, actually the ticket you linked, Guozhang, brings as back
> to the problem what should be considered a post-condition for
> each of the admin commands.
> In my understanding:
>
> 1) CreateTopic - broker created /brokers/topics/<topic>
> (Not the controller picked up changes from zk and broadcasted
> LeaderAndIsr and UpdateMetadata)
>
> 2) AlterTopic - same as 1) - broker changed assignment data
> in zookeeper or created admin path for topic config change
>
> 3) DeleteTopic - admin path /admin/delete_topics is created
>
> 4) ReassignPartitions and PreferredReplica - corresponding admin
> path is created
>
> Now what can be considered a completed operation from the client's
> perspective?
> 1) Topic is created once corresponding data is in zk
> (I remember there were some thoughts that it'd be good to consider
> topic created once all replicas receive information about it and thus
> clients can produce/consume from it, but as was discussed this seems
> to be a hard thing to do)
>
> 2) Probably same as 1), so right after AlterTopic is issued
>
> 3) The topic has been removed from /brokers/topics
>
> 4) ReassignPartitions and PrefferedReplica were discussed earlier -
> in short the former is completed once partition state info in zk matches
> reassignment request and admin path is empty, the latter - once data
> in zk shows that head of assignned replicas of the partition and leader
> is the same replica
>
> Thoughts?
>
> Thanks,
> Andrii Biletskyi
>
> On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > I think while loop is fine for supporting blocking, just that we need to
> > add back off to avoid bombarding brokers with DescribeTopic requests.
> >
> > Also I have linked KAFKA-1125
> > <https://issues.apache.org/jira/browse/KAFKA-1125> to your proposal, and
> > when KAFKA-1694 is done this ticket can also be closed.
> >
> > Guozhang
> >
> > On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Great.
> > > I want to elaborate this a bit more, to see we are on the same page
> > > concerning the client code.
> > >
> > > So with all topic commands being async a client (AdminClient in our
> > > case or any other other client people would like to implement) to
> support
> > > a blocking operation (which seems to be a natural use-case e.g. for
> topic
> > > creation): would have to do:
> > > 1. issue CreateTopicRequest
> > > 2. if successful, in a "while" loop send DescribeTopicRequest and
> > > break the loop once all topics are returned in response (or upon
> > timeout).
> > > 3. if unsuccessful throw exception
> > > Would it be okay?
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > >
> > > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Andrii,
> > > >
> > > > I think you are right. It seems that only ReassignPartitions needs a
> > > > separate verification request.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Guys,
> > > > > I like this idea too. Let's stick with that. I'll update KIP
> > > accordingly.
> > > > >
> > > > > I was also thinking we can avoid adding dedicated status check
> > > > > requests for topic commands. - We have everything in DescribeTopic
> > > > > for that! E.g.:
> > > > > User issued CreateTopic - to check the status client sends
> > > DescribeTopic
> > > > > and checks whether is something returned for that topic. The same
> for
> > > > > alteration, deletion.
> > > > > Btw, PreferredReplica status can be also checked with
> > > > DescribeTopicRequest
> > > > > (head of assigned replicas list == leader).
> > > > > For ReassignPartitions as discussed we'll need to have a separate
> > > > Verify...
> > > > > request.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > >
> > > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <wangguoz@gmail.com
> >
> > > > wrote:
> > > > >
> > > > > > +1 on broker writing to ZK for async handling. I was thinking
> that
> > in
> > > > the
> > > > > > end state the admin requests would be eventually sent to
> controller
> > > > > either
> > > > > > through re-routing or clients discovering them, instead of
> letting
> > > > > > controller listen on ZK admin path. But thinking about it a
> second
> > > > time,
> > > > > I
> > > > > > think it is actually simpler to let controller manage
> > > > > > incoming queued-up admin requests through ZK.
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <jjkoshy.w@gmail.com
> >
> > > > wrote:
> > > > > >
> > > > > > > +1 as well. I think it helps to keep the rerouting approach
> > > > orthogonal
> > > > > > > to this KIP.
> > > > > > >
> > > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps wrote:
> > > > > > > > I'm +1 on Jun's suggestion as long as it can work for all the
> > > > > requests.
> > > > > > > >
> > > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > > >
> > > > > > > > > Andrii,
> > > > > > > > >
> > > > > > > > > I think we agreed on the following.
> > > > > > > > >
> > > > > > > > > (a) Admin requests can be sent to and handled by any
> broker.
> > > > > > > > > (b) Admin requests are processed asynchronously, at least
> for
> > > > now.
> > > > > > > That is,
> > > > > > > > > when the client gets a response, it just means that the
> > request
> > > > is
> > > > > > > > > initiated, but not necessarily completed. Then, it's up to
> > the
> > > > > client
> > > > > > > to
> > > > > > > > > issue another request to check the status for completion.
> > > > > > > > >
> > > > > > > > > To support (a), we were thinking of doing request
> forwarding
> > to
> > > > the
> > > > > > > > > controller (utilizing KAFKA-1912). I am making an
> alternative
> > > > > > proposal.
> > > > > > > > > Basically, the broker can just write to ZooKeeper to inform
> > the
> > > > > > > controller
> > > > > > > > > about the request. For example, to handle
> > > partitionReassignment,
> > > > > the
> > > > > > > broker
> > > > > > > > > will just write the requested partitions to
> > > > > > /admin/reassign_partitions
> > > > > > > > > (like what AdminUtils currently does) and then send a
> > response
> > > to
> > > > > the
> > > > > > > > > client. This shouldn't take long and the implementation
> will
> > be
> > > > > > simpler
> > > > > > > > > than forwarding the requests to the controller through RPC.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >
> > > > > > > > > > Jun,
> > > > > > > > > >
> > > > > > > > > > I might be wrong but didn't we agree we will let any
> broker
> > > > from
> > > > > > the
> > > > > > > > > > cluster handle *long-running* admin requests (at this
> time
> > > > > > > > > preferredReplica
> > > > > > > > > > and
> > > > > > > > > > reassignPartitions), via zk admin path. Thus CreateTopics
> > etc
> > > > > > should
> > > > > > > be
> > > > > > > > > > sent
> > > > > > > > > > only to the controller.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Andrii Biletskyi
> > > > > > > > > >
> > > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> > jun@confluent.io>
> > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Joel, Andril,
> > > > > > > > > > >
> > > > > > > > > > > I think we agreed that those admin requests can be
> issued
> > > to
> > > > > any
> > > > > > > > > broker.
> > > > > > > > > > > Because of that, there doesn't seem to be a strong need
> > to
> > > > know
> > > > > > the
> > > > > > > > > > > controller. So, perhaps we can proceed by not making
> any
> > > > change
> > > > > > to
> > > > > > > the
> > > > > > > > > > > format of TMR right now. When we start using create
> topic
> > > > > request
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > producer, we will need a new version of TMR that
> doesn't
> > > > > trigger
> > > > > > > auto
> > > > > > > > > > topic
> > > > > > > > > > > creation. But that can be done later.
> > > > > > > > > > >
> > > > > > > > > > > As a first cut implementation, I think the broker can
> > just
> > > > > write
> > > > > > > to ZK
> > > > > > > > > > > directly for
> > > > > > > > > > >
> > > > > >
> createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > > requests, instead of forwarding them to the controller.
> > > This
> > > > > will
> > > > > > > > > > simplify
> > > > > > > > > > > the implementation on the broker side.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > > >
> > > > > > > > > > > > For (1) yes we will circle back on that shortly after
> > > > syncing
> > > > > > up
> > > > > > > in
> > > > > > > > > > > > person. I think it is close to getting committed
> > although
> > > > > > > development
> > > > > > > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > > > > > > >
> > > > > > > > > > > > There is one more item we covered at the hangout.
> i.e.,
> > > > > whether
> > > > > > > we
> > > > > > > > > > > > want to add the coordinator to the topic metadata
> > > response
> > > > or
> > > > > > > provide
> > > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > > >
> > > > > > > > > > > > There are two reasons I think we should try and avoid
> > > > adding
> > > > > > the
> > > > > > > > > > > > field:
> > > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > > - If we finally do request rerouting in Kafka then
> the
> > > > field
> > > > > > > would
> > > > > > > > > add
> > > > > > > > > > > >   little to no value. (It still helps to have a
> > separate
> > > > > > > > > > > >   ClusterMetadataRequest to query for cluster-wide
> > > > > information
> > > > > > > such
> > > > > > > > > as
> > > > > > > > > > > >   'which broker is the controller?' as Joe
> mentioned.)
> > > > > > > > > > > >
> > > > > > > > > > > > I think it would be cleaner to have an explicit
> > > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > > that you can send to any broker in order to obtain
> the
> > > > > > controller
> > > > > > > > > (and
> > > > > > > > > > > > in the future possibly other cluster-wide
> > information). I
> > > > > think
> > > > > > > the
> > > > > > > > > > > > main argument against doing this and instead adding
> it
> > to
> > > > the
> > > > > > > topic
> > > > > > > > > > > > metadata response was convenience - i.e., you don't
> > have
> > > to
> > > > > > > discover
> > > > > > > > > > > > the controller in advance. However, I don't see much
> > > actual
> > > > > > > > > > > > benefit/convenience in this and in fact think it is a
> > > > > > non-issue.
> > > > > > > Let
> > > > > > > > > > > > me know if I'm overlooking something here.
> > > > > > > > > > > >
> > > > > > > > > > > > As an example, say we need to initiate partition
> > > > reassignment
> > > > > > by
> > > > > > > > > > > > issuing the new ReassignPartitionsRequest to the
> > > controller
> > > > > > > (assume
> > > > > > > > > we
> > > > > > > > > > > > already have the desired manual partition
> assignment).
> > > If
> > > > we
> > > > > > > are to
> > > > > > > > > > > > augment topic metadata response then the flow be
> > > something
> > > > > like
> > > > > > > this
> > > > > > > > > :
> > > > > > > > > > > >
> > > > > > > > > > > > - Issue topic metadata request to any broker (and
> > > discover
> > > > > the
> > > > > > > > > > > >   controller
> > > > > > > > > > > > - Connect to controller if required (i.e., if the
> > broker
> > > > > above
> > > > > > !=
> > > > > > > > > > > >   controller)
> > > > > > > > > > > > - Issue the partition reassignment request to the
> > > > controller.
> > > > > > > > > > > >
> > > > > > > > > > > > With an explicit cluster metadata request it would
> be:
> > > > > > > > > > > > - Issue cluster metadata request to any broker
> > > > > > > > > > > > - Connect to controller if required (i.e., if the
> > broker
> > > > > above
> > > > > > !=
> > > > > > > > > > > >   controller)
> > > > > > > > > > > > - Issue the partition reassignment request
> > > > > > > > > > > >
> > > > > > > > > > > > So it seems to add little practical value and bloats
> > > topic
> > > > > > > metadata
> > > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > > >
> > > > > > > > > > > > The other angle to this is the following - is it a
> > matter
> > > > of
> > > > > > > naming?
> > > > > > > > > > > > Should we just rename topic metadata request/response
> > to
> > > > just
> > > > > > > > > > > > MetadataRequest/Response and add cluster metadata to
> > it?
> > > By
> > > > > > that
> > > > > > > same
> > > > > > > > > > > > token should we also allow querying for the consumer
> > > > > > coordinator
> > > > > > > (and
> > > > > > > > > > > > in future transaction coordinator) as well? This
> leads
> > > to a
> > > > > > > bloated
> > > > > > > > > > > > request which isn't very appealing and altogether
> > > > confusing.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Joel
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao
> > wrote:
> > > > > > > > > > > > > Andri,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1. I just realized that in order to start working
> on
> > > > > > > KAFKA-1927, we
> > > > > > > > > > > will
> > > > > > > > > > > > > need to merge the changes to OffsetCommitRequest
> > (from
> > > > > 0.8.2)
> > > > > > > to
> > > > > > > > > > trunk.
> > > > > > > > > > > > > This is planned to be done as part of KAFKA-1634.
> So,
> > > we
> > > > > will
> > > > > > > need
> > > > > > > > > > > > Guozhang
> > > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 2. Thinking about this a bit more, if the semantic
> of
> > > > those
> > > > > > > "write"
> > > > > > > > > > > > > requests is async (i.e., after the client gets a
> > > > response,
> > > > > it
> > > > > > > just
> > > > > > > > > > > means
> > > > > > > > > > > > > that the operation is initiated, but not
> necessarily
> > > > > > > completed), we
> > > > > > > > > > > don't
> > > > > > > > > > > > > really need to forward the requests to the
> > controller.
> > > > > > > Instead, the
> > > > > > > > > > > > > receiving broker can just write the operation to ZK
> > as
> > > > the
> > > > > > > admin
> > > > > > > > > > > command
> > > > > > > > > > > > > line tool previously does. This will simplify the
> > > > > > > implementation.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 8. There is another implementation detail for
> > describe
> > > > > topic.
> > > > > > > > > > Ideally,
> > > > > > > > > > > we
> > > > > > > > > > > > > want to read the topic config from the broker
> cache,
> > > > > instead
> > > > > > of
> > > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > > Currently, every broker reads the topic-level
> config
> > > for
> > > > > all
> > > > > > > > > topics.
> > > > > > > > > > > > > However, it ignores those for topics not hosted on
> > > > itself.
> > > > > > So,
> > > > > > > we
> > > > > > > > > may
> > > > > > > > > > > > need
> > > > > > > > > > > > > to change TopicConfigManager a bit so that it
> caches
> > > the
> > > > > > > configs
> > > > > > > > > for
> > > > > > > > > > > all
> > > > > > > > > > > > > topics.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Jun
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Guys,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 1. Q: Get rid of all scala requests objects, use
> > java
> > > > > > > protocol
> > > > > > > > > > > > definitions.
> > > > > > > > > > > > > >     A: Gwen kindly took that (KAFKA-1927). It's
> > > > important
> > > > > > to
> > > > > > > > > speed
> > > > > > > > > > up
> > > > > > > > > > > > > > review procedure
> > > > > > > > > > > > > >          there since this ticket blocks other
> > > important
> > > > > > > changes.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 2. Q: Generic re-reroute facility vs client
> > > maintaining
> > > > > > > cluster
> > > > > > > > > > > state.
> > > > > > > > > > > > > >     A: Jay has added pseudo code to KAFKA-1912 -
> > need
> > > > to
> > > > > > > consider
> > > > > > > > > > > > whether
> > > > > > > > > > > > > > this will be
> > > > > > > > > > > > > >         easy to implement as a server-side
> feature
> > > > > > (comments
> > > > > > > are
> > > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > > > > > > >     A: This might be useful for clients, add this
> > to
> > > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > > > > > > >     A: I will add proposed by Jun solution (using
> > > > > clientId
> > > > > > > for
> > > > > > > > > > that)
> > > > > > > > > > > > to the
> > > > > > > > > > > > > > KIP.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs grabbing all
> > > > > protocol
> > > > > > > > > changes
> > > > > > > > > > in
> > > > > > > > > > > > one
> > > > > > > > > > > > > > version.
> > > > > > > > > > > > > >     A: It was decided to try to gather all
> changes
> > to
> > > > > > > protocol
> > > > > > > > > > > (before
> > > > > > > > > > > > > > release).
> > > > > > > > > > > > > >         In case of TMR it worth checking:
> > KAFKA-2020
> > > > and
> > > > > > > KIP-13
> > > > > > > > > > > > (quotas)
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 6. Q: JSON lib is needed to deserialize user's
> > input
> > > in
> > > > > CLI
> > > > > > > tool.
> > > > > > > > > > > > > >     A: Use jackson for that, /tools project is a
> > > > separate
> > > > > > > jar so
> > > > > > > > > > > > shouldn't
> > > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs generic status
> > > check
> > > > > > > command.
> > > > > > > > > > > > > >      A: For long-running requests like reassign
> > > > > partitions
> > > > > > > > > > *progress*
> > > > > > > > > > > > check
> > > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > > >          it makes sense to introduce it.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >  Please add, correct me if I missed something.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii
> Biletskyi <
> > > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > You are right, I removed ClusterMetadata
> because
> > we
> > > > > have
> > > > > > > > > > partially
> > > > > > > > > > > > > > > what we need in TopicMetadata. Also, as Jay
> > pointed
> > > > out
> > > > > > > > > earlier,
> > > > > > > > > > we
> > > > > > > > > > > > > > > would like to have "orthogonal" API, but at the
> > > same
> > > > > time
> > > > > > > we
> > > > > > > > > need
> > > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > But I like your idea and even have some other
> > > > arguments
> > > > > > for
> > > > > > > > > this
> > > > > > > > > > > > option:
> > > > > > > > > > > > > > > There is also DescribeTopicRequest which was
> > > proposed
> > > > > in
> > > > > > > this
> > > > > > > > > > KIP,
> > > > > > > > > > > > > > > it returns topic configs, partitions,
> replication
> > > > > factor
> > > > > > > plus
> > > > > > > > > > > > partition
> > > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > > leader replica. The later part is really
> already
> > > > there
> > > > > in
> > > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > > So again we'll have to add stuff to TMR, not to
> > > > > duplicate
> > > > > > > some
> > > > > > > > > > info
> > > > > > > > > > > > in
> > > > > > > > > > > > > > > newly added requests. However, this way we'll
> end
> > > up
> > > > > with
> > > > > > > > > > "monster"
> > > > > > > > > > > > > > > request which returns cluster metadata, topic
> > > > > replication
> > > > > > > and
> > > > > > > > > > > config
> > > > > > > > > > > > info
> > > > > > > > > > > > > > > plus partition replication data. Seems logical
> to
> > > > split
> > > > > > > TMR to
> > > > > > > > > > > > > > > - ClusterMetadata (brokers + controller, maybe
> > smth
> > > > > else)
> > > > > > > > > > > > > > > - TopicMetadata (topic info + partition
> details)
> > > > > > > > > > > > > > > But since current TMR is involved in lots of
> > places
> > > > > > > (including
> > > > > > > > > > > > network
> > > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > > as I understand) this might be very serious
> > change
> > > > and
> > > > > it
> > > > > > > > > > probably
> > > > > > > > > > > > makes
> > > > > > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <
> > > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > >> I may be missing some context but hopefully
> this
> > > > will
> > > > > > > also be
> > > > > > > > > > > > covered
> > > > > > > > > > > > > > >> today: I thought the earlier proposal where
> > there
> > > > was
> > > > > an
> > > > > > > > > > explicit
> > > > > > > > > > > > > > >> ClusterMetadata request was clearer and
> > explicit.
> > > > > During
> > > > > > > the
> > > > > > > > > > > course
> > > > > > > > > > > > of
> > > > > > > > > > > > > > >> this thread I think the conclusion was that
> the
> > > main
> > > > > > need
> > > > > > > was
> > > > > > > > > > for
> > > > > > > > > > > > > > >> controller information and that can be rolled
> > into
> > > > the
> > > > > > > topic
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > >> response but that seems a bit irrelevant to
> > topic
> > > > > > > metadata.
> > > > > > > > > > FWIW I
> > > > > > > > > > > > > > >> think the full broker-list is also irrelevant
> to
> > > > topic
> > > > > > > > > metadata,
> > > > > > > > > > > but
> > > > > > > > > > > > > > >> it is already there and in use. I think there
> is
> > > > still
> > > > > > > room
> > > > > > > > > for
> > > > > > > > > > an
> > > > > > > > > > > > > > >> explicit ClusterMetadata request since there
> may
> > > be
> > > > > > other
> > > > > > > > > > > > > > >> cluster-level information that we may want to
> > add
> > > > over
> > > > > > > time
> > > > > > > > > (and
> > > > > > > > > > > > that
> > > > > > > > > > > > > > >> have nothing to do with topic metadata).
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200,
> Andrii
> > > > > > Biletskyi
> > > > > > > > > > wrote:
> > > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >> > 101. Okay, if you say that such use case is
> > > > > > important. I
> > > > > > > > > also
> > > > > > > > > > > > think
> > > > > > > > > > > > > > >> > using clientId for these purposes is fine -
> if
> > > we
> > > > > > > already
> > > > > > > > > have
> > > > > > > > > > > > this
> > > > > > > > > > > > > > >> field
> > > > > > > > > > > > > > >> > as part of all Wire protocol messages, why
> not
> > > use
> > > > > > that.
> > > > > > > > > > > > > > >> > I will update KIP-4 page if nobody has other
> > > ideas
> > > > > > > (which
> > > > > > > > > may
> > > > > > > > > > > > come up
> > > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP
> accordingly.
> > I
> > > > > think
> > > > > > > we can
> > > > > > > > > > add
> > > > > > > > > > > > new,
> > > > > > > > > > > > > > >> > fine-grained error codes if some error code
> > > > received
> > > > > > in
> > > > > > > > > > specific
> > > > > > > > > > > > case
> > > > > > > > > > > > > > >> > won't give enough context to return a
> > > descriptive
> > > > > > error
> > > > > > > > > > message
> > > > > > > > > > > > for
> > > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >> > Look forward to discussing all outstanding
> > > issues
> > > > in
> > > > > > > detail
> > > > > > > > > > > today
> > > > > > > > > > > > > > during
> > > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <
> > > > > > > jun@confluent.io
> > > > > > > > > >
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >> >
> > > > > > > > > > > > > > >> > > 101. There may be a use case where you
> only
> > > want
> > > > > the
> > > > > > > > > topics
> > > > > > > > > > to
> > > > > > > > > > > > be
> > > > > > > > > > > > > > >> created
> > > > > > > > > > > > > > >> > > manually by admins. Currently, you can do
> > that
> > > > by
> > > > > > > > > disabling
> > > > > > > > > > > auto
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > >> > > creation and issue topic creation from the
> > > > > > > TopicCommand.
> > > > > > > > > If
> > > > > > > > > > we
> > > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > > >> > > topic creation completely on the broker
> and
> > > > don't
> > > > > > > have a
> > > > > > > > > way
> > > > > > > > > > > to
> > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > >> > > between topic creation requests from the
> > > regular
> > > > > > > clients
> > > > > > > > > and
> > > > > > > > > > > the
> > > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > > >> > > can't support manual topic creation any
> > more.
> > > I
> > > > > was
> > > > > > > > > thinking
> > > > > > > > > > > > that
> > > > > > > > > > > > > > >> another
> > > > > > > > > > > > > > >> > > way of distinguishing the clients making
> the
> > > > topic
> > > > > > > > > creation
> > > > > > > > > > > > requests
> > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > >> > > using clientId. For example, the admin
> tool
> > > can
> > > > > set
> > > > > > > it to
> > > > > > > > > > > > something
> > > > > > > > > > > > > > >> like
> > > > > > > > > > > > > > >> > > admin and the broker can treat that
> clientId
> > > > > > > specially.
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > Also, there is a related discussion in
> > > > KAFKA-2020.
> > > > > > > > > > Currently,
> > > > > > > > > > > > we do
> > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > 1. If leader is not available, we set the
> > > > > partition
> > > > > > > level
> > > > > > > > > > > error
> > > > > > > > > > > > code
> > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > > >> > > 2. If a non-leader replica is not
> available,
> > > we
> > > > > take
> > > > > > > that
> > > > > > > > > > > > replica
> > > > > > > > > > > > > > out
> > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > >> > > the assigned replica list and isr in the
> > > > response.
> > > > > > As
> > > > > > > an
> > > > > > > > > > > > indication
> > > > > > > > > > > > > > >> for
> > > > > > > > > > > > > > >> > > doing that, we set the partition level
> error
> > > > code
> > > > > to
> > > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > This has a few problems. First,
> > > > > ReplicaNotAvailable
> > > > > > > > > probably
> > > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > > >> > > an error, at least for the normal
> > > > > producer/consumer
> > > > > > > > > clients
> > > > > > > > > > > that
> > > > > > > > > > > > > > just
> > > > > > > > > > > > > > >> want
> > > > > > > > > > > > > > >> > > to find out the leader. Second, it can
> > happen
> > > > that
> > > > > > > both
> > > > > > > > > the
> > > > > > > > > > > > leader
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > >> > > another replica are not available at the
> > same
> > > > > time.
> > > > > > > There
> > > > > > > > > is
> > > > > > > > > > > no
> > > > > > > > > > > > > > error
> > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > >> > > to indicate both. Third, even if a replica
> > is
> > > > not
> > > > > > > > > available,
> > > > > > > > > > > > it's
> > > > > > > > > > > > > > >> still
> > > > > > > > > > > > > > >> > > useful to return its replica id since some
> > > > clients
> > > > > > > (e.g.
> > > > > > > > > > admin
> > > > > > > > > > > > tool)
> > > > > > > > > > > > > > >> may
> > > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > One way to address this issue is to always
> > > > return
> > > > > > the
> > > > > > > > > > replica
> > > > > > > > > > > > id for
> > > > > > > > > > > > > > >> > > leader, assigned replicas, and isr
> > regardless
> > > of
> > > > > > > whether
> > > > > > > > > the
> > > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > > >> > > broker is live or not. Since we also
> return
> > > the
> > > > > list
> > > > > > > of
> > > > > > > > > live
> > > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > >> > > client can figure out whether a leader or
> a
> > > > > replica
> > > > > > is
> > > > > > > > > live
> > > > > > > > > > or
> > > > > > > > > > > > not
> > > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > > >> > > accordingly. This way, we don't need to
> set
> > > the
> > > > > > > partition
> > > > > > > > > > > level
> > > > > > > > > > > > > > error
> > > > > > > > > > > > > > >> code
> > > > > > > > > > > > > > >> > > when the leader or a replica is not
> > available.
> > > > > This
> > > > > > > > > doesn't
> > > > > > > > > > > > change
> > > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > > >> > > protocol, but does change the semantics.
> > Since
> > > > we
> > > > > > are
> > > > > > > > > > evolving
> > > > > > > > > > > > the
> > > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > > >> > > of TopicMetadataRequest here, we can
> > > potentially
> > > > > > > piggyback
> > > > > > > > > > the
> > > > > > > > > > > > > > change.
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > 102.1 For those types of errors due to
> > invalid
> > > > > > input,
> > > > > > > > > > > shouldn't
> > > > > > > > > > > > we
> > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > >> > > guard it at parameter validation time and
> > > throw
> > > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > > >> > > without even sending the request to the
> > > broker?
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii
> > > > > Biletskyi <
> > > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > > > 101. If I understand you correctly, you
> > are
> > > > > saying
> > > > > > > > > future
> > > > > > > > > > > > producer
> > > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > > >> > > > will be ported to TMR_V1) won't be able
> to
> > > > > > > automatically
> > > > > > > > > > > > create
> > > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > >> > > > unconditionally remove topic creation
> from
> > > > > there).
> > > > > > > But
> > > > > > > > > we
> > > > > > > > > > > > need to
> > > > > > > > > > > > > > >> this
> > > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > > >> > > > Ok, about your proposal: I'm not a big
> fan
> > > > too,
> > > > > > > when it
> > > > > > > > > > > comes
> > > > > > > > > > > > to
> > > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > > >> > > > clients directly in protocol schema. And
> > > also
> > > > > I'm
> > > > > > > not
> > > > > > > > > > sure I
> > > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > > >> > > > auto.create.topics.enable is a server
> side
> > > > > > > > > configuration.
> > > > > > > > > > > Can
> > > > > > > > > > > > we
> > > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > > >> > > > in future versions, add this setting to
> > > > producer
> > > > > > and
> > > > > > > > > based
> > > > > > > > > > > on
> > > > > > > > > > > > that
> > > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > > >> > > > UnknownTopic create topic explicitly by
> a
> > > > > separate
> > > > > > > > > > producer
> > > > > > > > > > > > call
> > > > > > > > > > > > > > via
> > > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because we want to
> > > > support
> > > > > > > batching
> > > > > > > > > > and
> > > > > > > > > > > > at
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > >> same
> > > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > > >> > > > want to give descriptive error messages
> > for
> > > > > > clients.
> > > > > > > > > Since
> > > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > > >> > > > to construct such messages (e.g.
> > AdminClient
> > > > > layer
> > > > > > > can
> > > > > > > > > > know
> > > > > > > > > > > > that
> > > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > > >> > > > means two cases: either invalid number -
> > > e.g.
> > > > > -1;
> > > > > > or
> > > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > > >> > > > partitions argument wasn't) - I wrapped
> > > > > responses
> > > > > > in
> > > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > > >> > > > other ideas, this was just initial
> > version.
> > > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change that to
> > > > > probably
> > > > > > > some
> > > > > > > > > > other
> > > > > > > > > > > > dto.
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun
> Rao <
> > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > >> > > > > 101. That's what I was thinking too,
> but
> > > it
> > > > > may
> > > > > > > not be
> > > > > > > > > > > that
> > > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > > >> > > > > we can let it not trigger auto topic
> > > > creation.
> > > > > > > Then,
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > > >> > > > > if it gets an UnknownTopicException,
> it
> > > can
> > > > > > > explicitly
> > > > > > > > > > > > issue a
> > > > > > > > > > > > > > >> > > > > createTopicRequest for auto topic
> > > creation.
> > > > On
> > > > > > the
> > > > > > > > > > > consumer
> > > > > > > > > > > > > > side,
> > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > > >> > > > > never issue createTopicRequest. This
> > works
> > > > > when
> > > > > > > auto
> > > > > > > > > > topic
> > > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > > >> > > > > enabled on the broker side. However, I
> > am
> > > > not
> > > > > > > sure how
> > > > > > > > > > > > things
> > > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > > >> > > > > when auto topic creation is disabled
> on
> > > the
> > > > > > broker
> > > > > > > > > side.
> > > > > > > > > > > In
> > > > > > > > > > > > this
> > > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > >> > > > > want to have a way to manually create
> a
> > > > topic,
> > > > > > > > > > potentially
> > > > > > > > > > > > > > through
> > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > >> > > > > commands. However, then we need a way
> to
> > > > > > > distinguish
> > > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > > >> > > > > issued from the producer clients and
> the
> > > > admin
> > > > > > > tools.
> > > > > > > > > > May
> > > > > > > > > > > > be we
> > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > >> > > > > new field in createTopicRequest and
> set
> > it
> > > > > > > differently
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > > >> > > > > client and the admin client. However,
> I
> > am
> > > > not
> > > > > > > sure if
> > > > > > > > > > > > that's
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > >> best
> > > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > >> > > > > 2. Yes, refactoring existing requests
> > is a
> > > > > > > non-trivial
> > > > > > > > > > > > amount of
> > > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > > >> > > > > posted some comments in KAFKA-1927. We
> > > will
> > > > > > > probably
> > > > > > > > > > have
> > > > > > > > > > > > to fix
> > > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > > >> > > > > first, before adding the new logic in
> > > > > > KAFKA-1694.
> > > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird that we return
> > > > > exception
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > api. It
> > > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > >> > > > > we should either return error code or
> > > throw
> > > > an
> > > > > > > > > exception
> > > > > > > > > > > > when
> > > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > > >> > > > > 102.2. We probably shouldn't
> explicitly
> > > use
> > > > > the
> > > > > > > > > request
> > > > > > > > > > > > object
> > > > > > > > > > > > > > in
> > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > > >> > > > > Not every request evolution requires
> an
> > > api
> > > > > > > change.
> > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM,
> Andrii
> > > > > > Biletskyi
> > > > > > > <
> > > > > > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > > Thanks for you comments. Answers
> > inline:
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > > 100. There are a few fields such as
> > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> > > > > > represented
> > > > > > > as a
> > > > > > > > > > > > string,
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > >> > > > > > > composite structures in json.
> Could
> > we
> > > > > > flatten
> > > > > > > > > them
> > > > > > > > > > > out
> > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > arrays/records?
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > > Yes, now with Admin Client this
> looks
> > a
> > > > bit
> > > > > > > weird.
> > > > > > > > > My
> > > > > > > > > > > > initial
> > > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > > >> > > > > > ReassignPartitionCommand accepts
> input
> > > in
> > > > > > json,
> > > > > > > we
> > > > > > > > > > want
> > > > > > > > > > > to
> > > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > > >> > > > > > interfaces unchanged, where
> possible.
> > > > > > > > > > > > > > >> > > > > > If we port it to deserialized
> format,
> > in
> > > > CLI
> > > > > > > (/tools
> > > > > > > > > > > > project)
> > > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > > >> > > > > > json library since /tools is written
> > in
> > > > java
> > > > > > and
> > > > > > > > > we'll
> > > > > > > > > > > > need to
> > > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > > >> > > > > > provided by a user. Can we quickly
> > agree
> > > > on
> > > > > > what
> > > > > > > > > this
> > > > > > > > > > > > library
> > > > > > > > > > > > > > >> should
> > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > > 101. Does TopicMetadataRequest v1
> > still
> > > > > > trigger
> > > > > > > auto
> > > > > > > > > > > topic
> > > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > > >> > > > > > > will be a bit weird now that we
> > have a
> > > > > > > separate
> > > > > > > > > > topic
> > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > > > > createTopicRequest
> > > > > > > > > and
> > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > producer/consumer
> > > > > > > client,
> > > > > > > > > in
> > > > > > > > > > > > addition
> > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > >> > > > > > > tools? For example, ideally, we
> > don't
> > > > want
> > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto topic
> > > creation.
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > > I agree, this strange logic should
> be
> > > > fixed.
> > > > > > > I'm not
> > > > > > > > > > > > confident
> > > > > > > > > > > > > > >> in
> > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > > >> > > > > > correct me if I'm wrong, but it
> > doesn't
> > > > look
> > > > > > > like a
> > > > > > > > > > hard
> > > > > > > > > > > > thing
> > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > > >> > > > > > leverage AdminClient for that in
> > > Producer
> > > > > and
> > > > > > > > > > > > unconditionally
> > > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > >> > > > > > creation from the
> > > TopicMetadataRequest_V1.
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > > 2. I think Jay meant getting rid of
> > > scala
> > > > > > > classes
> > > > > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when adding
> the
> > > new
> > > > > > > requests
> > > > > > > > > for
> > > > > > > > > > > the
> > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > >> > > > > > > However, the long term plan is to
> > get
> > > > rid
> > > > > of
> > > > > > > all
> > > > > > > > > > those
> > > > > > > > > > > > and
> > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > >> > > > > > > java request/response in the
> client.
> > > > Since
> > > > > > > this
> > > > > > > > > KIP
> > > > > > > > > > > > proposes
> > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > >> > > > > > > significant number of new
> requests,
> > > > > perhaps
> > > > > > we
> > > > > > > > > > should
> > > > > > > > > > > > bite
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > >> > > > > > > clean up the existing scala
> requests
> > > > first
> > > > > > > before
> > > > > > > > > > > > adding new
> > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > > Yes, looks like I misunderstood the
> > > point
> > > > of
> > > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > > >> > > > > > rework that. The only thing is that
> I
> > > > don't
> > > > > > see
> > > > > > > any
> > > > > > > > > > > > example
> > > > > > > > > > > > > > how
> > > > > > > > > > > > > > >> it
> > > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > > >> > > > > > least one existing protocol message.
> > > Thus,
> > > > > as
> > > > > > I
> > > > > > > > > > > > understand, I
> > > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > > >> > > > > > Re porting all existing RQ/RP in
> this
> > > > patch.
> > > > > > > Sounds
> > > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > > >> but
> > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > > >> > > > > > requirement to have Admin KIP done,
> > I'm
> > > > > afraid
> > > > > > > this
> > > > > > > > > > can
> > > > > > > > > > > > be a
> > > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > > >> > > > > > There are 13 protocol messages and
> all
> > > > that
> > > > > > > would
> > > > > > > > > > > require
> > > > > > > > > > > > not
> > > > > > > > > > > > > > >> only
> > > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > > >> > > > > > intensive manual testing, no? I'm
> > afraid
> > > > I'm
> > > > > > > not the
> > > > > > > > > > > > right guy
> > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > > >> > > > > > Kafka core internals :). Let me know
> > > your
> > > > > > > thoughts
> > > > > > > > > on
> > > > > > > > > > > this
> > > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > > >> > >
> > > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun
> > > Rao <
> > > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > > 100. There are a few fields such
> as
> > > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> > > > > > represented
> > > > > > > as a
> > > > > > > > > > > > string,
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > > >> > > > > > > composite structures in json.
> Could
> > we
> > > > > > flatten
> > > > > > > > > them
> > > > > > > > > > > out
> > > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > >> > > > > > > protocol definition as
> > arrays/records?
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > > 101. Does TopicMetadataRequest v1
> > > still
> > > > > > > trigger
> > > > > > > > > auto
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > > >> > > > > > > will be a bit weird now that we
> > have a
> > > > > > > separate
> > > > > > > > > > topic
> > > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > > > > createTopicRequest
> > > > > > > > > and
> > > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > > producer/consumer
> > > > > > > client,
> > > > > > > > > in
> > > > > > > > > > > > addition
> > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > > >> > > > > > > tools? For example, ideally, we
> > don't
> > > > want
> > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > >> > > > > > > consumer to trigger auto topic
> > > creation.
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant getting rid
> of
> > > > scala
> > > > > > > classes
> > > > > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when adding
> the
> > > new
> > > > > > > requests
> > > > > > > > > for
> > > > > > > > > > > the
> > > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > > >> > > > > > > However, the long term plan is to
> > get
> > > > rid
> > > > > of
> > > > > > > all
> > > > > > > > > > those
> > > > > > > > > > > > and
> > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > >> > > > > > > java request/response in the
> client.
> > > > Since
> > > > > > > this
> > > > > > > > > KIP
> > > > > > > > > > > > proposes
> > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > > >> > > > > > > significant number of new
> requests,
> > > > > perhaps
> > > > > > we
> > > > > > > > > > should
> > > > > > > > > > > > bite
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > >> > > > > > > clean up the existing scala
> requests
> > > > first
> > > > > > > before
> > > > > > > > > > > > adding new
> > > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM,
> > > Andrii
> > > > > > > Biletskyi
> > > > > > > > > <
> > > > > > > > > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly>
> wrote:
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > As said above - I list again all
> > > > > comments
> > > > > > > from
> > > > > > > > > > this
> > > > > > > > > > > > thread
> > > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > > >> > > > > > > > can see what's left and finalize
> > all
> > > > > > pending
> > > > > > > > > > issues.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > > >> > > > > > > > 1. This is much needed
> > > functionality,
> > > > > but
> > > > > > > there
> > > > > > > > > > are
> > > > > > > > > > > a
> > > > > > > > > > > > lot
> > > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > > >> > > > > > > > really think these protocols
> > > through.
> > > > We
> > > > > > > really
> > > > > > > > > > want
> > > > > > > > > > > > to
> > > > > > > > > > > > > > end
> > > > > > > > > > > > > > >> up
> > > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > > >> > > > > > > > of well thought-out, orthoganol
> > > apis.
> > > > > For
> > > > > > > this
> > > > > > > > > > > reason
> > > > > > > > > > > > I
> > > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > > >> > > > > > > > important to think through the
> end
> > > > state
> > > > > > > even if
> > > > > > > > > > > that
> > > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > >> > > > > > > > won't implement in the first
> > phase.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: Definitely behind this. Would
> > > > > > appreciate
> > > > > > > if
> > > > > > > > > > there
> > > > > > > > > > > > are
> > > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 2. Let's please please please
> wait
> > > > until
> > > > > > we
> > > > > > > have
> > > > > > > > > > > > switched
> > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > > >> > > > > > > > to the new java protocol
> > > definitions.
> > > > If
> > > > > > we
> > > > > > > add
> > > > > > > > > > > upteen
> > > > > > > > > > > > > > more
> > > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > > >> > > > > > > > objects that is just generating
> > more
> > > > > work
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > > removed
> > > > > > scala
> > > > > > > > > > > protocol
> > > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 3. This proposal introduces a
> new
> > > type
> > > > > of
> > > > > > > > > optional
> > > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > > >> > > > > > > > inconsistent with everything
> else
> > in
> > > > the
> > > > > > > > > protocol
> > > > > > > > > > > > where we
> > > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > > >> > > > > > > > other marker value. You could
> > argue
> > > > > either
> > > > > > > way
> > > > > > > > > but
> > > > > > > > > > > > let's
> > > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > >> > > > > > > > for consistency. For clients
> that
> > > > > > > implemented
> > > > > > > > > the
> > > > > > > > > > > > protocol
> > > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > > >> > > > > > > > than our scala code these basic
> > > > > primitives
> > > > > > > are
> > > > > > > > > > hard
> > > > > > > > > > > to
> > > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > > removed
> > > > > > > MaybeOf
> > > > > > > > > > type
> > > > > > > > > > > > and
> > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This seems
> to
> > > > > > duplicate
> > > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > >> > > > > > > > brokers, topics, and
> partitions. I
> > > > think
> > > > > > we
> > > > > > > > > should
> > > > > > > > > > > > rename
> > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > > >> > > > > > > > ClusterMetadataRequest (or just
> > > > > > > MetadataRequest)
> > > > > > > > > > and
> > > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > >> > > > > > > > controller. Or are there other
> > > things
> > > > we
> > > > > > > could
> > > > > > > > > add
> > > > > > > > > > > > here?
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: I agree. Updated the KIP.
> Let's
> > > > > extends
> > > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 5. We have a tendency to try to
> > > make a
> > > > > lot
> > > > > > > of
> > > > > > > > > > > requests
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > > >> > > > > > > > particular nodes. This adds a
> lot
> > of
> > > > > > burden
> > > > > > > for
> > > > > > > > > > > client
> > > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > > >> > > > > > > > sounds easy but each discovery
> can
> > > > fail
> > > > > in
> > > > > > > many
> > > > > > > > > > > parts
> > > > > > > > > > > > so
> > > > > > > > > > > > > > it
> > > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > > >> > > > > > > > full state machine to do
> right). I
> > > > think
> > > > > > we
> > > > > > > > > should
> > > > > > > > > > > > > > consider
> > > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > > >> > > > > > > > commands and ideally as many of
> > the
> > > > > other
> > > > > > > apis
> > > > > > > > > as
> > > > > > > > > > > > possible
> > > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > > >> > > > > > > > brokers and just redirect to the
> > > > > > controller
> > > > > > > on
> > > > > > > > > the
> > > > > > > > > > > > broker
> > > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > > >> > > > > > > > there would be a general way to
> > > > > > encapsulate
> > > > > > > this
> > > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: It's a very interesting idea,
> > but
> > > > > seems
> > > > > > > there
> > > > > > > > > > are
> > > > > > > > > > > > some
> > > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > > >> > > > > > > > feature (like performance
> > > > > considerations,
> > > > > > > how
> > > > > > > > > this
> > > > > > > > > > > > will
> > > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > > >> > > > > > > > I believe this shouldn't be a
> > > blocker.
> > > > > If
> > > > > > > this
> > > > > > > > > > > > feature is
> > > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > > >> > > > > > > > point it won't affect Admin
> > changes
> > > -
> > > > at
> > > > > > > least
> > > > > > > > > no
> > > > > > > > > > > > changes
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 6. We should probably normalize
> > the
> > > > key
> > > > > > > value
> > > > > > > > > > pairs
> > > > > > > > > > > > used
> > > > > > > > > > > > > > for
> > > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > >> > > > > > > > than embedding a new formatting.
> > So
> > > > two
> > > > > > > strings
> > > > > > > > > > > rather
> > > > > > > > > > > > > > than
> > > > > > > > > > > > > > >> one
> > > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > > > > normalized
> > > > > > > > > configs
> > > > > > > > > > > and
> > > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 7. Is the postcondition of these
> > > APIs
> > > > > that
> > > > > > > the
> > > > > > > > > > > > command has
> > > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > > >> > > > > > > > the command has been completed?
> It
> > > is
> > > > a
> > > > > > lot
> > > > > > > more
> > > > > > > > > > > > usable if
> > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > > >> > > > > > > > been completed so you know that
> if
> > > you
> > > > > > > create a
> > > > > > > > > > > topic
> > > > > > > > > > > > and
> > > > > > > > > > > > > > >> then
> > > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > >> > > > > > > > it you won't get an exception
> > about
> > > > > there
> > > > > > > being
> > > > > > > > > no
> > > > > > > > > > > > such
> > > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: For long running requests
> (like
> > > > > > reassign
> > > > > > > > > > > > partitions) -
> > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > > >> > > > > > > > command has begun - so we don't
> > > block
> > > > > the
> > > > > > > > > client.
> > > > > > > > > > In
> > > > > > > > > > > > case
> > > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > > >> > > > > > > > topic commands, this will be
> > > > refactored
> > > > > > and
> > > > > > > > > topic
> > > > > > > > > > > > commands
> > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > > >> > > > > > > > immediately, since the
> Controller
> > > will
> > > > > > serve
> > > > > > > > > Admin
> > > > > > > > > > > > > > requests
> > > > > > > > > > > > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 8. Describe topic and list
> topics
> > > > > > duplicate
> > > > > > > a
> > > > > > > > > lot
> > > > > > > > > > of
> > > > > > > > > > > > stuff
> > > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > > >> > > > > > > > request. Is there a reason to
> give
> > > > back
> > > > > > > topics
> > > > > > > > > > > marked
> > > > > > > > > > > > for
> > > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > > >> > > > > > > > like if we just make the
> > > > post-condition
> > > > > of
> > > > > > > the
> > > > > > > > > > > delete
> > > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > >> > > > > > > > topic is deleted that will get
> rid
> > > of
> > > > > the
> > > > > > > need
> > > > > > > > > for
> > > > > > > > > > > > this
> > > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > > removed
> > > > > > > topics
> > > > > > > > > > marked
> > > > > > > > > > > > for
> > > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 9. Should we consider batching
> > these
> > > > > > > requests?
> > > > > > > > > We
> > > > > > > > > > > have
> > > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > >> > > > > > > > allow multiple operations to be
> > > > batched.
> > > > > > My
> > > > > > > > > > > suspicion
> > > > > > > > > > > > is
> > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > >> > > > > > > > we will get a lot of code that
> > does
> > > > > > > something
> > > > > > > > > like
> > > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > > adminClient.listTopics())
> > > > > > > > > > > > > > >> > > > > > > >
> > >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > > >> > > > > > > > this code will work great when
> you
> > > > test
> > > > > > on 5
> > > > > > > > > > topics
> > > > > > > > > > > > but
> > > > > > > > > > > > > > not
> > > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please
> check
> > > > "Topic
> > > > > > > Admin
> > > > > > > > > > > Schema"
> > > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 10. I think we should also
> discuss
> > > how
> > > > > we
> > > > > > > want
> > > > > > > > > to
> > > > > > > > > > > > expose a
> > > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > > >> > > > > > > > client api for these operations.
> > > > > Currently
> > > > > > > > > people
> > > > > > > > > > > > rely on
> > > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > > >> > > > > > > > is totally sketchy. I think we
> > > > probably
> > > > > > need
> > > > > > > > > > another
> > > > > > > > > > > > > > client
> > > > > > > > > > > > > > >> under
> > > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > > >> > > > > > > > that exposes administrative
> > > > > functionality.
> > > > > > > We
> > > > > > > > > will
> > > > > > > > > > > > need
> > > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > >> > > > > > > > properly test the new apis, I
> > > suspect.
> > > > > We
> > > > > > > should
> > > > > > > > > > > > figure
> > > > > > > > > > > > > > out
> > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please
> check
> > > > "Admin
> > > > > > > Client"
> > > > > > > > > > > > section
> > > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 11. The other information that
> > would
> > > > be
> > > > > > > really
> > > > > > > > > > > useful
> > > > > > > > > > > > to
> > > > > > > > > > > > > > get
> > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > >> > > > > > > > information about
> partitions--how
> > > much
> > > > > > data
> > > > > > > is
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > > >> > > > > > > > the segment offsets, what is the
> > > > log-end
> > > > > > > offset
> > > > > > > > > > > (i.e.
> > > > > > > > > > > > last
> > > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > > >> > > > > > > > the compaction point, etc. I
> think
> > > > that
> > > > > > done
> > > > > > > > > right
> > > > > > > > > > > > this
> > > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > >> > > > > > > > successor to the very awkward
> > > > > > OffsetRequest
> > > > > > > we
> > > > > > > > > > have
> > > > > > > > > > > > today.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > > ConsumerGroupOffsetsRequest
> > > > > > in
> > > > > > > the
> > > > > > > > > > > latest
> > > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > > >> > > > > > > > be resolved in a separate KIP /
> > jira
> > > > > > ticket.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 12. Generally we can do good
> error
> > > > > > handling
> > > > > > > > > > without
> > > > > > > > > > > > > > needing
> > > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > > >> > > > > > > > messages. I.e. generally the
> > client
> > > > has
> > > > > > the
> > > > > > > > > > context
> > > > > > > > > > > to
> > > > > > > > > > > > > > know
> > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > > >> > > > > > > > an error that the topic doesn't
> > > exist
> > > > to
> > > > > > say
> > > > > > > > > > "Topic
> > > > > > > > > > > X
> > > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > > >> > > > > > > > than "error code 14" (or
> > whatever).
> > > > > Maybe
> > > > > > > there
> > > > > > > > > > are
> > > > > > > > > > > > > > specific
> > > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > > >> > > > > > > > this is hard? If we want to add
> > > > > > server-side
> > > > > > > > > error
> > > > > > > > > > > > messages
> > > > > > > > > > > > > > >> we
> > > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > > >> > > > > > > > need to do this in a consistent
> > way
> > > > > across
> > > > > > > the
> > > > > > > > > > > > protocol.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please
> check
> > > > > > "Protocol
> > > > > > > > > > Errors"
> > > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > >> > > > > > > > comprehensive, fine-grained list
> > of
> > > > > error
> > > > > > > codes.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > > > > > > >> > > > > > > > 13. Describe topic request: it
> > would
> > > > be
> > > > > > > great to
> > > > > > > > > > go
> > > > > > > > > > > > beyond
> > > > > > > > > > > > > > >> just
> > > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > >> > > > > > > > topic name regex for this
> request.
> > > For
> > > > > > > example,
> > > > > > > > > a
> > > > > > > > > > > very
> > > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > > >> > > > > > > > the topic command is to list all
> > > > topics
> > > > > > > whose
> > > > > > > > > > config
> > > > > > > > > > > > A's
> > > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > > >> > > > > > > > topic name regex then we have to
> > > first
> > > > > > > retrieve
> > > > > > > > > > > > __all__
> > > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > > >> > > > > > > > description info and then filter
> > at
> > > > the
> > > > > > > client
> > > > > > > > > > end,
> > > > > > > > > > > > which
> > > > > > > > > > > > > > >> will
> > > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in create topic:
> > > this
> > > > is
> > > > > > > related
> > > > > > > > > > to
> > > > > > > > > > > > the
> > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > > >> > > > > > > > maybe we can add another
> metadata
> > > K-V
> > > > or
> > > > > > > just a
> > > > > > > > > > > > metadata
> > > > > > > > > > > > > > >> string
> > > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > > >> > > > > > > > with config K-V in create topic
> > like
> > > > we
> > > > > > did
> > > > > > > for
> > > > > > > > > > > offset
> > > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > > >> > > > > > > > field can be quite useful in
> > storing
> > > > > > > information
> > > > > > > > > > > like
> > > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > > >> > > > > > > > who issue the create command,
> etc,
> > > > which
> > > > > > is
> > > > > > > > > quite
> > > > > > > > > > > > > > important
> > > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > > >> > > > > > > > multi-tenant setting. Then in
> the
> > > > > describe
> > > > > > > topic
> > > > > > > > > > > > request
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > > >> > > > > > > > on regex of the metadata field.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: As discussed it is very
> > > interesting
> > > > > but
> > > > > > > can
> > > > > > > > > be
> > > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > > >> > > > > > > > we have some basic functionality
> > > > there.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > 15. Today all the admin
> operations
> > > are
> > > > > > > async in
> > > > > > > > > > the
> > > > > > > > > > > > sense
> > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > > >> > > > > > > > return once it is written in ZK,
> > and
> > > > > that
> > > > > > > is why
> > > > > > > > > > we
> > > > > > > > > > > > need
> > > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > > >> > > > > > > > like
> > testUtil.waitForTopicCreated()
> > > /
> > > > > > verify
> > > > > > > > > > > partition
> > > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > > >> > > > > > > > request, etc. With admin
> requests
> > we
> > > > > could
> > > > > > > add a
> > > > > > > > > > > flag
> > > > > > > > > > > > to
> > > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > > >> > > > > > > > synchronous requests; when it is
> > > > turned
> > > > > > on,
> > > > > > > the
> > > > > > > > > > > > response
> > > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > > >> > > > > > > > until the request has been
> > > completed.
> > > > > And
> > > > > > > for
> > > > > > > > > > async
> > > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > > >> > > > > > > > "token" field in the response,
> and
> > > > then
> > > > > > only
> > > > > > > > > need
> > > > > > > > > > a
> > > > > > > > > > > > > > general
> > > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > > >> > > > > > > > verification request" with the
> > given
> > > > > token
> > > > > > > to
> > > > > > > > > > check
> > > > > > > > > > > > if the
> > > > > > > > > > > > > > >> async
> > > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: I see your point. My idea was
> > to
> > > > > > provide
> > > > > > > > > > specific
> > > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > > >> > > > > > > > long running request, where
> > needed.
> > > We
> > > > > can
> > > > > > > do it
> > > > > > > > > > the
> > > > > > > > > > > > way
> > > > > > > > > > > > > > you
> > > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > > >> > > > > > > > concern is that introducing a
> > token
> > > we
> > > > > > again
> > > > > > > > > will
> > > > > > > > > > > make
> > > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > > >> > > > > > > > to do similar thing introducing
> > > single
> > > > > > > > > > AdminRequest
> > > > > > > > > > > > for
> > > > > > > > > > > > > > all
> > > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > > >> > > > > > > > this idea because we wanted to
> > have
> > > > > schema
> > > > > > > > > > defined.
> > > > > > > > > > > So
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > >> is
> > > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > > >> > > > > > > > a) have fixed schema but
> introduce
> > > > each
> > > > > > > time new
> > > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > > > > > >> > > > > > > > b) use one request for
> > verification
> > > > but
> > > > > > > > > generalize
> > > > > > > > > > > it
> > > > > > > > > > > > with
> > > > > > > > > > > > > > >> token
> > > > > > > > > > > > > > >> > > > > > > > I'm fine with whatever decision
> > > > > community
> > > > > > > come
> > > > > > > > > to.
> > > > > > > > > > > > Just
> > > > > > > > > > > > > > let
> > > > > > > > > > > > > > >> me
> > > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > > >> > > > > > > > 16. Specifically for ownership,
> I
> > > > think
> > > > > > the
> > > > > > > plan
> > > > > > > > > > is
> > > > > > > > > > > > to add
> > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > > >> > > > > > > > like you are describing ACL) via
> > an
> > > > > > external
> > > > > > > > > > system
> > > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > > >> > > > > > > > I remember KIP-11 described
> this,
> > > but
> > > > I
> > > > > > > can't
> > > > > > > > > find
> > > > > > > > > > > > the KIP
> > > > > > > > > > > > > > >> any
> > > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > A: Okay, no problem. Not sure
> > though
> > > > how
> > > > > > we
> > > > > > > are
> > > > > > > > > > > going
> > > > > > > > > > > > to
> > > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > > >> > > > > > > > will be committed first and
> > include
> > > > > > changes
> > > > > > > to
> > > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > > >> > > > > > > > Anyway, I added this note to
> "Open
> > > > > > > Questions"
> > > > > > > > > > > section
> > > > > > > > > > > > so
> > > > > > > > > > > > > > we
> > > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34
> AM,
> > > > Andrii
> > > > > > > > > > Biletskyi <
> > > > > > > > > > > > > > >> > > > > > > > andrii.biletskyi@stealth.ly>
> > wrote:
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > > Today I uploaded the patch
> that
> > > > covers
> > > > > > > some of
> > > > > > > > > > the
> > > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf optional
> type
> > > > > > > > > > > > > > >> > > > > > > > > - switched to java protocol
> > > > > definitions
> > > > > > > > > > > > > > >> > > > > > > > > - simplified messages
> > (normalized
> > > > > > configs,
> > > > > > > > > > removed
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > > I also updated the KIP-4 with
> > > > > respective
> > > > > > > > > changes
> > > > > > > > > > > and
> > > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > > >> > > > > > > > > - Batch Admin Operations ->
> > > updated
> > > > > Wire
> > > > > > > > > > Protocol
> > > > > > > > > > > > schema
> > > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > > >> > > > > > > > > - Remove ClusterMetadata ->
> > > changed
> > > > to
> > > > > > > extend
> > > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > > >> > > > > > > > > - Admin Client -> updated my
> > > initial
> > > > > > > proposal
> > > > > > > > > to
> > > > > > > > > > > > reflect
> > > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > > >> > > > > > > > > - Error codes -> proposed
> > > > fine-grained
> > > > > > > error
> > > > > > > > > > code
> > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > >> of
> > > > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > > I will also send a separate
> > email
> > > to
> > > > > > > cover all
> > > > > > > > > > > > comments
> > > > > > > > > > > > > > >> from
> > > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26
> PM,
> > > > Gwen
> > > > > > > Shapira
> > > > > > > > > <
> > > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > > >> > > > > > > > >> It actually specifies changes
> > to
> > > > the
> > > > > > > Metadata
> > > > > > > > > > > > protocol,
> > > > > > > > > > > > > > >> so
> > > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > > >> > > > > > > > >> both KIPs are consistent in
> > this
> > > > > regard
> > > > > > > will
> > > > > > > > > be
> > > > > > > > > > > > good.
> > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21
> > PM,
> > > > > Gwen
> > > > > > > > > Shapira
> > > > > > > > > > <
> > > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> > Specifically for
> ownership, I
> > > > think
> > > > > > the
> > > > > > > > > plan
> > > > > > > > > > is
> > > > > > > > > > > > to
> > > > > > > > > > > > > > add
> > > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > > >> > > > > > > > >> > like you are describing
> ACL)
> > > via
> > > > an
> > > > > > > > > external
> > > > > > > > > > > > system
> > > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > > >> > > > > > > > >> > I remember KIP-11 described
> > > this,
> > > > > > but I
> > > > > > > > > can't
> > > > > > > > > > > > find
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I think KIP-4
> > > focuses
> > > > > on
> > > > > > > > > getting
> > > > > > > > > > > > > > >> information
> > > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > > >> > > > > > > > >> > exists from Kafka brokers,
> > not
> > > on
> > > > > > > adding
> > > > > > > > > > > > information
> > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > > >> > > > > > > > >> > should exist but doesn't
> yet?
> > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at
> 6:37
> > > AM,
> > > > > > > Guozhang
> > > > > > > > > > Wang
> > > > > > > > > > > <
> > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > >> > > > > > > > >> >> Just want to elaborate a
> bit
> > > > more
> > > > > on
> > > > > > > the
> > > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > > >> > > > > > > > >> >> describe-topic based on
> > > config /
> > > > > > > metadata
> > > > > > > > > in
> > > > > > > > > > > my
> > > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The main
> > > > motivation
> > > > > > is
> > > > > > > to
> > > > > > > > > > have
> > > > > > > > > > > > some
> > > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms, which I think
> is
> > > > quite
> > > > > > > > > important
> > > > > > > > > > > in
> > > > > > > > > > > > a
> > > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > > >> > > > > > > > >> >> architecture: today anyone
> > can
> > > > > > create
> > > > > > > > > topics
> > > > > > > > > > > in
> > > > > > > > > > > > a
> > > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > > >> > > > > > > > >> >> there is no concept or
> > > > "ownership"
> > > > > > of
> > > > > > > > > topics
> > > > > > > > > > > > that
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > > >> > > > > > > > >> >> users. For example, at
> > > LinkedIn
> > > > we
> > > > > > > > > basically
> > > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > > >> > > > > > > > >> >> some casual topic name
> > prefix,
> > > > > which
> > > > > > > is a
> > > > > > > > > > bit
> > > > > > > > > > > > > > awkward
> > > > > > > > > > > > > > >> and
> > > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > > >> > > > > > > > >> >> we scale our customers. It
> > > would
> > > > > be
> > > > > > > great
> > > > > > > > > to
> > > > > > > > > > > use
> > > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics that
> is
> > > > > created
> > > > > > > by me.
> > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics whose
> > > > > retention
> > > > > > > time
> > > > > > > > > is
> > > > > > > > > > > > > > overriden
> > > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics whose
> > > > writable
> > > > > > > group
> > > > > > > > > > > include
> > > > > > > > > > > > > > user
> > > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > >> > > > > > > > >> >> One possible way to
> achieve
> > > this
> > > > > is
> > > > > > to
> > > > > > > > > add a
> > > > > > > > > > > > > > metadata
> > > > > > > > > > > > > > >> file
> > > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > > >> > > > > > > > >> >> create-topic request,
> whose
> > > > value
> > > > > > will
> > > > > > > > > also
> > > > > > > > > > be
> > > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > > >> > > > > > > > >> >> topic; then
> describe-topics
> > > can
> > > > > > > choose to
> > > > > > > > > > > batch
> > > > > > > > > > > > > > topics
> > > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2) config K-V
> > matching,
> > > > 3)
> > > > > > > metadata
> > > > > > > > > > > > regex,
> > > > > > > > > > > > > > etc.
> > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at
> 4:37
> > > PM,
> > > > > > > Guozhang
> > > > > > > > > > Wang
> > > > > > > > > > > <
> > > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for the updated
> > wiki.
> > > A
> > > > > few
> > > > > > > > > comments
> > > > > > > > > > > > below:
> > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error description in
> > > > > response: I
> > > > > > > think
> > > > > > > > > > if
> > > > > > > > > > > > some
> > > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > > >> > > > > > > > >> >>> several different error
> > cases
> > > > > then
> > > > > > we
> > > > > > > > > > should
> > > > > > > > > > > > really
> > > > > > > > > > > > > > >> change
> > > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In general the
> > > errorCode
> > > > > > > itself
> > > > > > > > > > would
> > > > > > > > > > > be
> > > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > > >> > > > > > > > >> >>> describing the server
> side
> > > > > errors.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic
> request:
> > it
> > > > > would
> > > > > > > be
> > > > > > > > > > great
> > > > > > > > > > > > to go
> > > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex for this
> > > > > request.
> > > > > > > For
> > > > > > > > > > > > example, a
> > > > > > > > > > > > > > >> very
> > > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > > >> > > > > > > > >> >>> the topic command is to
> > list
> > > > all
> > > > > > > topics
> > > > > > > > > > whose
> > > > > > > > > > > > > > config
> > > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex then we
> > have
> > > > to
> > > > > > > first
> > > > > > > > > > > retrieve
> > > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > > >> > > > > > > > >> >>> description info and then
> > > > filter
> > > > > at
> > > > > > > the
> > > > > > > > > > > client
> > > > > > > > > > > > end,
> > > > > > > > > > > > > > >> which
> > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in create
> > > topic:
> > > > > > this
> > > > > > > is
> > > > > > > > > > > > related to
> > > > > > > > > > > > > > >> the
> > > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can add another
> > > > metadata
> > > > > > > K-V or
> > > > > > > > > > > just a
> > > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > > >> > > > > > > > >> >>> with config K-V in create
> > > topic
> > > > > > like
> > > > > > > we
> > > > > > > > > did
> > > > > > > > > > > for
> > > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > > >> > > > > > > > >> >>> field can be quite useful
> > in
> > > > > > storing
> > > > > > > > > > > > information
> > > > > > > > > > > > > > like
> > > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > > >> > > > > > > > >> >>> who issue the create
> > command,
> > > > > etc,
> > > > > > > which
> > > > > > > > > is
> > > > > > > > > > > > quite
> > > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant setting.
> Then
> > in
> > > > the
> > > > > > > > > describe
> > > > > > > > > > > > topic
> > > > > > > > > > > > > > >> request
> > > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of the metadata
> > > field.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all the admin
> > > > operations
> > > > > > are
> > > > > > > > > async
> > > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > > >> > > > > > > > >> >>> return once it is written
> > in
> > > > ZK,
> > > > > > and
> > > > > > > that
> > > > > > > > > > is
> > > > > > > > > > > > why we
> > > > > > > > > > > > > > >> need
> > > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > > verify
> > > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > > >> > > > > > > > >> >>> request, etc. With admin
> > > > requests
> > > > > > we
> > > > > > > > > could
> > > > > > > > > > > add
> > > > > > > > > > > > a
> > > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous requests;
> when
> > it
> > > > is
> > > > > > > turned
> > > > > > > > > on,
> > > > > > > > > > > the
> > > > > > > > > > > > > > >> response
> > > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > > >> > > > > > > > >> >>> until the request has
> been
> > > > > > > completed. And
> > > > > > > > > > for
> > > > > > > > > > > > async
> > > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > > >> > > > > > > > >> >>> "token" field in the
> > > response,
> > > > > and
> > > > > > > then
> > > > > > > > > > only
> > > > > > > > > > > > need a
> > > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > > >> > > > > > > > >> >>> verification request"
> with
> > > the
> > > > > > given
> > > > > > > > > token
> > > > > > > > > > to
> > > > > > > > > > > > check
> > > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for extending
> > Metadata
> > > > > > request
> > > > > > > to
> > > > > > > > > > > include
> > > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > > >> > > > > > > > >> >>> information, and then we
> > can
> > > > > remove
> > > > > > > the
> > > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at
> > 10:23
> > > > AM,
> > > > > > Joel
> > > > > > > > > > Koshy <
> > > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for sending that
> > out
> > > > Joe
> > > > > -
> > > > > > I
> > > > > > > > > don't
> > > > > > > > > > > > think I
> > > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so if notes
> can
> > be
> > > > > sent
> > > > > > > out
> > > > > > > > > > > > afterward
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at
> > > > > 09:16:13AM
> > > > > > > > > -0800,
> > > > > > > > > > > Gwen
> > > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for sending
> this
> > > out
> > > > > Joe.
> > > > > > > > > Looking
> > > > > > > > > > > > forward
> > > > > > > > > > > > > > >> to
> > > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at
> > > 6:46
> > > > > AM,
> > > > > > > Joe
> > > > > > > > > > Stein
> > > > > > > > > > > <
> > > > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just sent
> out a
> > > > > google
> > > > > > > > > hangout
> > > > > > > > > > > > invite
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > >> all
> > > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I found
> > working
> > > > on
> > > > > a
> > > > > > > KIP.
> > > > > > > > > If
> > > > > > > > > > I
> > > > > > > > > > > > missed
> > > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me know and can
> > > update
> > > > > it,
> > > > > > > np.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We should do this
> > every
> > > > > > Tuesday
> > > > > > > @
> > > > > > > > > 2pm
> > > > > > > > > > > > Eastern
> > > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to make a
> google
> > > > > account
> > > > > > > so we
> > > > > > > > > > can
> > > > > > > > > > > > manage
> > > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > > >>
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in progress and
> > related
> > > > JIRA
> > > > > > > that
> > > > > > > > > are
> > > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015
> > at
> > > > 2:59
> > > > > > > PM, Jay
> > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on
> Google
> > > > > hangouts
> > > > > > > that
> > > > > > > > > > will
> > > > > > > > > > > > also
> > > > > > > > > > > > > > >> record
> > > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> available on
> youtube.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24,
> 2015
> > at
> > > > > 11:49
> > > > > > > AM,
> > > > > > > > > > Jeff
> > > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're happy to
> send
> > > > out a
> > > > > > > Webex
> > > > > > > > > for
> > > > > > > > > > > > this
> > > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions if there
> > is
> > > > > > > interest and
> > > > > > > > > > > > publish
> > > > > > > > > > > > > > >> them
> > > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24,
> > 2015
> > > at
> > > > > > > 11:28 AM,
> > > > > > > > > > Jay
> > > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try to
> get
> > > the
> > > > > > > technical
> > > > > > > > > > > > hang-ups
> > > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there is some
> > > benefit
> > > > > to
> > > > > > > live
> > > > > > > > > > > > discussion
> > > > > > > > > > > > > > vs
> > > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post
> > > instructions
> > > > > and
> > > > > > > give
> > > > > > > > > > > > ourselves a
> > > > > > > > > > > > > > >> few
> > > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that
> > > time
> > > > > > would
> > > > > > > work
> > > > > > > > > > for
> > > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24,
> > > 2015
> > > > at
> > > > > > > 8:18
> > > > > > > > > AM,
> > > > > > > > > > > Joe
> > > > > > > > > > > > > > Stein
> > > > > > > > > > > > > > >> <
> > > > > > > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly would
> be
> > > > great
> > > > > > > maybe
> > > > > > > > > > like
> > > > > > > > > > > > every
> > > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't mind
> > > google
> > > > > > > hangout
> > > > > > > > > but
> > > > > > > > > > > > there
> > > > > > > > > > > > > > is
> > > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we know the
> > > apache
> > > > > irc
> > > > > > > > > channel
> > > > > > > > > > > > works.
> > > > > > > > > > > > > > We
> > > > > > > > > > > > > > >> can
> > > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes? We can
> > pull
> > > > > > > transcripts
> > > > > > > > > > too
> > > > > > > > > > > > and
> > > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it helpful
> for
> > > > > things.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb
> 24,
> > > > 2015
> > > > > at
> > > > > > > 11:10
> > > > > > > > > > AM,
> > > > > > > > > > > > Jay
> > > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked
> > > about
> > > > > > > doing a
> > > > > > > > > > > Google
> > > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> generalizing
> > > > that a
> > > > > > > little
> > > > > > > > > > > > > > further...I
> > > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > everyone
> > > > spending a
> > > > > > > > > > reasonable
> > > > > > > > > > > > chunk
> > > > > > > > > > > > > > of
> > > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync up
> once
> > a
> > > > > week.
> > > > > > I
> > > > > > > > > think
> > > > > > > > > > we
> > > > > > > > > > > > could
> > > > > > > > > > > > > > >> use
> > > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make
> > > sure
> > > > we
> > > > > > > are on
> > > > > > > > > > top
> > > > > > > > > > > of
> > > > > > > > > > > > > > code
> > > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > issues,
> etc.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We can make
> > it
> > > > > > publicly
> > > > > > > > > > > > available so
> > > > > > > > > > > > > > >> that
> > > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any
> interest
> > in
> > > > > doing
> > > > > > > this?
> > > > > > > > > > If
> > > > > > > > > > > so
> > > > > > > > > > > > > > I'll
> > > > > > > > > > > > > > >> try
> > > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb
> > 24,
> > > > > 2015
> > > > > > at
> > > > > > > > > 3:57
> > > > > > > > > > > AM,
> > > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > I've
> > updated
> > > > KIP
> > > > > > > page,
> > > > > > > > > > fixed
> > > > > > > > > > > /
> > > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > > >
> > > > ...
> > > >
> > > > [Message clipped]
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Hm, actually the ticket you linked, Guozhang, brings as back
to the problem what should be considered a post-condition for
each of the admin commands.
In my understanding:

1) CreateTopic - broker created /brokers/topics/<topic>
(Not the controller picked up changes from zk and broadcasted
LeaderAndIsr and UpdateMetadata)

2) AlterTopic - same as 1) - broker changed assignment data
in zookeeper or created admin path for topic config change

3) DeleteTopic - admin path /admin/delete_topics is created

4) ReassignPartitions and PreferredReplica - corresponding admin
path is created

Now what can be considered a completed operation from the client's
perspective?
1) Topic is created once corresponding data is in zk
(I remember there were some thoughts that it'd be good to consider
topic created once all replicas receive information about it and thus
clients can produce/consume from it, but as was discussed this seems
to be a hard thing to do)

2) Probably same as 1), so right after AlterTopic is issued

3) The topic has been removed from /brokers/topics

4) ReassignPartitions and PrefferedReplica were discussed earlier -
in short the former is completed once partition state info in zk matches
reassignment request and admin path is empty, the latter - once data
in zk shows that head of assignned replicas of the partition and leader
is the same replica

Thoughts?

Thanks,
Andrii Biletskyi

On Fri, Mar 20, 2015 at 7:10 PM, Guozhang Wang <wa...@gmail.com> wrote:

> I think while loop is fine for supporting blocking, just that we need to
> add back off to avoid bombarding brokers with DescribeTopic requests.
>
> Also I have linked KAFKA-1125
> <https://issues.apache.org/jira/browse/KAFKA-1125> to your proposal, and
> when KAFKA-1694 is done this ticket can also be closed.
>
> Guozhang
>
> On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Great.
> > I want to elaborate this a bit more, to see we are on the same page
> > concerning the client code.
> >
> > So with all topic commands being async a client (AdminClient in our
> > case or any other other client people would like to implement) to support
> > a blocking operation (which seems to be a natural use-case e.g. for topic
> > creation): would have to do:
> > 1. issue CreateTopicRequest
> > 2. if successful, in a "while" loop send DescribeTopicRequest and
> > break the loop once all topics are returned in response (or upon
> timeout).
> > 3. if unsuccessful throw exception
> > Would it be okay?
> >
> > Thanks,
> > Andrii Biletskyi
> >
> >
> > On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Andrii,
> > >
> > > I think you are right. It seems that only ReassignPartitions needs a
> > > separate verification request.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Guys,
> > > > I like this idea too. Let's stick with that. I'll update KIP
> > accordingly.
> > > >
> > > > I was also thinking we can avoid adding dedicated status check
> > > > requests for topic commands. - We have everything in DescribeTopic
> > > > for that! E.g.:
> > > > User issued CreateTopic - to check the status client sends
> > DescribeTopic
> > > > and checks whether is something returned for that topic. The same for
> > > > alteration, deletion.
> > > > Btw, PreferredReplica status can be also checked with
> > > DescribeTopicRequest
> > > > (head of assigned replicas list == leader).
> > > > For ReassignPartitions as discussed we'll need to have a separate
> > > Verify...
> > > > request.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > >
> > > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <wa...@gmail.com>
> > > wrote:
> > > >
> > > > > +1 on broker writing to ZK for async handling. I was thinking that
> in
> > > the
> > > > > end state the admin requests would be eventually sent to controller
> > > > either
> > > > > through re-routing or clients discovering them, instead of letting
> > > > > controller listen on ZK admin path. But thinking about it a second
> > > time,
> > > > I
> > > > > think it is actually simpler to let controller manage
> > > > > incoming queued-up admin requests through ZK.
> > > > >
> > > > > Guozhang
> > > > >
> > > > >
> > > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <jj...@gmail.com>
> > > wrote:
> > > > >
> > > > > > +1 as well. I think it helps to keep the rerouting approach
> > > orthogonal
> > > > > > to this KIP.
> > > > > >
> > > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps wrote:
> > > > > > > I'm +1 on Jun's suggestion as long as it can work for all the
> > > > requests.
> > > > > > >
> > > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > > >
> > > > > > > > Andrii,
> > > > > > > >
> > > > > > > > I think we agreed on the following.
> > > > > > > >
> > > > > > > > (a) Admin requests can be sent to and handled by any broker.
> > > > > > > > (b) Admin requests are processed asynchronously, at least for
> > > now.
> > > > > > That is,
> > > > > > > > when the client gets a response, it just means that the
> request
> > > is
> > > > > > > > initiated, but not necessarily completed. Then, it's up to
> the
> > > > client
> > > > > > to
> > > > > > > > issue another request to check the status for completion.
> > > > > > > >
> > > > > > > > To support (a), we were thinking of doing request forwarding
> to
> > > the
> > > > > > > > controller (utilizing KAFKA-1912). I am making an alternative
> > > > > proposal.
> > > > > > > > Basically, the broker can just write to ZooKeeper to inform
> the
> > > > > > controller
> > > > > > > > about the request. For example, to handle
> > partitionReassignment,
> > > > the
> > > > > > broker
> > > > > > > > will just write the requested partitions to
> > > > > /admin/reassign_partitions
> > > > > > > > (like what AdminUtils currently does) and then send a
> response
> > to
> > > > the
> > > > > > > > client. This shouldn't take long and the implementation will
> be
> > > > > simpler
> > > > > > > > than forwarding the requests to the controller through RPC.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > >
> > > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >
> > > > > > > > > Jun,
> > > > > > > > >
> > > > > > > > > I might be wrong but didn't we agree we will let any broker
> > > from
> > > > > the
> > > > > > > > > cluster handle *long-running* admin requests (at this time
> > > > > > > > preferredReplica
> > > > > > > > > and
> > > > > > > > > reassignPartitions), via zk admin path. Thus CreateTopics
> etc
> > > > > should
> > > > > > be
> > > > > > > > > sent
> > > > > > > > > only to the controller.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Andrii Biletskyi
> > > > > > > > >
> > > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <
> jun@confluent.io>
> > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Joel, Andril,
> > > > > > > > > >
> > > > > > > > > > I think we agreed that those admin requests can be issued
> > to
> > > > any
> > > > > > > > broker.
> > > > > > > > > > Because of that, there doesn't seem to be a strong need
> to
> > > know
> > > > > the
> > > > > > > > > > controller. So, perhaps we can proceed by not making any
> > > change
> > > > > to
> > > > > > the
> > > > > > > > > > format of TMR right now. When we start using create topic
> > > > request
> > > > > > in
> > > > > > > > the
> > > > > > > > > > producer, we will need a new version of TMR that doesn't
> > > > trigger
> > > > > > auto
> > > > > > > > > topic
> > > > > > > > > > creation. But that can be done later.
> > > > > > > > > >
> > > > > > > > > > As a first cut implementation, I think the broker can
> just
> > > > write
> > > > > > to ZK
> > > > > > > > > > directly for
> > > > > > > > > >
> > > > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > > requests, instead of forwarding them to the controller.
> > This
> > > > will
> > > > > > > > > simplify
> > > > > > > > > > the implementation on the broker side.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> > > > > jjkoshy.w@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > > >
> > > > > > > > > > > For (1) yes we will circle back on that shortly after
> > > syncing
> > > > > up
> > > > > > in
> > > > > > > > > > > person. I think it is close to getting committed
> although
> > > > > > development
> > > > > > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > > > > > >
> > > > > > > > > > > There is one more item we covered at the hangout. i.e.,
> > > > whether
> > > > > > we
> > > > > > > > > > > want to add the coordinator to the topic metadata
> > response
> > > or
> > > > > > provide
> > > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > > >
> > > > > > > > > > > There are two reasons I think we should try and avoid
> > > adding
> > > > > the
> > > > > > > > > > > field:
> > > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > > - If we finally do request rerouting in Kafka then the
> > > field
> > > > > > would
> > > > > > > > add
> > > > > > > > > > >   little to no value. (It still helps to have a
> separate
> > > > > > > > > > >   ClusterMetadataRequest to query for cluster-wide
> > > > information
> > > > > > such
> > > > > > > > as
> > > > > > > > > > >   'which broker is the controller?' as Joe mentioned.)
> > > > > > > > > > >
> > > > > > > > > > > I think it would be cleaner to have an explicit
> > > > > > > > ClusterMetadataRequest
> > > > > > > > > > > that you can send to any broker in order to obtain the
> > > > > controller
> > > > > > > > (and
> > > > > > > > > > > in the future possibly other cluster-wide
> information). I
> > > > think
> > > > > > the
> > > > > > > > > > > main argument against doing this and instead adding it
> to
> > > the
> > > > > > topic
> > > > > > > > > > > metadata response was convenience - i.e., you don't
> have
> > to
> > > > > > discover
> > > > > > > > > > > the controller in advance. However, I don't see much
> > actual
> > > > > > > > > > > benefit/convenience in this and in fact think it is a
> > > > > non-issue.
> > > > > > Let
> > > > > > > > > > > me know if I'm overlooking something here.
> > > > > > > > > > >
> > > > > > > > > > > As an example, say we need to initiate partition
> > > reassignment
> > > > > by
> > > > > > > > > > > issuing the new ReassignPartitionsRequest to the
> > controller
> > > > > > (assume
> > > > > > > > we
> > > > > > > > > > > already have the desired manual partition assignment).
> > If
> > > we
> > > > > > are to
> > > > > > > > > > > augment topic metadata response then the flow be
> > something
> > > > like
> > > > > > this
> > > > > > > > :
> > > > > > > > > > >
> > > > > > > > > > > - Issue topic metadata request to any broker (and
> > discover
> > > > the
> > > > > > > > > > >   controller
> > > > > > > > > > > - Connect to controller if required (i.e., if the
> broker
> > > > above
> > > > > !=
> > > > > > > > > > >   controller)
> > > > > > > > > > > - Issue the partition reassignment request to the
> > > controller.
> > > > > > > > > > >
> > > > > > > > > > > With an explicit cluster metadata request it would be:
> > > > > > > > > > > - Issue cluster metadata request to any broker
> > > > > > > > > > > - Connect to controller if required (i.e., if the
> broker
> > > > above
> > > > > !=
> > > > > > > > > > >   controller)
> > > > > > > > > > > - Issue the partition reassignment request
> > > > > > > > > > >
> > > > > > > > > > > So it seems to add little practical value and bloats
> > topic
> > > > > > metadata
> > > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > > >
> > > > > > > > > > > The other angle to this is the following - is it a
> matter
> > > of
> > > > > > naming?
> > > > > > > > > > > Should we just rename topic metadata request/response
> to
> > > just
> > > > > > > > > > > MetadataRequest/Response and add cluster metadata to
> it?
> > By
> > > > > that
> > > > > > same
> > > > > > > > > > > token should we also allow querying for the consumer
> > > > > coordinator
> > > > > > (and
> > > > > > > > > > > in future transaction coordinator) as well? This leads
> > to a
> > > > > > bloated
> > > > > > > > > > > request which isn't very appealing and altogether
> > > confusing.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Joel
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao
> wrote:
> > > > > > > > > > > > Andri,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > > >
> > > > > > > > > > > > 1. I just realized that in order to start working on
> > > > > > KAFKA-1927, we
> > > > > > > > > > will
> > > > > > > > > > > > need to merge the changes to OffsetCommitRequest
> (from
> > > > 0.8.2)
> > > > > > to
> > > > > > > > > trunk.
> > > > > > > > > > > > This is planned to be done as part of KAFKA-1634. So,
> > we
> > > > will
> > > > > > need
> > > > > > > > > > > Guozhang
> > > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > > >
> > > > > > > > > > > > 2. Thinking about this a bit more, if the semantic of
> > > those
> > > > > > "write"
> > > > > > > > > > > > requests is async (i.e., after the client gets a
> > > response,
> > > > it
> > > > > > just
> > > > > > > > > > means
> > > > > > > > > > > > that the operation is initiated, but not necessarily
> > > > > > completed), we
> > > > > > > > > > don't
> > > > > > > > > > > > really need to forward the requests to the
> controller.
> > > > > > Instead, the
> > > > > > > > > > > > receiving broker can just write the operation to ZK
> as
> > > the
> > > > > > admin
> > > > > > > > > > command
> > > > > > > > > > > > line tool previously does. This will simplify the
> > > > > > implementation.
> > > > > > > > > > > >
> > > > > > > > > > > > 8. There is another implementation detail for
> describe
> > > > topic.
> > > > > > > > > Ideally,
> > > > > > > > > > we
> > > > > > > > > > > > want to read the topic config from the broker cache,
> > > > instead
> > > > > of
> > > > > > > > > > > ZooKeeper.
> > > > > > > > > > > > Currently, every broker reads the topic-level config
> > for
> > > > all
> > > > > > > > topics.
> > > > > > > > > > > > However, it ignores those for topics not hosted on
> > > itself.
> > > > > So,
> > > > > > we
> > > > > > > > may
> > > > > > > > > > > need
> > > > > > > > > > > > to change TopicConfigManager a bit so that it caches
> > the
> > > > > > configs
> > > > > > > > for
> > > > > > > > > > all
> > > > > > > > > > > > topics.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > >
> > > > > > > > > > > > Jun
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Guys,
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1. Q: Get rid of all scala requests objects, use
> java
> > > > > > protocol
> > > > > > > > > > > definitions.
> > > > > > > > > > > > >     A: Gwen kindly took that (KAFKA-1927). It's
> > > important
> > > > > to
> > > > > > > > speed
> > > > > > > > > up
> > > > > > > > > > > > > review procedure
> > > > > > > > > > > > >          there since this ticket blocks other
> > important
> > > > > > changes.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 2. Q: Generic re-reroute facility vs client
> > maintaining
> > > > > > cluster
> > > > > > > > > > state.
> > > > > > > > > > > > >     A: Jay has added pseudo code to KAFKA-1912 -
> need
> > > to
> > > > > > consider
> > > > > > > > > > > whether
> > > > > > > > > > > > > this will be
> > > > > > > > > > > > >         easy to implement as a server-side feature
> > > > > (comments
> > > > > > are
> > > > > > > > > > > > > welcomed!).
> > > > > > > > > > > > >
> > > > > > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > > > > > >     A: This might be useful for clients, add this
> to
> > > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > > >
> > > > > > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > > > > > >     A: I will add proposed by Jun solution (using
> > > > clientId
> > > > > > for
> > > > > > > > > that)
> > > > > > > > > > > to the
> > > > > > > > > > > > > KIP.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs grabbing all
> > > > protocol
> > > > > > > > changes
> > > > > > > > > in
> > > > > > > > > > > one
> > > > > > > > > > > > > version.
> > > > > > > > > > > > >     A: It was decided to try to gather all changes
> to
> > > > > > protocol
> > > > > > > > > > (before
> > > > > > > > > > > > > release).
> > > > > > > > > > > > >         In case of TMR it worth checking:
> KAFKA-2020
> > > and
> > > > > > KIP-13
> > > > > > > > > > > (quotas)
> > > > > > > > > > > > >
> > > > > > > > > > > > > 6. Q: JSON lib is needed to deserialize user's
> input
> > in
> > > > CLI
> > > > > > tool.
> > > > > > > > > > > > >     A: Use jackson for that, /tools project is a
> > > separate
> > > > > > jar so
> > > > > > > > > > > shouldn't
> > > > > > > > > > > > > be a big deal.
> > > > > > > > > > > > >
> > > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs generic status
> > check
> > > > > > command.
> > > > > > > > > > > > >      A: For long-running requests like reassign
> > > > partitions
> > > > > > > > > *progress*
> > > > > > > > > > > check
> > > > > > > > > > > > > request is useful,
> > > > > > > > > > > > >          it makes sense to introduce it.
> > > > > > > > > > > > >
> > > > > > > > > > > > >  Please add, correct me if I missed something.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Joel,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > You are right, I removed ClusterMetadata because
> we
> > > > have
> > > > > > > > > partially
> > > > > > > > > > > > > > what we need in TopicMetadata. Also, as Jay
> pointed
> > > out
> > > > > > > > earlier,
> > > > > > > > > we
> > > > > > > > > > > > > > would like to have "orthogonal" API, but at the
> > same
> > > > time
> > > > > > we
> > > > > > > > need
> > > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > But I like your idea and even have some other
> > > arguments
> > > > > for
> > > > > > > > this
> > > > > > > > > > > option:
> > > > > > > > > > > > > > There is also DescribeTopicRequest which was
> > proposed
> > > > in
> > > > > > this
> > > > > > > > > KIP,
> > > > > > > > > > > > > > it returns topic configs, partitions, replication
> > > > factor
> > > > > > plus
> > > > > > > > > > > partition
> > > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > > leader replica. The later part is really already
> > > there
> > > > in
> > > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > > So again we'll have to add stuff to TMR, not to
> > > > duplicate
> > > > > > some
> > > > > > > > > info
> > > > > > > > > > > in
> > > > > > > > > > > > > > newly added requests. However, this way we'll end
> > up
> > > > with
> > > > > > > > > "monster"
> > > > > > > > > > > > > > request which returns cluster metadata, topic
> > > > replication
> > > > > > and
> > > > > > > > > > config
> > > > > > > > > > > info
> > > > > > > > > > > > > > plus partition replication data. Seems logical to
> > > split
> > > > > > TMR to
> > > > > > > > > > > > > > - ClusterMetadata (brokers + controller, maybe
> smth
> > > > else)
> > > > > > > > > > > > > > - TopicMetadata (topic info + partition details)
> > > > > > > > > > > > > > But since current TMR is involved in lots of
> places
> > > > > > (including
> > > > > > > > > > > network
> > > > > > > > > > > > > > client,
> > > > > > > > > > > > > > as I understand) this might be very serious
> change
> > > and
> > > > it
> > > > > > > > > probably
> > > > > > > > > > > makes
> > > > > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <
> > > > > > > > jjkoshy.w@gmail.com
> > > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >> I may be missing some context but hopefully this
> > > will
> > > > > > also be
> > > > > > > > > > > covered
> > > > > > > > > > > > > >> today: I thought the earlier proposal where
> there
> > > was
> > > > an
> > > > > > > > > explicit
> > > > > > > > > > > > > >> ClusterMetadata request was clearer and
> explicit.
> > > > During
> > > > > > the
> > > > > > > > > > course
> > > > > > > > > > > of
> > > > > > > > > > > > > >> this thread I think the conclusion was that the
> > main
> > > > > need
> > > > > > was
> > > > > > > > > for
> > > > > > > > > > > > > >> controller information and that can be rolled
> into
> > > the
> > > > > > topic
> > > > > > > > > > > metadata
> > > > > > > > > > > > > >> response but that seems a bit irrelevant to
> topic
> > > > > > metadata.
> > > > > > > > > FWIW I
> > > > > > > > > > > > > >> think the full broker-list is also irrelevant to
> > > topic
> > > > > > > > metadata,
> > > > > > > > > > but
> > > > > > > > > > > > > >> it is already there and in use. I think there is
> > > still
> > > > > > room
> > > > > > > > for
> > > > > > > > > an
> > > > > > > > > > > > > >> explicit ClusterMetadata request since there may
> > be
> > > > > other
> > > > > > > > > > > > > >> cluster-level information that we may want to
> add
> > > over
> > > > > > time
> > > > > > > > (and
> > > > > > > > > > > that
> > > > > > > > > > > > > >> have nothing to do with topic metadata).
> > > > > > > > > > > > > >>
> > > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii
> > > > > Biletskyi
> > > > > > > > > wrote:
> > > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > 101. Okay, if you say that such use case is
> > > > > important. I
> > > > > > > > also
> > > > > > > > > > > think
> > > > > > > > > > > > > >> > using clientId for these purposes is fine - if
> > we
> > > > > > already
> > > > > > > > have
> > > > > > > > > > > this
> > > > > > > > > > > > > >> field
> > > > > > > > > > > > > >> > as part of all Wire protocol messages, why not
> > use
> > > > > that.
> > > > > > > > > > > > > >> > I will update KIP-4 page if nobody has other
> > ideas
> > > > > > (which
> > > > > > > > may
> > > > > > > > > > > come up
> > > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP accordingly.
> I
> > > > think
> > > > > > we can
> > > > > > > > > add
> > > > > > > > > > > new,
> > > > > > > > > > > > > >> > fine-grained error codes if some error code
> > > received
> > > > > in
> > > > > > > > > specific
> > > > > > > > > > > case
> > > > > > > > > > > > > >> > won't give enough context to return a
> > descriptive
> > > > > error
> > > > > > > > > message
> > > > > > > > > > > for
> > > > > > > > > > > > > >> user.
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > Look forward to discussing all outstanding
> > issues
> > > in
> > > > > > detail
> > > > > > > > > > today
> > > > > > > > > > > > > during
> > > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <
> > > > > > jun@confluent.io
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >> >
> > > > > > > > > > > > > >> > > 101. There may be a use case where you only
> > want
> > > > the
> > > > > > > > topics
> > > > > > > > > to
> > > > > > > > > > > be
> > > > > > > > > > > > > >> created
> > > > > > > > > > > > > >> > > manually by admins. Currently, you can do
> that
> > > by
> > > > > > > > disabling
> > > > > > > > > > auto
> > > > > > > > > > > > > topic
> > > > > > > > > > > > > >> > > creation and issue topic creation from the
> > > > > > TopicCommand.
> > > > > > > > If
> > > > > > > > > we
> > > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > > >> > > topic creation completely on the broker and
> > > don't
> > > > > > have a
> > > > > > > > way
> > > > > > > > > > to
> > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > >> > > between topic creation requests from the
> > regular
> > > > > > clients
> > > > > > > > and
> > > > > > > > > > the
> > > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > > >> > > can't support manual topic creation any
> more.
> > I
> > > > was
> > > > > > > > thinking
> > > > > > > > > > > that
> > > > > > > > > > > > > >> another
> > > > > > > > > > > > > >> > > way of distinguishing the clients making the
> > > topic
> > > > > > > > creation
> > > > > > > > > > > requests
> > > > > > > > > > > > > >> is
> > > > > > > > > > > > > >> > > using clientId. For example, the admin tool
> > can
> > > > set
> > > > > > it to
> > > > > > > > > > > something
> > > > > > > > > > > > > >> like
> > > > > > > > > > > > > >> > > admin and the broker can treat that clientId
> > > > > > specially.
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > Also, there is a related discussion in
> > > KAFKA-2020.
> > > > > > > > > Currently,
> > > > > > > > > > > we do
> > > > > > > > > > > > > >> the
> > > > > > > > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > 1. If leader is not available, we set the
> > > > partition
> > > > > > level
> > > > > > > > > > error
> > > > > > > > > > > code
> > > > > > > > > > > > > >> to
> > > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > > >> > > 2. If a non-leader replica is not available,
> > we
> > > > take
> > > > > > that
> > > > > > > > > > > replica
> > > > > > > > > > > > > out
> > > > > > > > > > > > > >> of
> > > > > > > > > > > > > >> > > the assigned replica list and isr in the
> > > response.
> > > > > As
> > > > > > an
> > > > > > > > > > > indication
> > > > > > > > > > > > > >> for
> > > > > > > > > > > > > >> > > doing that, we set the partition level error
> > > code
> > > > to
> > > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > This has a few problems. First,
> > > > ReplicaNotAvailable
> > > > > > > > probably
> > > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > > >> > > an error, at least for the normal
> > > > producer/consumer
> > > > > > > > clients
> > > > > > > > > > that
> > > > > > > > > > > > > just
> > > > > > > > > > > > > >> want
> > > > > > > > > > > > > >> > > to find out the leader. Second, it can
> happen
> > > that
> > > > > > both
> > > > > > > > the
> > > > > > > > > > > leader
> > > > > > > > > > > > > and
> > > > > > > > > > > > > >> > > another replica are not available at the
> same
> > > > time.
> > > > > > There
> > > > > > > > is
> > > > > > > > > > no
> > > > > > > > > > > > > error
> > > > > > > > > > > > > >> code
> > > > > > > > > > > > > >> > > to indicate both. Third, even if a replica
> is
> > > not
> > > > > > > > available,
> > > > > > > > > > > it's
> > > > > > > > > > > > > >> still
> > > > > > > > > > > > > >> > > useful to return its replica id since some
> > > clients
> > > > > > (e.g.
> > > > > > > > > admin
> > > > > > > > > > > tool)
> > > > > > > > > > > > > >> may
> > > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > One way to address this issue is to always
> > > return
> > > > > the
> > > > > > > > > replica
> > > > > > > > > > > id for
> > > > > > > > > > > > > >> > > leader, assigned replicas, and isr
> regardless
> > of
> > > > > > whether
> > > > > > > > the
> > > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > > >> > > broker is live or not. Since we also return
> > the
> > > > list
> > > > > > of
> > > > > > > > live
> > > > > > > > > > > > > brokers,
> > > > > > > > > > > > > >> the
> > > > > > > > > > > > > >> > > client can figure out whether a leader or a
> > > > replica
> > > > > is
> > > > > > > > live
> > > > > > > > > or
> > > > > > > > > > > not
> > > > > > > > > > > > > >> and act
> > > > > > > > > > > > > >> > > accordingly. This way, we don't need to set
> > the
> > > > > > partition
> > > > > > > > > > level
> > > > > > > > > > > > > error
> > > > > > > > > > > > > >> code
> > > > > > > > > > > > > >> > > when the leader or a replica is not
> available.
> > > > This
> > > > > > > > doesn't
> > > > > > > > > > > change
> > > > > > > > > > > > > >> the wire
> > > > > > > > > > > > > >> > > protocol, but does change the semantics.
> Since
> > > we
> > > > > are
> > > > > > > > > evolving
> > > > > > > > > > > the
> > > > > > > > > > > > > >> protocol
> > > > > > > > > > > > > >> > > of TopicMetadataRequest here, we can
> > potentially
> > > > > > piggyback
> > > > > > > > > the
> > > > > > > > > > > > > change.
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > 102.1 For those types of errors due to
> invalid
> > > > > input,
> > > > > > > > > > shouldn't
> > > > > > > > > > > we
> > > > > > > > > > > > > >> just
> > > > > > > > > > > > > >> > > guard it at parameter validation time and
> > throw
> > > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > > >> > > without even sending the request to the
> > broker?
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii
> > > > Biletskyi <
> > > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > 101. If I understand you correctly, you
> are
> > > > saying
> > > > > > > > future
> > > > > > > > > > > producer
> > > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > > >> > > > will be ported to TMR_V1) won't be able to
> > > > > > automatically
> > > > > > > > > > > create
> > > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > >> > > > unconditionally remove topic creation from
> > > > there).
> > > > > > But
> > > > > > > > we
> > > > > > > > > > > need to
> > > > > > > > > > > > > >> this
> > > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > > >> > > > Ok, about your proposal: I'm not a big fan
> > > too,
> > > > > > when it
> > > > > > > > > > comes
> > > > > > > > > > > to
> > > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > > >> > > > clients directly in protocol schema. And
> > also
> > > > I'm
> > > > > > not
> > > > > > > > > sure I
> > > > > > > > > > > > > >> understand
> > > > > > > > > > > > > >> > > at
> > > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > > >> > > > auto.create.topics.enable is a server side
> > > > > > > > configuration.
> > > > > > > > > > Can
> > > > > > > > > > > we
> > > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > > >> > > > in future versions, add this setting to
> > > producer
> > > > > and
> > > > > > > > based
> > > > > > > > > > on
> > > > > > > > > > > that
> > > > > > > > > > > > > >> upon
> > > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > > >> > > > UnknownTopic create topic explicitly by a
> > > > separate
> > > > > > > > > producer
> > > > > > > > > > > call
> > > > > > > > > > > > > via
> > > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because we want to
> > > support
> > > > > > batching
> > > > > > > > > and
> > > > > > > > > > > at
> > > > > > > > > > > > > the
> > > > > > > > > > > > > >> same
> > > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > > >> > > > want to give descriptive error messages
> for
> > > > > clients.
> > > > > > > > Since
> > > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > > >> > > > to construct such messages (e.g.
> AdminClient
> > > > layer
> > > > > > can
> > > > > > > > > know
> > > > > > > > > > > that
> > > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > > >> > > > means two cases: either invalid number -
> > e.g.
> > > > -1;
> > > > > or
> > > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > > >> > > > partitions argument wasn't) - I wrapped
> > > > responses
> > > > > in
> > > > > > > > > > > Exceptions.
> > > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > > >> > > > other ideas, this was just initial
> version.
> > > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change that to
> > > > probably
> > > > > > some
> > > > > > > > > other
> > > > > > > > > > > dto.
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <
> > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > >> > > > > 101. That's what I was thinking too, but
> > it
> > > > may
> > > > > > not be
> > > > > > > > > > that
> > > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > > >> > > > > we can let it not trigger auto topic
> > > creation.
> > > > > > Then,
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > > >> > > > > if it gets an UnknownTopicException, it
> > can
> > > > > > explicitly
> > > > > > > > > > > issue a
> > > > > > > > > > > > > >> > > > > createTopicRequest for auto topic
> > creation.
> > > On
> > > > > the
> > > > > > > > > > consumer
> > > > > > > > > > > > > side,
> > > > > > > > > > > > > >> it
> > > > > > > > > > > > > >> > > will
> > > > > > > > > > > > > >> > > > > never issue createTopicRequest. This
> works
> > > > when
> > > > > > auto
> > > > > > > > > topic
> > > > > > > > > > > > > >> creation is
> > > > > > > > > > > > > >> > > > > enabled on the broker side. However, I
> am
> > > not
> > > > > > sure how
> > > > > > > > > > > things
> > > > > > > > > > > > > >> will work
> > > > > > > > > > > > > >> > > > > when auto topic creation is disabled on
> > the
> > > > > broker
> > > > > > > > side.
> > > > > > > > > > In
> > > > > > > > > > > this
> > > > > > > > > > > > > >> case,
> > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > >> > > > > want to have a way to manually create a
> > > topic,
> > > > > > > > > potentially
> > > > > > > > > > > > > through
> > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > >> > > > > commands. However, then we need a way to
> > > > > > distinguish
> > > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > > >> > > > > issued from the producer clients and the
> > > admin
> > > > > > tools.
> > > > > > > > > May
> > > > > > > > > > > be we
> > > > > > > > > > > > > >> can
> > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > >> > > > > new field in createTopicRequest and set
> it
> > > > > > differently
> > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > >> producer
> > > > > > > > > > > > > >> > > > > client and the admin client. However, I
> am
> > > not
> > > > > > sure if
> > > > > > > > > > > that's
> > > > > > > > > > > > > the
> > > > > > > > > > > > > >> best
> > > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > >> > > > > 2. Yes, refactoring existing requests
> is a
> > > > > > non-trivial
> > > > > > > > > > > amount of
> > > > > > > > > > > > > >> work.
> > > > > > > > > > > > > >> > > I
> > > > > > > > > > > > > >> > > > > posted some comments in KAFKA-1927. We
> > will
> > > > > > probably
> > > > > > > > > have
> > > > > > > > > > > to fix
> > > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > > >> > > > > first, before adding the new logic in
> > > > > KAFKA-1694.
> > > > > > > > > > > Otherwise, the
> > > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > > > > >> > > > > 102.1. It's a bit weird that we return
> > > > exception
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > api. It
> > > > > > > > > > > > > >> seems
> > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > >> > > > > we should either return error code or
> > throw
> > > an
> > > > > > > > exception
> > > > > > > > > > > when
> > > > > > > > > > > > > >> getting
> > > > > > > > > > > > > >> > > the
> > > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > > >> > > > > 102.2. We probably shouldn't explicitly
> > use
> > > > the
> > > > > > > > request
> > > > > > > > > > > object
> > > > > > > > > > > > > in
> > > > > > > > > > > > > >> the
> > > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > > >> > > > > Not every request evolution requires an
> > api
> > > > > > change.
> > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii
> > > > > Biletskyi
> > > > > > <
> > > > > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > > Thanks for you comments. Answers
> inline:
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > > 100. There are a few fields such as
> > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> > > > > represented
> > > > > > as a
> > > > > > > > > > > string,
> > > > > > > > > > > > > but
> > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > >> > > > > > > composite structures in json. Could
> we
> > > > > flatten
> > > > > > > > them
> > > > > > > > > > out
> > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > >> > > > > > > protocol definition as
> arrays/records?
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > > Yes, now with Admin Client this looks
> a
> > > bit
> > > > > > weird.
> > > > > > > > My
> > > > > > > > > > > initial
> > > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > > >> > > > > > ReassignPartitionCommand accepts input
> > in
> > > > > json,
> > > > > > we
> > > > > > > > > want
> > > > > > > > > > to
> > > > > > > > > > > > > >> remain
> > > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > > >> > > > > > interfaces unchanged, where possible.
> > > > > > > > > > > > > >> > > > > > If we port it to deserialized format,
> in
> > > CLI
> > > > > > (/tools
> > > > > > > > > > > project)
> > > > > > > > > > > > > >> we will
> > > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > > >> > > > > > json library since /tools is written
> in
> > > java
> > > > > and
> > > > > > > > we'll
> > > > > > > > > > > need to
> > > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > > >> > > > > > provided by a user. Can we quickly
> agree
> > > on
> > > > > what
> > > > > > > > this
> > > > > > > > > > > library
> > > > > > > > > > > > > >> should
> > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > > 101. Does TopicMetadataRequest v1
> still
> > > > > trigger
> > > > > > auto
> > > > > > > > > > topic
> > > > > > > > > > > > > >> creation?
> > > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > > >> > > > > > > will be a bit weird now that we
> have a
> > > > > > separate
> > > > > > > > > topic
> > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > > > createTopicRequest
> > > > > > > > and
> > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > producer/consumer
> > > > > > client,
> > > > > > > > in
> > > > > > > > > > > addition
> > > > > > > > > > > > > >> to
> > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > >> > > > > > > tools? For example, ideally, we
> don't
> > > want
> > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > >> > > > > > > consumer to trigger auto topic
> > creation.
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > > I agree, this strange logic should be
> > > fixed.
> > > > > > I'm not
> > > > > > > > > > > confident
> > > > > > > > > > > > > >> in
> > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > > >> > > > > > correct me if I'm wrong, but it
> doesn't
> > > look
> > > > > > like a
> > > > > > > > > hard
> > > > > > > > > > > thing
> > > > > > > > > > > > > >> to
> > > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > > >> > > > > > leverage AdminClient for that in
> > Producer
> > > > and
> > > > > > > > > > > unconditionally
> > > > > > > > > > > > > >> remove
> > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > >> > > > > > creation from the
> > TopicMetadataRequest_V1.
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > > 2. I think Jay meant getting rid of
> > scala
> > > > > > classes
> > > > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when adding the
> > new
> > > > > > requests
> > > > > > > > for
> > > > > > > > > > the
> > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > >> > > > > > > However, the long term plan is to
> get
> > > rid
> > > > of
> > > > > > all
> > > > > > > > > those
> > > > > > > > > > > and
> > > > > > > > > > > > > >> just
> > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > >> > > > > > > java request/response in the client.
> > > Since
> > > > > > this
> > > > > > > > KIP
> > > > > > > > > > > proposes
> > > > > > > > > > > > > >> to
> > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > >> > > > > > > significant number of new requests,
> > > > perhaps
> > > > > we
> > > > > > > > > should
> > > > > > > > > > > bite
> > > > > > > > > > > > > the
> > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > >> > > > > > > clean up the existing scala requests
> > > first
> > > > > > before
> > > > > > > > > > > adding new
> > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > > Yes, looks like I misunderstood the
> > point
> > > of
> > > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > > >> > > > > > rework that. The only thing is that I
> > > don't
> > > > > see
> > > > > > any
> > > > > > > > > > > example
> > > > > > > > > > > > > how
> > > > > > > > > > > > > >> it
> > > > > > > > > > > > > >> > > was
> > > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > > >> > > > > > least one existing protocol message.
> > Thus,
> > > > as
> > > > > I
> > > > > > > > > > > understand, I
> > > > > > > > > > > > > >> have to
> > > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > > >> > > > > > Re porting all existing RQ/RP in this
> > > patch.
> > > > > > Sounds
> > > > > > > > > > > > > reasonable,
> > > > > > > > > > > > > >> but
> > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > > >> > > > > > requirement to have Admin KIP done,
> I'm
> > > > afraid
> > > > > > this
> > > > > > > > > can
> > > > > > > > > > > be a
> > > > > > > > > > > > > >> serious
> > > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > > >> > > > > > There are 13 protocol messages and all
> > > that
> > > > > > would
> > > > > > > > > > require
> > > > > > > > > > > not
> > > > > > > > > > > > > >> only
> > > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > > >> > > > > > intensive manual testing, no? I'm
> afraid
> > > I'm
> > > > > > not the
> > > > > > > > > > > right guy
> > > > > > > > > > > > > >> to
> > > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > > >> > > > > > Kafka core internals :). Let me know
> > your
> > > > > > thoughts
> > > > > > > > on
> > > > > > > > > > this
> > > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > > >> > >
> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun
> > Rao <
> > > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > > 100. There are a few fields such as
> > > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> > > > > represented
> > > > > > as a
> > > > > > > > > > > string,
> > > > > > > > > > > > > but
> > > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > > >> > > > > > > composite structures in json. Could
> we
> > > > > flatten
> > > > > > > > them
> > > > > > > > > > out
> > > > > > > > > > > > > >> directly in
> > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > >> > > > > > > protocol definition as
> arrays/records?
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > > 101. Does TopicMetadataRequest v1
> > still
> > > > > > trigger
> > > > > > > > auto
> > > > > > > > > > > topic
> > > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > > >> > > > > > > will be a bit weird now that we
> have a
> > > > > > separate
> > > > > > > > > topic
> > > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > > > createTopicRequest
> > > > > > > > and
> > > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > > >> > > > > > > v1 will be used in the
> > producer/consumer
> > > > > > client,
> > > > > > > > in
> > > > > > > > > > > addition
> > > > > > > > > > > > > >> to
> > > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > > >> > > > > > > tools? For example, ideally, we
> don't
> > > want
> > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > >> > > from
> > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > >> > > > > > > consumer to trigger auto topic
> > creation.
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > > 2. I think Jay meant getting rid of
> > > scala
> > > > > > classes
> > > > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > > >> > > did
> > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > >> > > > > > > as a stop-gap thing when adding the
> > new
> > > > > > requests
> > > > > > > > for
> > > > > > > > > > the
> > > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > > >> > > > > > > However, the long term plan is to
> get
> > > rid
> > > > of
> > > > > > all
> > > > > > > > > those
> > > > > > > > > > > and
> > > > > > > > > > > > > >> just
> > > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > >> > > > > > > java request/response in the client.
> > > Since
> > > > > > this
> > > > > > > > KIP
> > > > > > > > > > > proposes
> > > > > > > > > > > > > >> to
> > > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > > >> > > > > > > significant number of new requests,
> > > > perhaps
> > > > > we
> > > > > > > > > should
> > > > > > > > > > > bite
> > > > > > > > > > > > > the
> > > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > >> > > > > > > clean up the existing scala requests
> > > first
> > > > > > before
> > > > > > > > > > > adding new
> > > > > > > > > > > > > >> ones?
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM,
> > Andrii
> > > > > > Biletskyi
> > > > > > > > <
> > > > > > > > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > As said above - I list again all
> > > > comments
> > > > > > from
> > > > > > > > > this
> > > > > > > > > > > thread
> > > > > > > > > > > > > >> so we
> > > > > > > > > > > > > >> > > > > > > > can see what's left and finalize
> all
> > > > > pending
> > > > > > > > > issues.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > > >> > > > > > > > 1. This is much needed
> > functionality,
> > > > but
> > > > > > there
> > > > > > > > > are
> > > > > > > > > > a
> > > > > > > > > > > lot
> > > > > > > > > > > > > >> of the
> > > > > > > > > > > > > >> > > so
> > > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > > >> > > > > > > > really think these protocols
> > through.
> > > We
> > > > > > really
> > > > > > > > > want
> > > > > > > > > > > to
> > > > > > > > > > > > > end
> > > > > > > > > > > > > >> up
> > > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > > >> > > > > > > > of well thought-out, orthoganol
> > apis.
> > > > For
> > > > > > this
> > > > > > > > > > reason
> > > > > > > > > > > I
> > > > > > > > > > > > > >> think it
> > > > > > > > > > > > > >> > > is
> > > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > > >> > > > > > > > important to think through the end
> > > state
> > > > > > even if
> > > > > > > > > > that
> > > > > > > > > > > > > >> includes
> > > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > >> > > > > > > > won't implement in the first
> phase.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: Definitely behind this. Would
> > > > > appreciate
> > > > > > if
> > > > > > > > > there
> > > > > > > > > > > are
> > > > > > > > > > > > > >> concrete
> > > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 2. Let's please please please wait
> > > until
> > > > > we
> > > > > > have
> > > > > > > > > > > switched
> > > > > > > > > > > > > >> the
> > > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > > >> > > > > > > > to the new java protocol
> > definitions.
> > > If
> > > > > we
> > > > > > add
> > > > > > > > > > upteen
> > > > > > > > > > > > > more
> > > > > > > > > > > > > >> ad
> > > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > > >> > > > > > > > objects that is just generating
> more
> > > > work
> > > > > > for
> > > > > > > > the
> > > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > removed
> > > > > scala
> > > > > > > > > > protocol
> > > > > > > > > > > > > >> classes.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 3. This proposal introduces a new
> > type
> > > > of
> > > > > > > > optional
> > > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > > >> > > This
> > > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > > >> > > > > > > > inconsistent with everything else
> in
> > > the
> > > > > > > > protocol
> > > > > > > > > > > where we
> > > > > > > > > > > > > >> use -1
> > > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > > >> > > > > > > > other marker value. You could
> argue
> > > > either
> > > > > > way
> > > > > > > > but
> > > > > > > > > > > let's
> > > > > > > > > > > > > >> stick
> > > > > > > > > > > > > >> > > with
> > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > >> > > > > > > > for consistency. For clients that
> > > > > > implemented
> > > > > > > > the
> > > > > > > > > > > protocol
> > > > > > > > > > > > > >> in a
> > > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > > >> > > > > > > > than our scala code these basic
> > > > primitives
> > > > > > are
> > > > > > > > > hard
> > > > > > > > > > to
> > > > > > > > > > > > > >> change.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > removed
> > > > > > MaybeOf
> > > > > > > > > type
> > > > > > > > > > > and
> > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This seems to
> > > > > duplicate
> > > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > >> > > > > > > > brokers, topics, and partitions. I
> > > think
> > > > > we
> > > > > > > > should
> > > > > > > > > > > rename
> > > > > > > > > > > > > >> that
> > > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > > >> > > > > > > > ClusterMetadataRequest (or just
> > > > > > MetadataRequest)
> > > > > > > > > and
> > > > > > > > > > > > > >> include the
> > > > > > > > > > > > > >> > > id
> > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > >> > > > > > > > controller. Or are there other
> > things
> > > we
> > > > > > could
> > > > > > > > add
> > > > > > > > > > > here?
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: I agree. Updated the KIP. Let's
> > > > extends
> > > > > > > > > > > TopicMetadata
> > > > > > > > > > > > > to
> > > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 5. We have a tendency to try to
> > make a
> > > > lot
> > > > > > of
> > > > > > > > > > requests
> > > > > > > > > > > > > that
> > > > > > > > > > > > > >> can
> > > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > > >> > > > > > > > particular nodes. This adds a lot
> of
> > > > > burden
> > > > > > for
> > > > > > > > > > client
> > > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > > >> > > > > > > > sounds easy but each discovery can
> > > fail
> > > > in
> > > > > > many
> > > > > > > > > > parts
> > > > > > > > > > > so
> > > > > > > > > > > > > it
> > > > > > > > > > > > > >> ends
> > > > > > > > > > > > > >> > > up
> > > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > > >> > > > > > > > full state machine to do right). I
> > > think
> > > > > we
> > > > > > > > should
> > > > > > > > > > > > > consider
> > > > > > > > > > > > > >> > > making
> > > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > > >> > > > > > > > commands and ideally as many of
> the
> > > > other
> > > > > > apis
> > > > > > > > as
> > > > > > > > > > > possible
> > > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > > >> > > > > > > > brokers and just redirect to the
> > > > > controller
> > > > > > on
> > > > > > > > the
> > > > > > > > > > > broker
> > > > > > > > > > > > > >> side.
> > > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > > >> > > > > > > > there would be a general way to
> > > > > encapsulate
> > > > > > this
> > > > > > > > > > > > > re-routing
> > > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: It's a very interesting idea,
> but
> > > > seems
> > > > > > there
> > > > > > > > > are
> > > > > > > > > > > some
> > > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > > >> > > > > > > > feature (like performance
> > > > considerations,
> > > > > > how
> > > > > > > > this
> > > > > > > > > > > will
> > > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > > >> > > > > > > > I believe this shouldn't be a
> > blocker.
> > > > If
> > > > > > this
> > > > > > > > > > > feature is
> > > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > > >> > > > > > > > point it won't affect Admin
> changes
> > -
> > > at
> > > > > > least
> > > > > > > > no
> > > > > > > > > > > changes
> > > > > > > > > > > > > to
> > > > > > > > > > > > > >> > > public
> > > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 6. We should probably normalize
> the
> > > key
> > > > > > value
> > > > > > > > > pairs
> > > > > > > > > > > used
> > > > > > > > > > > > > for
> > > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > >> > > > > > > > than embedding a new formatting.
> So
> > > two
> > > > > > strings
> > > > > > > > > > rather
> > > > > > > > > > > > > than
> > > > > > > > > > > > > >> one
> > > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > > > normalized
> > > > > > > > configs
> > > > > > > > > > and
> > > > > > > > > > > > > >> changed
> > > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 7. Is the postcondition of these
> > APIs
> > > > that
> > > > > > the
> > > > > > > > > > > command has
> > > > > > > > > > > > > >> begun
> > > > > > > > > > > > > >> > > or
> > > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > > >> > > > > > > > the command has been completed? It
> > is
> > > a
> > > > > lot
> > > > > > more
> > > > > > > > > > > usable if
> > > > > > > > > > > > > >> the
> > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > > >> > > > > > > > been completed so you know that if
> > you
> > > > > > create a
> > > > > > > > > > topic
> > > > > > > > > > > and
> > > > > > > > > > > > > >> then
> > > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > >> > > > > > > > it you won't get an exception
> about
> > > > there
> > > > > > being
> > > > > > > > no
> > > > > > > > > > > such
> > > > > > > > > > > > > >> topic.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: For long running requests (like
> > > > > reassign
> > > > > > > > > > > partitions) -
> > > > > > > > > > > > > >> the
> > > > > > > > > > > > > >> > > post
> > > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > > >> > > > > > > > command has begun - so we don't
> > block
> > > > the
> > > > > > > > client.
> > > > > > > > > In
> > > > > > > > > > > case
> > > > > > > > > > > > > >> of your
> > > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > > >> > > > > > > > topic commands, this will be
> > > refactored
> > > > > and
> > > > > > > > topic
> > > > > > > > > > > commands
> > > > > > > > > > > > > >> will
> > > > > > > > > > > > > >> > > be
> > > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > > >> > > > > > > > immediately, since the Controller
> > will
> > > > > serve
> > > > > > > > Admin
> > > > > > > > > > > > > requests
> > > > > > > > > > > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 8. Describe topic and list topics
> > > > > duplicate
> > > > > > a
> > > > > > > > lot
> > > > > > > > > of
> > > > > > > > > > > stuff
> > > > > > > > > > > > > >> in the
> > > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > > >> > > > > > > > request. Is there a reason to give
> > > back
> > > > > > topics
> > > > > > > > > > marked
> > > > > > > > > > > for
> > > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > > >> > > > > > > > like if we just make the
> > > post-condition
> > > > of
> > > > > > the
> > > > > > > > > > delete
> > > > > > > > > > > > > >> command be
> > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > >> > > > > > > > topic is deleted that will get rid
> > of
> > > > the
> > > > > > need
> > > > > > > > for
> > > > > > > > > > > this
> > > > > > > > > > > > > >> right?
> > > > > > > > > > > > > >> > > And
> > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > removed
> > > > > > topics
> > > > > > > > > marked
> > > > > > > > > > > for
> > > > > > > > > > > > > >> deletion
> > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 9. Should we consider batching
> these
> > > > > > requests?
> > > > > > > > We
> > > > > > > > > > have
> > > > > > > > > > > > > >> generally
> > > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > >> > > > > > > > allow multiple operations to be
> > > batched.
> > > > > My
> > > > > > > > > > suspicion
> > > > > > > > > > > is
> > > > > > > > > > > > > >> that
> > > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > >> > > > > > > > we will get a lot of code that
> does
> > > > > > something
> > > > > > > > like
> > > > > > > > > > > > > >> > > > > > > >    for(topic:
> > > adminClient.listTopics())
> > > > > > > > > > > > > >> > > > > > > >
> >  adminClient.describeTopic(topic)
> > > > > > > > > > > > > >> > > > > > > > this code will work great when you
> > > test
> > > > > on 5
> > > > > > > > > topics
> > > > > > > > > > > but
> > > > > > > > > > > > > not
> > > > > > > > > > > > > >> do as
> > > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check
> > > "Topic
> > > > > > Admin
> > > > > > > > > > Schema"
> > > > > > > > > > > > > >> section.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 10. I think we should also discuss
> > how
> > > > we
> > > > > > want
> > > > > > > > to
> > > > > > > > > > > expose a
> > > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > > >> > > > > > > > client api for these operations.
> > > > Currently
> > > > > > > > people
> > > > > > > > > > > rely on
> > > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > > >> > > > > > > > is totally sketchy. I think we
> > > probably
> > > > > need
> > > > > > > > > another
> > > > > > > > > > > > > client
> > > > > > > > > > > > > >> under
> > > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > > >> > > > > > > > that exposes administrative
> > > > functionality.
> > > > > > We
> > > > > > > > will
> > > > > > > > > > > need
> > > > > > > > > > > > > >> this just
> > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > >> > > > > > > > properly test the new apis, I
> > suspect.
> > > > We
> > > > > > should
> > > > > > > > > > > figure
> > > > > > > > > > > > > out
> > > > > > > > > > > > > >> that
> > > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check
> > > "Admin
> > > > > > Client"
> > > > > > > > > > > section
> > > > > > > > > > > > > >> with an
> > > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 11. The other information that
> would
> > > be
> > > > > > really
> > > > > > > > > > useful
> > > > > > > > > > > to
> > > > > > > > > > > > > get
> > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > >> > > > > > > > information about partitions--how
> > much
> > > > > data
> > > > > > is
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > > >> partition,
> > > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > > >> > > > > > > > the segment offsets, what is the
> > > log-end
> > > > > > offset
> > > > > > > > > > (i.e.
> > > > > > > > > > > last
> > > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > > >> > > > > > > > the compaction point, etc. I think
> > > that
> > > > > done
> > > > > > > > right
> > > > > > > > > > > this
> > > > > > > > > > > > > >> would be
> > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > >> > > > > > > > successor to the very awkward
> > > > > OffsetRequest
> > > > > > we
> > > > > > > > > have
> > > > > > > > > > > today.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: I removed
> > > ConsumerGroupOffsetsRequest
> > > > > in
> > > > > > the
> > > > > > > > > > latest
> > > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > > >> > > > > > > > be resolved in a separate KIP /
> jira
> > > > > ticket.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 12. Generally we can do good error
> > > > > handling
> > > > > > > > > without
> > > > > > > > > > > > > needing
> > > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > > >> > > > > > > > messages. I.e. generally the
> client
> > > has
> > > > > the
> > > > > > > > > context
> > > > > > > > > > to
> > > > > > > > > > > > > know
> > > > > > > > > > > > > >> that
> > > > > > > > > > > > > >> > > if
> > > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > > >> > > > > > > > an error that the topic doesn't
> > exist
> > > to
> > > > > say
> > > > > > > > > "Topic
> > > > > > > > > > X
> > > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > > >> > > > > > > > than "error code 14" (or
> whatever).
> > > > Maybe
> > > > > > there
> > > > > > > > > are
> > > > > > > > > > > > > specific
> > > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > > >> > > > > > > > this is hard? If we want to add
> > > > > server-side
> > > > > > > > error
> > > > > > > > > > > messages
> > > > > > > > > > > > > >> we
> > > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > > >> > > > > > > > need to do this in a consistent
> way
> > > > across
> > > > > > the
> > > > > > > > > > > protocol.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check
> > > > > "Protocol
> > > > > > > > > Errors"
> > > > > > > > > > > > > >> section. I
> > > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > >> > > > > > > > comprehensive, fine-grained list
> of
> > > > error
> > > > > > codes.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > > > > > >> > > > > > > > 13. Describe topic request: it
> would
> > > be
> > > > > > great to
> > > > > > > > > go
> > > > > > > > > > > beyond
> > > > > > > > > > > > > >> just
> > > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > >> > > > > > > > topic name regex for this request.
> > For
> > > > > > example,
> > > > > > > > a
> > > > > > > > > > very
> > > > > > > > > > > > > >> common use
> > > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > > >> > > > > > > > the topic command is to list all
> > > topics
> > > > > > whose
> > > > > > > > > config
> > > > > > > > > > > A's
> > > > > > > > > > > > > >> value is
> > > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > > >> > > > > > > > topic name regex then we have to
> > first
> > > > > > retrieve
> > > > > > > > > > > __all__
> > > > > > > > > > > > > >> topics's
> > > > > > > > > > > > > >> > > > > > > > description info and then filter
> at
> > > the
> > > > > > client
> > > > > > > > > end,
> > > > > > > > > > > which
> > > > > > > > > > > > > >> will
> > > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in create topic:
> > this
> > > is
> > > > > > related
> > > > > > > > > to
> > > > > > > > > > > the
> > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > > >> > > > > > > > maybe we can add another metadata
> > K-V
> > > or
> > > > > > just a
> > > > > > > > > > > metadata
> > > > > > > > > > > > > >> string
> > > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > > >> > > > > > > > with config K-V in create topic
> like
> > > we
> > > > > did
> > > > > > for
> > > > > > > > > > offset
> > > > > > > > > > > > > >> commit
> > > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > > >> > > > > > > > field can be quite useful in
> storing
> > > > > > information
> > > > > > > > > > like
> > > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > > >> > > > > > > > who issue the create command, etc,
> > > which
> > > > > is
> > > > > > > > quite
> > > > > > > > > > > > > important
> > > > > > > > > > > > > >> for a
> > > > > > > > > > > > > >> > > > > > > > multi-tenant setting. Then in the
> > > > describe
> > > > > > topic
> > > > > > > > > > > request
> > > > > > > > > > > > > we
> > > > > > > > > > > > > >> can
> > > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > > >> > > > > > > > on regex of the metadata field.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: As discussed it is very
> > interesting
> > > > but
> > > > > > can
> > > > > > > > be
> > > > > > > > > > > > > >> implemented
> > > > > > > > > > > > > >> > > later
> > > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > > >> > > > > > > > we have some basic functionality
> > > there.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > 15. Today all the admin operations
> > are
> > > > > > async in
> > > > > > > > > the
> > > > > > > > > > > sense
> > > > > > > > > > > > > >> that
> > > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > > >> > > > > > > > return once it is written in ZK,
> and
> > > > that
> > > > > > is why
> > > > > > > > > we
> > > > > > > > > > > need
> > > > > > > > > > > > > >> extra
> > > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > > >> > > > > > > > like
> testUtil.waitForTopicCreated()
> > /
> > > > > verify
> > > > > > > > > > partition
> > > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > > >> > > > > > > > request, etc. With admin requests
> we
> > > > could
> > > > > > add a
> > > > > > > > > > flag
> > > > > > > > > > > to
> > > > > > > > > > > > > >> enable /
> > > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > > >> > > > > > > > synchronous requests; when it is
> > > turned
> > > > > on,
> > > > > > the
> > > > > > > > > > > response
> > > > > > > > > > > > > >> will not
> > > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > > >> > > > > > > > until the request has been
> > completed.
> > > > And
> > > > > > for
> > > > > > > > > async
> > > > > > > > > > > > > >> requests we
> > > > > > > > > > > > > >> > > can
> > > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > > >> > > > > > > > "token" field in the response, and
> > > then
> > > > > only
> > > > > > > > need
> > > > > > > > > a
> > > > > > > > > > > > > general
> > > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > > >> > > > > > > > verification request" with the
> given
> > > > token
> > > > > > to
> > > > > > > > > check
> > > > > > > > > > > if the
> > > > > > > > > > > > > >> async
> > > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: I see your point. My idea was
> to
> > > > > provide
> > > > > > > > > specific
> > > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > > >> > > > > > > > long running request, where
> needed.
> > We
> > > > can
> > > > > > do it
> > > > > > > > > the
> > > > > > > > > > > way
> > > > > > > > > > > > > you
> > > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > > >> > > > > > > > concern is that introducing a
> token
> > we
> > > > > again
> > > > > > > > will
> > > > > > > > > > make
> > > > > > > > > > > > > >> schema
> > > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > > >> > > > > > > > to do similar thing introducing
> > single
> > > > > > > > > AdminRequest
> > > > > > > > > > > for
> > > > > > > > > > > > > all
> > > > > > > > > > > > > >> topic
> > > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > > >> > > > > > > > this idea because we wanted to
> have
> > > > schema
> > > > > > > > > defined.
> > > > > > > > > > So
> > > > > > > > > > > > > this
> > > > > > > > > > > > > >> is
> > > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > > >> > > > > > > > a) have fixed schema but introduce
> > > each
> > > > > > time new
> > > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > > > > >> > > > > > > > b) use one request for
> verification
> > > but
> > > > > > > > generalize
> > > > > > > > > > it
> > > > > > > > > > > with
> > > > > > > > > > > > > >> token
> > > > > > > > > > > > > >> > > > > > > > I'm fine with whatever decision
> > > > community
> > > > > > come
> > > > > > > > to.
> > > > > > > > > > > Just
> > > > > > > > > > > > > let
> > > > > > > > > > > > > >> me
> > > > > > > > > > > > > >> > > know
> > > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > > >> > > > > > > > 16. Specifically for ownership, I
> > > think
> > > > > the
> > > > > > plan
> > > > > > > > > is
> > > > > > > > > > > to add
> > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > > >> > > > > > > > like you are describing ACL) via
> an
> > > > > external
> > > > > > > > > system
> > > > > > > > > > > > > (Argus,
> > > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > > >> > > > > > > > I remember KIP-11 described this,
> > but
> > > I
> > > > > > can't
> > > > > > > > find
> > > > > > > > > > > the KIP
> > > > > > > > > > > > > >> any
> > > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > A: Okay, no problem. Not sure
> though
> > > how
> > > > > we
> > > > > > are
> > > > > > > > > > going
> > > > > > > > > > > to
> > > > > > > > > > > > > >> handle
> > > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > > >> > > > > > > > will be committed first and
> include
> > > > > changes
> > > > > > to
> > > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > > >> > > > > > > > Anyway, I added this note to "Open
> > > > > > Questions"
> > > > > > > > > > section
> > > > > > > > > > > so
> > > > > > > > > > > > > we
> > > > > > > > > > > > > >> don't
> > > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM,
> > > Andrii
> > > > > > > > > Biletskyi <
> > > > > > > > > > > > > >> > > > > > > > andrii.biletskyi@stealth.ly>
> wrote:
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > > Today I uploaded the patch that
> > > covers
> > > > > > some of
> > > > > > > > > the
> > > > > > > > > > > > > >> discussed
> > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf optional type
> > > > > > > > > > > > > >> > > > > > > > > - switched to java protocol
> > > > definitions
> > > > > > > > > > > > > >> > > > > > > > > - simplified messages
> (normalized
> > > > > configs,
> > > > > > > > > removed
> > > > > > > > > > > topic
> > > > > > > > > > > > > >> marked
> > > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > > I also updated the KIP-4 with
> > > > respective
> > > > > > > > changes
> > > > > > > > > > and
> > > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > > >> > > > > > > > > - Batch Admin Operations ->
> > updated
> > > > Wire
> > > > > > > > > Protocol
> > > > > > > > > > > schema
> > > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > > >> > > > > > > > > - Remove ClusterMetadata ->
> > changed
> > > to
> > > > > > extend
> > > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > > >> > > > > > > > > - Admin Client -> updated my
> > initial
> > > > > > proposal
> > > > > > > > to
> > > > > > > > > > > reflect
> > > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > > >> > > > > > > > > - Error codes -> proposed
> > > fine-grained
> > > > > > error
> > > > > > > > > code
> > > > > > > > > > > > > instead
> > > > > > > > > > > > > >> of
> > > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > > I will also send a separate
> email
> > to
> > > > > > cover all
> > > > > > > > > > > comments
> > > > > > > > > > > > > >> from
> > > > > > > > > > > > > >> > > this
> > > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM,
> > > Gwen
> > > > > > Shapira
> > > > > > > > <
> > > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >>
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > > >> > > > > > > > >> It actually specifies changes
> to
> > > the
> > > > > > Metadata
> > > > > > > > > > > protocol,
> > > > > > > > > > > > > >> so
> > > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > > >> > > > > > > > >> both KIPs are consistent in
> this
> > > > regard
> > > > > > will
> > > > > > > > be
> > > > > > > > > > > good.
> > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21
> PM,
> > > > Gwen
> > > > > > > > Shapira
> > > > > > > > > <
> > > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > >> > > > > > > > >> > Specifically for ownership, I
> > > think
> > > > > the
> > > > > > > > plan
> > > > > > > > > is
> > > > > > > > > > > to
> > > > > > > > > > > > > add
> > > > > > > > > > > > > >> ACL
> > > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > > >> > > > > > > > >> > like you are describing ACL)
> > via
> > > an
> > > > > > > > external
> > > > > > > > > > > system
> > > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > > >> > > > > > > > >> > I remember KIP-11 described
> > this,
> > > > > but I
> > > > > > > > can't
> > > > > > > > > > > find
> > > > > > > > > > > > > the
> > > > > > > > > > > > > >> KIP
> > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > >> > > > > > > > >> > Regardless, I think KIP-4
> > focuses
> > > > on
> > > > > > > > getting
> > > > > > > > > > > > > >> information
> > > > > > > > > > > > > >> > > that
> > > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > > >> > > > > > > > >> > exists from Kafka brokers,
> not
> > on
> > > > > > adding
> > > > > > > > > > > information
> > > > > > > > > > > > > >> that
> > > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > > >> > > > > > > > >> > should exist but doesn't yet?
> > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37
> > AM,
> > > > > > Guozhang
> > > > > > > > > Wang
> > > > > > > > > > <
> > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > >> > > > > > > > >> >> Just want to elaborate a bit
> > > more
> > > > on
> > > > > > the
> > > > > > > > > > > > > create-topic
> > > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > > >> > > > > > > > >> >> describe-topic based on
> > config /
> > > > > > metadata
> > > > > > > > in
> > > > > > > > > > my
> > > > > > > > > > > > > >> previous
> > > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The main
> > > motivation
> > > > > is
> > > > > > to
> > > > > > > > > have
> > > > > > > > > > > some
> > > > > > > > > > > > > >> sort of
> > > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > > >> > > > > > > > >> >> mechanisms, which I think is
> > > quite
> > > > > > > > important
> > > > > > > > > > in
> > > > > > > > > > > a
> > > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > > >> > > > > > > > >> >> architecture: today anyone
> can
> > > > > create
> > > > > > > > topics
> > > > > > > > > > in
> > > > > > > > > > > a
> > > > > > > > > > > > > >> shared
> > > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > > >> > > > > > > > >> >> there is no concept or
> > > "ownership"
> > > > > of
> > > > > > > > topics
> > > > > > > > > > > that
> > > > > > > > > > > > > are
> > > > > > > > > > > > > >> > > created
> > > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > > >> > > > > > > > >> >> users. For example, at
> > LinkedIn
> > > we
> > > > > > > > basically
> > > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > > >> > > > > > > > >> >> some casual topic name
> prefix,
> > > > which
> > > > > > is a
> > > > > > > > > bit
> > > > > > > > > > > > > awkward
> > > > > > > > > > > > > >> and
> > > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > > >> > > > > > > > >> >> we scale our customers. It
> > would
> > > > be
> > > > > > great
> > > > > > > > to
> > > > > > > > > > use
> > > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics that is
> > > > created
> > > > > > by me.
> > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics whose
> > > > retention
> > > > > > time
> > > > > > > > is
> > > > > > > > > > > > > overriden
> > > > > > > > > > > > > >> to X.
> > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics whose
> > > writable
> > > > > > group
> > > > > > > > > > include
> > > > > > > > > > > > > user
> > > > > > > > > > > > > >> Y
> > > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > >> > > > > > > > >> >> One possible way to achieve
> > this
> > > > is
> > > > > to
> > > > > > > > add a
> > > > > > > > > > > > > metadata
> > > > > > > > > > > > > >> file
> > > > > > > > > > > > > >> > > in
> > > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > > >> > > > > > > > >> >> create-topic request, whose
> > > value
> > > > > will
> > > > > > > > also
> > > > > > > > > be
> > > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > > >> > > as
> > > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > > >> > > > > > > > >> >> topic; then describe-topics
> > can
> > > > > > choose to
> > > > > > > > > > batch
> > > > > > > > > > > > > topics
> > > > > > > > > > > > > >> > > based
> > > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > > >> > > > > > > > >> >> regex, 2) config K-V
> matching,
> > > 3)
> > > > > > metadata
> > > > > > > > > > > regex,
> > > > > > > > > > > > > etc.
> > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37
> > PM,
> > > > > > Guozhang
> > > > > > > > > Wang
> > > > > > > > > > <
> > > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for the updated
> wiki.
> > A
> > > > few
> > > > > > > > comments
> > > > > > > > > > > below:
> > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error description in
> > > > response: I
> > > > > > think
> > > > > > > > > if
> > > > > > > > > > > some
> > > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > > >> > > > > > > > >> >>> several different error
> cases
> > > > then
> > > > > we
> > > > > > > > > should
> > > > > > > > > > > really
> > > > > > > > > > > > > >> change
> > > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > > >> > > > > > > > >> >>> codes. In general the
> > errorCode
> > > > > > itself
> > > > > > > > > would
> > > > > > > > > > be
> > > > > > > > > > > > > >> precise
> > > > > > > > > > > > > >> > > and
> > > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > > >> > > > > > > > >> >>> describing the server side
> > > > errors.
> > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic request:
> it
> > > > would
> > > > > > be
> > > > > > > > > great
> > > > > > > > > > > to go
> > > > > > > > > > > > > >> beyond
> > > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex for this
> > > > request.
> > > > > > For
> > > > > > > > > > > example, a
> > > > > > > > > > > > > >> very
> > > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > > >> > > > > > > > >> >>> the topic command is to
> list
> > > all
> > > > > > topics
> > > > > > > > > whose
> > > > > > > > > > > > > config
> > > > > > > > > > > > > >> A's
> > > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex then we
> have
> > > to
> > > > > > first
> > > > > > > > > > retrieve
> > > > > > > > > > > > > >> __all__
> > > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > > >> > > > > > > > >> >>> description info and then
> > > filter
> > > > at
> > > > > > the
> > > > > > > > > > client
> > > > > > > > > > > end,
> > > > > > > > > > > > > >> which
> > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in create
> > topic:
> > > > > this
> > > > > > is
> > > > > > > > > > > related to
> > > > > > > > > > > > > >> the
> > > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can add another
> > > metadata
> > > > > > K-V or
> > > > > > > > > > just a
> > > > > > > > > > > > > >> metadata
> > > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > > >> > > > > > > > >> >>> with config K-V in create
> > topic
> > > > > like
> > > > > > we
> > > > > > > > did
> > > > > > > > > > for
> > > > > > > > > > > > > >> offset
> > > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > > >> > > > > > > > >> >>> field can be quite useful
> in
> > > > > storing
> > > > > > > > > > > information
> > > > > > > > > > > > > like
> > > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > > >> > > > > > > > >> >>> who issue the create
> command,
> > > > etc,
> > > > > > which
> > > > > > > > is
> > > > > > > > > > > quite
> > > > > > > > > > > > > >> > > important
> > > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant setting. Then
> in
> > > the
> > > > > > > > describe
> > > > > > > > > > > topic
> > > > > > > > > > > > > >> request
> > > > > > > > > > > > > >> > > we
> > > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > > >> > > > > > > > >> >>> on regex of the metadata
> > field.
> > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all the admin
> > > operations
> > > > > are
> > > > > > > > async
> > > > > > > > > > in
> > > > > > > > > > > the
> > > > > > > > > > > > > >> sense
> > > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > > >> > > > > > > > >> >>> return once it is written
> in
> > > ZK,
> > > > > and
> > > > > > that
> > > > > > > > > is
> > > > > > > > > > > why we
> > > > > > > > > > > > > >> need
> > > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > > testUtil.waitForTopicCreated() /
> > > > > > > > > verify
> > > > > > > > > > > > > >> partition
> > > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > > >> > > > > > > > >> >>> request, etc. With admin
> > > requests
> > > > > we
> > > > > > > > could
> > > > > > > > > > add
> > > > > > > > > > > a
> > > > > > > > > > > > > >> flag to
> > > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > > >> > > > > > > > >> >>> synchronous requests; when
> it
> > > is
> > > > > > turned
> > > > > > > > on,
> > > > > > > > > > the
> > > > > > > > > > > > > >> response
> > > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > > >> > > > > > > > >> >>> until the request has been
> > > > > > completed. And
> > > > > > > > > for
> > > > > > > > > > > async
> > > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > > >> > > > > > > > >> >>> "token" field in the
> > response,
> > > > and
> > > > > > then
> > > > > > > > > only
> > > > > > > > > > > need a
> > > > > > > > > > > > > >> > > general
> > > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > > >> > > > > > > > >> >>> verification request" with
> > the
> > > > > given
> > > > > > > > token
> > > > > > > > > to
> > > > > > > > > > > check
> > > > > > > > > > > > > >> if the
> > > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for extending
> Metadata
> > > > > request
> > > > > > to
> > > > > > > > > > include
> > > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > > >> > > > > > > > >> >>> information, and then we
> can
> > > > remove
> > > > > > the
> > > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at
> 10:23
> > > AM,
> > > > > Joel
> > > > > > > > > Koshy <
> > > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for sending that
> out
> > > Joe
> > > > -
> > > > > I
> > > > > > > > don't
> > > > > > > > > > > think I
> > > > > > > > > > > > > >> will be
> > > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so if notes can
> be
> > > > sent
> > > > > > out
> > > > > > > > > > > afterward
> > > > > > > > > > > > > that
> > > > > > > > > > > > > >> > > would
> > > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at
> > > > 09:16:13AM
> > > > > > > > -0800,
> > > > > > > > > > Gwen
> > > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for sending this
> > out
> > > > Joe.
> > > > > > > > Looking
> > > > > > > > > > > forward
> > > > > > > > > > > > > >> to
> > > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at
> > 6:46
> > > > AM,
> > > > > > Joe
> > > > > > > > > Stein
> > > > > > > > > > <
> > > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just sent out a
> > > > google
> > > > > > > > hangout
> > > > > > > > > > > invite
> > > > > > > > > > > > > to
> > > > > > > > > > > > > >> all
> > > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I found
> working
> > > on
> > > > a
> > > > > > KIP.
> > > > > > > > If
> > > > > > > > > I
> > > > > > > > > > > missed
> > > > > > > > > > > > > >> anyone
> > > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me know and can
> > update
> > > > it,
> > > > > > np.
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > > We should do this
> every
> > > > > Tuesday
> > > > > > @
> > > > > > > > 2pm
> > > > > > > > > > > Eastern
> > > > > > > > > > > > > >> Time.
> > > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to make a google
> > > > account
> > > > > > so we
> > > > > > > > > can
> > > > > > > > > > > manage
> > > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > > >> > > >
> > > > > > > > > > > > > >> > >
> > > > > > > > > > > > > >>
> > > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > > in progress and
> related
> > > JIRA
> > > > > > that
> > > > > > > > are
> > > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015
> at
> > > 2:59
> > > > > > PM, Jay
> > > > > > > > > > > Kreps <
> > > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on Google
> > > > hangouts
> > > > > > that
> > > > > > > > > will
> > > > > > > > > > > also
> > > > > > > > > > > > > >> record
> > > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015
> at
> > > > 11:49
> > > > > > AM,
> > > > > > > > > Jeff
> > > > > > > > > > > > > Holoman
> > > > > > > > > > > > > >> <
> > > > > > > > > > > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're happy to send
> > > out a
> > > > > > Webex
> > > > > > > > for
> > > > > > > > > > > this
> > > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > > >> > > We
> > > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions if there
> is
> > > > > > interest and
> > > > > > > > > > > publish
> > > > > > > > > > > > > >> them
> > > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24,
> 2015
> > at
> > > > > > 11:28 AM,
> > > > > > > > > Jay
> > > > > > > > > > > > > Kreps <
> > > > > > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try to get
> > the
> > > > > > technical
> > > > > > > > > > > hang-ups
> > > > > > > > > > > > > >> sorted
> > > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there is some
> > benefit
> > > > to
> > > > > > live
> > > > > > > > > > > discussion
> > > > > > > > > > > > > vs
> > > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post
> > instructions
> > > > and
> > > > > > give
> > > > > > > > > > > ourselves a
> > > > > > > > > > > > > >> few
> > > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that
> > time
> > > > > would
> > > > > > work
> > > > > > > > > for
> > > > > > > > > > > > > >> me...any
> > > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24,
> > 2015
> > > at
> > > > > > 8:18
> > > > > > > > AM,
> > > > > > > > > > Joe
> > > > > > > > > > > > > Stein
> > > > > > > > > > > > > >> <
> > > > > > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly would be
> > > great
> > > > > > maybe
> > > > > > > > > like
> > > > > > > > > > > every
> > > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't mind
> > google
> > > > > > hangout
> > > > > > > > but
> > > > > > > > > > > there
> > > > > > > > > > > > > is
> > > > > > > > > > > > > >> > > always
> > > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we know the
> > apache
> > > > irc
> > > > > > > > channel
> > > > > > > > > > > works.
> > > > > > > > > > > > > We
> > > > > > > > > > > > > >> can
> > > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes? We can
> pull
> > > > > > transcripts
> > > > > > > > > too
> > > > > > > > > > > and
> > > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it helpful for
> > > > things.
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24,
> > > 2015
> > > > at
> > > > > > 11:10
> > > > > > > > > AM,
> > > > > > > > > > > Jay
> > > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked
> > about
> > > > > > doing a
> > > > > > > > > > Google
> > > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > generalizing
> > > that a
> > > > > > little
> > > > > > > > > > > > > further...I
> > > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > everyone
> > > spending a
> > > > > > > > > reasonable
> > > > > > > > > > > chunk
> > > > > > > > > > > > > of
> > > > > > > > > > > > > >> > > their
> > > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync up once
> a
> > > > week.
> > > > > I
> > > > > > > > think
> > > > > > > > > we
> > > > > > > > > > > could
> > > > > > > > > > > > > >> use
> > > > > > > > > > > > > >> > > time
> > > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make
> > sure
> > > we
> > > > > > are on
> > > > > > > > > top
> > > > > > > > > > of
> > > > > > > > > > > > > code
> > > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We can make
> it
> > > > > publicly
> > > > > > > > > > > available so
> > > > > > > > > > > > > >> that
> > > > > > > > > > > > > >> > > any
> > > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any interest
> in
> > > > doing
> > > > > > this?
> > > > > > > > > If
> > > > > > > > > > so
> > > > > > > > > > > > > I'll
> > > > > > > > > > > > > >> try
> > > > > > > > > > > > > >> > > to
> > > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb
> 24,
> > > > 2015
> > > > > at
> > > > > > > > 3:57
> > > > > > > > > > AM,
> > > > > > > > > > > > > Andrii
> > > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > > wrote:
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > I've
> updated
> > > KIP
> > > > > > page,
> > > > > > > > > fixed
> > > > > > > > > > /
> > > > > > > > > > > > > >> aligned
> > > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > >
> > > ...
> > >
> > > [Message clipped]
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Guozhang Wang <wa...@gmail.com>.
I think while loop is fine for supporting blocking, just that we need to
add back off to avoid bombarding brokers with DescribeTopic requests.

Also I have linked KAFKA-1125
<https://issues.apache.org/jira/browse/KAFKA-1125> to your proposal, and
when KAFKA-1694 is done this ticket can also be closed.

Guozhang

On Fri, Mar 20, 2015 at 9:41 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Great.
> I want to elaborate this a bit more, to see we are on the same page
> concerning the client code.
>
> So with all topic commands being async a client (AdminClient in our
> case or any other other client people would like to implement) to support
> a blocking operation (which seems to be a natural use-case e.g. for topic
> creation): would have to do:
> 1. issue CreateTopicRequest
> 2. if successful, in a "while" loop send DescribeTopicRequest and
> break the loop once all topics are returned in response (or upon timeout).
> 3. if unsuccessful throw exception
> Would it be okay?
>
> Thanks,
> Andrii Biletskyi
>
>
> On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Andrii,
> >
> > I think you are right. It seems that only ReassignPartitions needs a
> > separate verification request.
> >
> > Thanks,
> >
> > Jun
> >
> > On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Guys,
> > > I like this idea too. Let's stick with that. I'll update KIP
> accordingly.
> > >
> > > I was also thinking we can avoid adding dedicated status check
> > > requests for topic commands. - We have everything in DescribeTopic
> > > for that! E.g.:
> > > User issued CreateTopic - to check the status client sends
> DescribeTopic
> > > and checks whether is something returned for that topic. The same for
> > > alteration, deletion.
> > > Btw, PreferredReplica status can be also checked with
> > DescribeTopicRequest
> > > (head of assigned replicas list == leader).
> > > For ReassignPartitions as discussed we'll need to have a separate
> > Verify...
> > > request.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > >
> > > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > > > +1 on broker writing to ZK for async handling. I was thinking that in
> > the
> > > > end state the admin requests would be eventually sent to controller
> > > either
> > > > through re-routing or clients discovering them, instead of letting
> > > > controller listen on ZK admin path. But thinking about it a second
> > time,
> > > I
> > > > think it is actually simpler to let controller manage
> > > > incoming queued-up admin requests through ZK.
> > > >
> > > > Guozhang
> > > >
> > > >
> > > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > > >
> > > > > +1 as well. I think it helps to keep the rerouting approach
> > orthogonal
> > > > > to this KIP.
> > > > >
> > > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps wrote:
> > > > > > I'm +1 on Jun's suggestion as long as it can work for all the
> > > requests.
> > > > > >
> > > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Andrii,
> > > > > > >
> > > > > > > I think we agreed on the following.
> > > > > > >
> > > > > > > (a) Admin requests can be sent to and handled by any broker.
> > > > > > > (b) Admin requests are processed asynchronously, at least for
> > now.
> > > > > That is,
> > > > > > > when the client gets a response, it just means that the request
> > is
> > > > > > > initiated, but not necessarily completed. Then, it's up to the
> > > client
> > > > > to
> > > > > > > issue another request to check the status for completion.
> > > > > > >
> > > > > > > To support (a), we were thinking of doing request forwarding to
> > the
> > > > > > > controller (utilizing KAFKA-1912). I am making an alternative
> > > > proposal.
> > > > > > > Basically, the broker can just write to ZooKeeper to inform the
> > > > > controller
> > > > > > > about the request. For example, to handle
> partitionReassignment,
> > > the
> > > > > broker
> > > > > > > will just write the requested partitions to
> > > > /admin/reassign_partitions
> > > > > > > (like what AdminUtils currently does) and then send a response
> to
> > > the
> > > > > > > client. This shouldn't take long and the implementation will be
> > > > simpler
> > > > > > > than forwarding the requests to the controller through RPC.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >
> > > > > > > > Jun,
> > > > > > > >
> > > > > > > > I might be wrong but didn't we agree we will let any broker
> > from
> > > > the
> > > > > > > > cluster handle *long-running* admin requests (at this time
> > > > > > > preferredReplica
> > > > > > > > and
> > > > > > > > reassignPartitions), via zk admin path. Thus CreateTopics etc
> > > > should
> > > > > be
> > > > > > > > sent
> > > > > > > > only to the controller.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Andrii Biletskyi
> > > > > > > >
> > > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > > > > >
> > > > > > > > > Joel, Andril,
> > > > > > > > >
> > > > > > > > > I think we agreed that those admin requests can be issued
> to
> > > any
> > > > > > > broker.
> > > > > > > > > Because of that, there doesn't seem to be a strong need to
> > know
> > > > the
> > > > > > > > > controller. So, perhaps we can proceed by not making any
> > change
> > > > to
> > > > > the
> > > > > > > > > format of TMR right now. When we start using create topic
> > > request
> > > > > in
> > > > > > > the
> > > > > > > > > producer, we will need a new version of TMR that doesn't
> > > trigger
> > > > > auto
> > > > > > > > topic
> > > > > > > > > creation. But that can be done later.
> > > > > > > > >
> > > > > > > > > As a first cut implementation, I think the broker can just
> > > write
> > > > > to ZK
> > > > > > > > > directly for
> > > > > > > > >
> > > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > > requests, instead of forwarding them to the controller.
> This
> > > will
> > > > > > > > simplify
> > > > > > > > > the implementation on the broker side.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> > > > jjkoshy.w@gmail.com>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > > >
> > > > > > > > > > For (1) yes we will circle back on that shortly after
> > syncing
> > > > up
> > > > > in
> > > > > > > > > > person. I think it is close to getting committed although
> > > > > development
> > > > > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > > > > >
> > > > > > > > > > There is one more item we covered at the hangout. i.e.,
> > > whether
> > > > > we
> > > > > > > > > > want to add the coordinator to the topic metadata
> response
> > or
> > > > > provide
> > > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > > >
> > > > > > > > > > There are two reasons I think we should try and avoid
> > adding
> > > > the
> > > > > > > > > > field:
> > > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > > - If we finally do request rerouting in Kafka then the
> > field
> > > > > would
> > > > > > > add
> > > > > > > > > >   little to no value. (It still helps to have a separate
> > > > > > > > > >   ClusterMetadataRequest to query for cluster-wide
> > > information
> > > > > such
> > > > > > > as
> > > > > > > > > >   'which broker is the controller?' as Joe mentioned.)
> > > > > > > > > >
> > > > > > > > > > I think it would be cleaner to have an explicit
> > > > > > > ClusterMetadataRequest
> > > > > > > > > > that you can send to any broker in order to obtain the
> > > > controller
> > > > > > > (and
> > > > > > > > > > in the future possibly other cluster-wide information). I
> > > think
> > > > > the
> > > > > > > > > > main argument against doing this and instead adding it to
> > the
> > > > > topic
> > > > > > > > > > metadata response was convenience - i.e., you don't have
> to
> > > > > discover
> > > > > > > > > > the controller in advance. However, I don't see much
> actual
> > > > > > > > > > benefit/convenience in this and in fact think it is a
> > > > non-issue.
> > > > > Let
> > > > > > > > > > me know if I'm overlooking something here.
> > > > > > > > > >
> > > > > > > > > > As an example, say we need to initiate partition
> > reassignment
> > > > by
> > > > > > > > > > issuing the new ReassignPartitionsRequest to the
> controller
> > > > > (assume
> > > > > > > we
> > > > > > > > > > already have the desired manual partition assignment).
> If
> > we
> > > > > are to
> > > > > > > > > > augment topic metadata response then the flow be
> something
> > > like
> > > > > this
> > > > > > > :
> > > > > > > > > >
> > > > > > > > > > - Issue topic metadata request to any broker (and
> discover
> > > the
> > > > > > > > > >   controller
> > > > > > > > > > - Connect to controller if required (i.e., if the broker
> > > above
> > > > !=
> > > > > > > > > >   controller)
> > > > > > > > > > - Issue the partition reassignment request to the
> > controller.
> > > > > > > > > >
> > > > > > > > > > With an explicit cluster metadata request it would be:
> > > > > > > > > > - Issue cluster metadata request to any broker
> > > > > > > > > > - Connect to controller if required (i.e., if the broker
> > > above
> > > > !=
> > > > > > > > > >   controller)
> > > > > > > > > > - Issue the partition reassignment request
> > > > > > > > > >
> > > > > > > > > > So it seems to add little practical value and bloats
> topic
> > > > > metadata
> > > > > > > > > > response with an irrelevant detail.
> > > > > > > > > >
> > > > > > > > > > The other angle to this is the following - is it a matter
> > of
> > > > > naming?
> > > > > > > > > > Should we just rename topic metadata request/response to
> > just
> > > > > > > > > > MetadataRequest/Response and add cluster metadata to it?
> By
> > > > that
> > > > > same
> > > > > > > > > > token should we also allow querying for the consumer
> > > > coordinator
> > > > > (and
> > > > > > > > > > in future transaction coordinator) as well? This leads
> to a
> > > > > bloated
> > > > > > > > > > request which isn't very appealing and altogether
> > confusing.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Joel
> > > > > > > > > >
> > > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > > > > > > > > > > Andri,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for the summary.
> > > > > > > > > > >
> > > > > > > > > > > 1. I just realized that in order to start working on
> > > > > KAFKA-1927, we
> > > > > > > > > will
> > > > > > > > > > > need to merge the changes to OffsetCommitRequest (from
> > > 0.8.2)
> > > > > to
> > > > > > > > trunk.
> > > > > > > > > > > This is planned to be done as part of KAFKA-1634. So,
> we
> > > will
> > > > > need
> > > > > > > > > > Guozhang
> > > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > > >
> > > > > > > > > > > 2. Thinking about this a bit more, if the semantic of
> > those
> > > > > "write"
> > > > > > > > > > > requests is async (i.e., after the client gets a
> > response,
> > > it
> > > > > just
> > > > > > > > > means
> > > > > > > > > > > that the operation is initiated, but not necessarily
> > > > > completed), we
> > > > > > > > > don't
> > > > > > > > > > > really need to forward the requests to the controller.
> > > > > Instead, the
> > > > > > > > > > > receiving broker can just write the operation to ZK as
> > the
> > > > > admin
> > > > > > > > > command
> > > > > > > > > > > line tool previously does. This will simplify the
> > > > > implementation.
> > > > > > > > > > >
> > > > > > > > > > > 8. There is another implementation detail for describe
> > > topic.
> > > > > > > > Ideally,
> > > > > > > > > we
> > > > > > > > > > > want to read the topic config from the broker cache,
> > > instead
> > > > of
> > > > > > > > > > ZooKeeper.
> > > > > > > > > > > Currently, every broker reads the topic-level config
> for
> > > all
> > > > > > > topics.
> > > > > > > > > > > However, it ignores those for topics not hosted on
> > itself.
> > > > So,
> > > > > we
> > > > > > > may
> > > > > > > > > > need
> > > > > > > > > > > to change TopicConfigManager a bit so that it caches
> the
> > > > > configs
> > > > > > > for
> > > > > > > > > all
> > > > > > > > > > > topics.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > >
> > > > > > > > > > > Jun
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Guys,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > > Here are the actions points:
> > > > > > > > > > > >
> > > > > > > > > > > > 1. Q: Get rid of all scala requests objects, use java
> > > > > protocol
> > > > > > > > > > definitions.
> > > > > > > > > > > >     A: Gwen kindly took that (KAFKA-1927). It's
> > important
> > > > to
> > > > > > > speed
> > > > > > > > up
> > > > > > > > > > > > review procedure
> > > > > > > > > > > >          there since this ticket blocks other
> important
> > > > > changes.
> > > > > > > > > > > >
> > > > > > > > > > > > 2. Q: Generic re-reroute facility vs client
> maintaining
> > > > > cluster
> > > > > > > > > state.
> > > > > > > > > > > >     A: Jay has added pseudo code to KAFKA-1912 - need
> > to
> > > > > consider
> > > > > > > > > > whether
> > > > > > > > > > > > this will be
> > > > > > > > > > > >         easy to implement as a server-side feature
> > > > (comments
> > > > > are
> > > > > > > > > > > > welcomed!).
> > > > > > > > > > > >
> > > > > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > > > > >     A: This might be useful for clients, add this to
> > > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > > (already in KIP).
> > > > > > > > > > > >
> > > > > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > > > > >     A: I will add proposed by Jun solution (using
> > > clientId
> > > > > for
> > > > > > > > that)
> > > > > > > > > > to the
> > > > > > > > > > > > KIP.
> > > > > > > > > > > >
> > > > > > > > > > > > 5. Q: Bumping new versions of TMR vs grabbing all
> > > protocol
> > > > > > > changes
> > > > > > > > in
> > > > > > > > > > one
> > > > > > > > > > > > version.
> > > > > > > > > > > >     A: It was decided to try to gather all changes to
> > > > > protocol
> > > > > > > > > (before
> > > > > > > > > > > > release).
> > > > > > > > > > > >         In case of TMR it worth checking: KAFKA-2020
> > and
> > > > > KIP-13
> > > > > > > > > > (quotas)
> > > > > > > > > > > >
> > > > > > > > > > > > 6. Q: JSON lib is needed to deserialize user's input
> in
> > > CLI
> > > > > tool.
> > > > > > > > > > > >     A: Use jackson for that, /tools project is a
> > separate
> > > > > jar so
> > > > > > > > > > shouldn't
> > > > > > > > > > > > be a big deal.
> > > > > > > > > > > >
> > > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs generic status
> check
> > > > > command.
> > > > > > > > > > > >      A: For long-running requests like reassign
> > > partitions
> > > > > > > > *progress*
> > > > > > > > > > check
> > > > > > > > > > > > request is useful,
> > > > > > > > > > > >          it makes sense to introduce it.
> > > > > > > > > > > >
> > > > > > > > > > > >  Please add, correct me if I missed something.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > >
> > > > > > > > > > > > > Joel,
> > > > > > > > > > > > >
> > > > > > > > > > > > > You are right, I removed ClusterMetadata because we
> > > have
> > > > > > > > partially
> > > > > > > > > > > > > what we need in TopicMetadata. Also, as Jay pointed
> > out
> > > > > > > earlier,
> > > > > > > > we
> > > > > > > > > > > > > would like to have "orthogonal" API, but at the
> same
> > > time
> > > > > we
> > > > > > > need
> > > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > > >
> > > > > > > > > > > > > But I like your idea and even have some other
> > arguments
> > > > for
> > > > > > > this
> > > > > > > > > > option:
> > > > > > > > > > > > > There is also DescribeTopicRequest which was
> proposed
> > > in
> > > > > this
> > > > > > > > KIP,
> > > > > > > > > > > > > it returns topic configs, partitions, replication
> > > factor
> > > > > plus
> > > > > > > > > > partition
> > > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > > leader replica. The later part is really already
> > there
> > > in
> > > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > > So again we'll have to add stuff to TMR, not to
> > > duplicate
> > > > > some
> > > > > > > > info
> > > > > > > > > > in
> > > > > > > > > > > > > newly added requests. However, this way we'll end
> up
> > > with
> > > > > > > > "monster"
> > > > > > > > > > > > > request which returns cluster metadata, topic
> > > replication
> > > > > and
> > > > > > > > > config
> > > > > > > > > > info
> > > > > > > > > > > > > plus partition replication data. Seems logical to
> > split
> > > > > TMR to
> > > > > > > > > > > > > - ClusterMetadata (brokers + controller, maybe smth
> > > else)
> > > > > > > > > > > > > - TopicMetadata (topic info + partition details)
> > > > > > > > > > > > > But since current TMR is involved in lots of places
> > > > > (including
> > > > > > > > > > network
> > > > > > > > > > > > > client,
> > > > > > > > > > > > > as I understand) this might be very serious change
> > and
> > > it
> > > > > > > > probably
> > > > > > > > > > makes
> > > > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > > > >
> > > > > > > > > > > > > Thanks,
> > > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <
> > > > > > > jjkoshy.w@gmail.com
> > > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > >> I may be missing some context but hopefully this
> > will
> > > > > also be
> > > > > > > > > > covered
> > > > > > > > > > > > >> today: I thought the earlier proposal where there
> > was
> > > an
> > > > > > > > explicit
> > > > > > > > > > > > >> ClusterMetadata request was clearer and explicit.
> > > During
> > > > > the
> > > > > > > > > course
> > > > > > > > > > of
> > > > > > > > > > > > >> this thread I think the conclusion was that the
> main
> > > > need
> > > > > was
> > > > > > > > for
> > > > > > > > > > > > >> controller information and that can be rolled into
> > the
> > > > > topic
> > > > > > > > > > metadata
> > > > > > > > > > > > >> response but that seems a bit irrelevant to topic
> > > > > metadata.
> > > > > > > > FWIW I
> > > > > > > > > > > > >> think the full broker-list is also irrelevant to
> > topic
> > > > > > > metadata,
> > > > > > > > > but
> > > > > > > > > > > > >> it is already there and in use. I think there is
> > still
> > > > > room
> > > > > > > for
> > > > > > > > an
> > > > > > > > > > > > >> explicit ClusterMetadata request since there may
> be
> > > > other
> > > > > > > > > > > > >> cluster-level information that we may want to add
> > over
> > > > > time
> > > > > > > (and
> > > > > > > > > > that
> > > > > > > > > > > > >> have nothing to do with topic metadata).
> > > > > > > > > > > > >>
> > > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii
> > > > Biletskyi
> > > > > > > > wrote:
> > > > > > > > > > > > >> > Jun,
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > 101. Okay, if you say that such use case is
> > > > important. I
> > > > > > > also
> > > > > > > > > > think
> > > > > > > > > > > > >> > using clientId for these purposes is fine - if
> we
> > > > > already
> > > > > > > have
> > > > > > > > > > this
> > > > > > > > > > > > >> field
> > > > > > > > > > > > >> > as part of all Wire protocol messages, why not
> use
> > > > that.
> > > > > > > > > > > > >> > I will update KIP-4 page if nobody has other
> ideas
> > > > > (which
> > > > > > > may
> > > > > > > > > > come up
> > > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP accordingly. I
> > > think
> > > > > we can
> > > > > > > > add
> > > > > > > > > > new,
> > > > > > > > > > > > >> > fine-grained error codes if some error code
> > received
> > > > in
> > > > > > > > specific
> > > > > > > > > > case
> > > > > > > > > > > > >> > won't give enough context to return a
> descriptive
> > > > error
> > > > > > > > message
> > > > > > > > > > for
> > > > > > > > > > > > >> user.
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > Look forward to discussing all outstanding
> issues
> > in
> > > > > detail
> > > > > > > > > today
> > > > > > > > > > > > during
> > > > > > > > > > > > >> > the call.
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <
> > > > > jun@confluent.io
> > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >> >
> > > > > > > > > > > > >> > > 101. There may be a use case where you only
> want
> > > the
> > > > > > > topics
> > > > > > > > to
> > > > > > > > > > be
> > > > > > > > > > > > >> created
> > > > > > > > > > > > >> > > manually by admins. Currently, you can do that
> > by
> > > > > > > disabling
> > > > > > > > > auto
> > > > > > > > > > > > topic
> > > > > > > > > > > > >> > > creation and issue topic creation from the
> > > > > TopicCommand.
> > > > > > > If
> > > > > > > > we
> > > > > > > > > > > > >> disable auto
> > > > > > > > > > > > >> > > topic creation completely on the broker and
> > don't
> > > > > have a
> > > > > > > way
> > > > > > > > > to
> > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > >> > > between topic creation requests from the
> regular
> > > > > clients
> > > > > > > and
> > > > > > > > > the
> > > > > > > > > > > > >> admin, we
> > > > > > > > > > > > >> > > can't support manual topic creation any more.
> I
> > > was
> > > > > > > thinking
> > > > > > > > > > that
> > > > > > > > > > > > >> another
> > > > > > > > > > > > >> > > way of distinguishing the clients making the
> > topic
> > > > > > > creation
> > > > > > > > > > requests
> > > > > > > > > > > > >> is
> > > > > > > > > > > > >> > > using clientId. For example, the admin tool
> can
> > > set
> > > > > it to
> > > > > > > > > > something
> > > > > > > > > > > > >> like
> > > > > > > > > > > > >> > > admin and the broker can treat that clientId
> > > > > specially.
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > Also, there is a related discussion in
> > KAFKA-2020.
> > > > > > > > Currently,
> > > > > > > > > > we do
> > > > > > > > > > > > >> the
> > > > > > > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > 1. If leader is not available, we set the
> > > partition
> > > > > level
> > > > > > > > > error
> > > > > > > > > > code
> > > > > > > > > > > > >> to
> > > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > > >> > > 2. If a non-leader replica is not available,
> we
> > > take
> > > > > that
> > > > > > > > > > replica
> > > > > > > > > > > > out
> > > > > > > > > > > > >> of
> > > > > > > > > > > > >> > > the assigned replica list and isr in the
> > response.
> > > > As
> > > > > an
> > > > > > > > > > indication
> > > > > > > > > > > > >> for
> > > > > > > > > > > > >> > > doing that, we set the partition level error
> > code
> > > to
> > > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > This has a few problems. First,
> > > ReplicaNotAvailable
> > > > > > > probably
> > > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > > >> > > an error, at least for the normal
> > > producer/consumer
> > > > > > > clients
> > > > > > > > > that
> > > > > > > > > > > > just
> > > > > > > > > > > > >> want
> > > > > > > > > > > > >> > > to find out the leader. Second, it can happen
> > that
> > > > > both
> > > > > > > the
> > > > > > > > > > leader
> > > > > > > > > > > > and
> > > > > > > > > > > > >> > > another replica are not available at the same
> > > time.
> > > > > There
> > > > > > > is
> > > > > > > > > no
> > > > > > > > > > > > error
> > > > > > > > > > > > >> code
> > > > > > > > > > > > >> > > to indicate both. Third, even if a replica is
> > not
> > > > > > > available,
> > > > > > > > > > it's
> > > > > > > > > > > > >> still
> > > > > > > > > > > > >> > > useful to return its replica id since some
> > clients
> > > > > (e.g.
> > > > > > > > admin
> > > > > > > > > > tool)
> > > > > > > > > > > > >> may
> > > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > One way to address this issue is to always
> > return
> > > > the
> > > > > > > > replica
> > > > > > > > > > id for
> > > > > > > > > > > > >> > > leader, assigned replicas, and isr regardless
> of
> > > > > whether
> > > > > > > the
> > > > > > > > > > > > >> corresponding
> > > > > > > > > > > > >> > > broker is live or not. Since we also return
> the
> > > list
> > > > > of
> > > > > > > live
> > > > > > > > > > > > brokers,
> > > > > > > > > > > > >> the
> > > > > > > > > > > > >> > > client can figure out whether a leader or a
> > > replica
> > > > is
> > > > > > > live
> > > > > > > > or
> > > > > > > > > > not
> > > > > > > > > > > > >> and act
> > > > > > > > > > > > >> > > accordingly. This way, we don't need to set
> the
> > > > > partition
> > > > > > > > > level
> > > > > > > > > > > > error
> > > > > > > > > > > > >> code
> > > > > > > > > > > > >> > > when the leader or a replica is not available.
> > > This
> > > > > > > doesn't
> > > > > > > > > > change
> > > > > > > > > > > > >> the wire
> > > > > > > > > > > > >> > > protocol, but does change the semantics. Since
> > we
> > > > are
> > > > > > > > evolving
> > > > > > > > > > the
> > > > > > > > > > > > >> protocol
> > > > > > > > > > > > >> > > of TopicMetadataRequest here, we can
> potentially
> > > > > piggyback
> > > > > > > > the
> > > > > > > > > > > > change.
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > 102.1 For those types of errors due to invalid
> > > > input,
> > > > > > > > > shouldn't
> > > > > > > > > > we
> > > > > > > > > > > > >> just
> > > > > > > > > > > > >> > > guard it at parameter validation time and
> throw
> > > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > > >> > > without even sending the request to the
> broker?
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > Jun
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii
> > > Biletskyi <
> > > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > 101. If I understand you correctly, you are
> > > saying
> > > > > > > future
> > > > > > > > > > producer
> > > > > > > > > > > > >> > > versions
> > > > > > > > > > > > >> > > > (which
> > > > > > > > > > > > >> > > > will be ported to TMR_V1) won't be able to
> > > > > automatically
> > > > > > > > > > create
> > > > > > > > > > > > >> topic (if
> > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > >> > > > unconditionally remove topic creation from
> > > there).
> > > > > But
> > > > > > > we
> > > > > > > > > > need to
> > > > > > > > > > > > >> this
> > > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > > >> > > > Ok, about your proposal: I'm not a big fan
> > too,
> > > > > when it
> > > > > > > > > comes
> > > > > > > > > > to
> > > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > > >> > > > clients directly in protocol schema. And
> also
> > > I'm
> > > > > not
> > > > > > > > sure I
> > > > > > > > > > > > >> understand
> > > > > > > > > > > > >> > > at
> > > > > > > > > > > > >> > > > all why
> > > > > > > > > > > > >> > > > auto.create.topics.enable is a server side
> > > > > > > configuration.
> > > > > > > > > Can
> > > > > > > > > > we
> > > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > > >> > > > in future versions, add this setting to
> > producer
> > > > and
> > > > > > > based
> > > > > > > > > on
> > > > > > > > > > that
> > > > > > > > > > > > >> upon
> > > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > > >> > > > UnknownTopic create topic explicitly by a
> > > separate
> > > > > > > > producer
> > > > > > > > > > call
> > > > > > > > > > > > via
> > > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because we want to
> > support
> > > > > batching
> > > > > > > > and
> > > > > > > > > > at
> > > > > > > > > > > > the
> > > > > > > > > > > > >> same
> > > > > > > > > > > > >> > > > time we
> > > > > > > > > > > > >> > > > want to give descriptive error messages for
> > > > clients.
> > > > > > > Since
> > > > > > > > > > > > >> AdminClient
> > > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > > >> > > > to construct such messages (e.g. AdminClient
> > > layer
> > > > > can
> > > > > > > > know
> > > > > > > > > > that
> > > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > > >> > > > means two cases: either invalid number -
> e.g.
> > > -1;
> > > > or
> > > > > > > > > > > > >> replication-factor
> > > > > > > > > > > > >> > > was
> > > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > > >> > > > partitions argument wasn't) - I wrapped
> > > responses
> > > > in
> > > > > > > > > > Exceptions.
> > > > > > > > > > > > >> But I'm
> > > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > > >> > > > other ideas, this was just initial version.
> > > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change that to
> > > probably
> > > > > some
> > > > > > > > other
> > > > > > > > > > dto.
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <
> > > > > > > > jun@confluent.io>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > >> > > > > 101. That's what I was thinking too, but
> it
> > > may
> > > > > not be
> > > > > > > > > that
> > > > > > > > > > > > >> simple. In
> > > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > > >> > > > > we can let it not trigger auto topic
> > creation.
> > > > > Then,
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > >> producer
> > > > > > > > > > > > >> > > > side,
> > > > > > > > > > > > >> > > > > if it gets an UnknownTopicException, it
> can
> > > > > explicitly
> > > > > > > > > > issue a
> > > > > > > > > > > > >> > > > > createTopicRequest for auto topic
> creation.
> > On
> > > > the
> > > > > > > > > consumer
> > > > > > > > > > > > side,
> > > > > > > > > > > > >> it
> > > > > > > > > > > > >> > > will
> > > > > > > > > > > > >> > > > > never issue createTopicRequest. This works
> > > when
> > > > > auto
> > > > > > > > topic
> > > > > > > > > > > > >> creation is
> > > > > > > > > > > > >> > > > > enabled on the broker side. However, I am
> > not
> > > > > sure how
> > > > > > > > > > things
> > > > > > > > > > > > >> will work
> > > > > > > > > > > > >> > > > > when auto topic creation is disabled on
> the
> > > > broker
> > > > > > > side.
> > > > > > > > > In
> > > > > > > > > > this
> > > > > > > > > > > > >> case,
> > > > > > > > > > > > >> > > we
> > > > > > > > > > > > >> > > > > want to have a way to manually create a
> > topic,
> > > > > > > > potentially
> > > > > > > > > > > > through
> > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > >> > > > > commands. However, then we need a way to
> > > > > distinguish
> > > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > > >> > > > > issued from the producer clients and the
> > admin
> > > > > tools.
> > > > > > > > May
> > > > > > > > > > be we
> > > > > > > > > > > > >> can
> > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > >> > > > > new field in createTopicRequest and set it
> > > > > differently
> > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > >> producer
> > > > > > > > > > > > >> > > > > client and the admin client. However, I am
> > not
> > > > > sure if
> > > > > > > > > > that's
> > > > > > > > > > > > the
> > > > > > > > > > > > >> best
> > > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > >> > > > > 2. Yes, refactoring existing requests is a
> > > > > non-trivial
> > > > > > > > > > amount of
> > > > > > > > > > > > >> work.
> > > > > > > > > > > > >> > > I
> > > > > > > > > > > > >> > > > > posted some comments in KAFKA-1927. We
> will
> > > > > probably
> > > > > > > > have
> > > > > > > > > > to fix
> > > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > > >> > > > > first, before adding the new logic in
> > > > KAFKA-1694.
> > > > > > > > > > Otherwise, the
> > > > > > > > > > > > >> > > changes
> > > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > > > >> > > > > 102.1. It's a bit weird that we return
> > > exception
> > > > > in
> > > > > > > the
> > > > > > > > > > api. It
> > > > > > > > > > > > >> seems
> > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > >> > > > > we should either return error code or
> throw
> > an
> > > > > > > exception
> > > > > > > > > > when
> > > > > > > > > > > > >> getting
> > > > > > > > > > > > >> > > the
> > > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > > >> > > > > 102.2. We probably shouldn't explicitly
> use
> > > the
> > > > > > > request
> > > > > > > > > > object
> > > > > > > > > > > > in
> > > > > > > > > > > > >> the
> > > > > > > > > > > > >> > > > api.
> > > > > > > > > > > > >> > > > > Not every request evolution requires an
> api
> > > > > change.
> > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii
> > > > Biletskyi
> > > > > <
> > > > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > > Thanks for you comments. Answers inline:
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > > 100. There are a few fields such as
> > > > > > > ReplicaAssignment,
> > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> > > > represented
> > > > > as a
> > > > > > > > > > string,
> > > > > > > > > > > > but
> > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > >> > > > > > > composite structures in json. Could we
> > > > flatten
> > > > > > > them
> > > > > > > > > out
> > > > > > > > > > > > >> directly in
> > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > > Yes, now with Admin Client this looks a
> > bit
> > > > > weird.
> > > > > > > My
> > > > > > > > > > initial
> > > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > > >> > > > > > ReassignPartitionCommand accepts input
> in
> > > > json,
> > > > > we
> > > > > > > > want
> > > > > > > > > to
> > > > > > > > > > > > >> remain
> > > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > > >> > > > > > interfaces unchanged, where possible.
> > > > > > > > > > > > >> > > > > > If we port it to deserialized format, in
> > CLI
> > > > > (/tools
> > > > > > > > > > project)
> > > > > > > > > > > > >> we will
> > > > > > > > > > > > >> > > > > have
> > > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > > >> > > > > > json library since /tools is written in
> > java
> > > > and
> > > > > > > we'll
> > > > > > > > > > need to
> > > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > > >> > > > > > provided by a user. Can we quickly agree
> > on
> > > > what
> > > > > > > this
> > > > > > > > > > library
> > > > > > > > > > > > >> should
> > > > > > > > > > > > >> > > be
> > > > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > > 101. Does TopicMetadataRequest v1 still
> > > > trigger
> > > > > auto
> > > > > > > > > topic
> > > > > > > > > > > > >> creation?
> > > > > > > > > > > > >> > > > This
> > > > > > > > > > > > >> > > > > > > will be a bit weird now that we have a
> > > > > separate
> > > > > > > > topic
> > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > > createTopicRequest
> > > > > > > and
> > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > >> > > > > > > v1 will be used in the
> producer/consumer
> > > > > client,
> > > > > > > in
> > > > > > > > > > addition
> > > > > > > > > > > > >> to
> > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > >> > > > > > > tools? For example, ideally, we don't
> > want
> > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > >> > > from
> > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > >> > > > > > > consumer to trigger auto topic
> creation.
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > > I agree, this strange logic should be
> > fixed.
> > > > > I'm not
> > > > > > > > > > confident
> > > > > > > > > > > > >> in
> > > > > > > > > > > > >> > > this
> > > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > > >> > > > > > correct me if I'm wrong, but it doesn't
> > look
> > > > > like a
> > > > > > > > hard
> > > > > > > > > > thing
> > > > > > > > > > > > >> to
> > > > > > > > > > > > >> > > do, I
> > > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > > >> > > > > > leverage AdminClient for that in
> Producer
> > > and
> > > > > > > > > > unconditionally
> > > > > > > > > > > > >> remove
> > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > >> > > > > > creation from the
> TopicMetadataRequest_V1.
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > > 2. I think Jay meant getting rid of
> scala
> > > > > classes
> > > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > >> > > did
> > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > >> > > > > > > as a stop-gap thing when adding the
> new
> > > > > requests
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > >> > > > > > > However, the long term plan is to get
> > rid
> > > of
> > > > > all
> > > > > > > > those
> > > > > > > > > > and
> > > > > > > > > > > > >> just
> > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > >> > > > > > > java request/response in the client.
> > Since
> > > > > this
> > > > > > > KIP
> > > > > > > > > > proposes
> > > > > > > > > > > > >> to
> > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > >> > > > > > > significant number of new requests,
> > > perhaps
> > > > we
> > > > > > > > should
> > > > > > > > > > bite
> > > > > > > > > > > > the
> > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > >> > > > > > > clean up the existing scala requests
> > first
> > > > > before
> > > > > > > > > > adding new
> > > > > > > > > > > > >> ones?
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > > Yes, looks like I misunderstood the
> point
> > of
> > > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > > >> > > > > > rework that. The only thing is that I
> > don't
> > > > see
> > > > > any
> > > > > > > > > > example
> > > > > > > > > > > > how
> > > > > > > > > > > > >> it
> > > > > > > > > > > > >> > > was
> > > > > > > > > > > > >> > > > > done
> > > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > > >> > > > > > least one existing protocol message.
> Thus,
> > > as
> > > > I
> > > > > > > > > > understand, I
> > > > > > > > > > > > >> have to
> > > > > > > > > > > > >> > > > > think
> > > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > > >> > > > > > Re porting all existing RQ/RP in this
> > patch.
> > > > > Sounds
> > > > > > > > > > > > reasonable,
> > > > > > > > > > > > >> but
> > > > > > > > > > > > >> > > if
> > > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > > >> > > > > > requirement to have Admin KIP done, I'm
> > > afraid
> > > > > this
> > > > > > > > can
> > > > > > > > > > be a
> > > > > > > > > > > > >> serious
> > > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > > >> > > > > > There are 13 protocol messages and all
> > that
> > > > > would
> > > > > > > > > require
> > > > > > > > > > not
> > > > > > > > > > > > >> only
> > > > > > > > > > > > >> > > unit
> > > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > > >> > > > > > intensive manual testing, no? I'm afraid
> > I'm
> > > > > not the
> > > > > > > > > > right guy
> > > > > > > > > > > > >> to
> > > > > > > > > > > > >> > > cover
> > > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > > >> > > > > > Kafka core internals :). Let me know
> your
> > > > > thoughts
> > > > > > > on
> > > > > > > > > this
> > > > > > > > > > > > >> item. Btw
> > > > > > > > > > > > >> > > > > there
> > > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > > >> > >
> > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > > >> > > > ).
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun
> Rao <
> > > > > > > > > > jun@confluent.io>
> > > > > > > > > > > > >> wrote:
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > > 100. There are a few fields such as
> > > > > > > > ReplicaAssignment,
> > > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> > > > represented
> > > > > as a
> > > > > > > > > > string,
> > > > > > > > > > > > but
> > > > > > > > > > > > >> > > > contain
> > > > > > > > > > > > >> > > > > > > composite structures in json. Could we
> > > > flatten
> > > > > > > them
> > > > > > > > > out
> > > > > > > > > > > > >> directly in
> > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > > 101. Does TopicMetadataRequest v1
> still
> > > > > trigger
> > > > > > > auto
> > > > > > > > > > topic
> > > > > > > > > > > > >> > > creation?
> > > > > > > > > > > > >> > > > > This
> > > > > > > > > > > > >> > > > > > > will be a bit weird now that we have a
> > > > > separate
> > > > > > > > topic
> > > > > > > > > > > > >> creation api.
> > > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > > createTopicRequest
> > > > > > > and
> > > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > > >> > > > > > > v1 will be used in the
> producer/consumer
> > > > > client,
> > > > > > > in
> > > > > > > > > > addition
> > > > > > > > > > > > >> to
> > > > > > > > > > > > >> > > admin
> > > > > > > > > > > > >> > > > > > > tools? For example, ideally, we don't
> > want
> > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > >> > > from
> > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > >> > > > > > > consumer to trigger auto topic
> creation.
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > > 2. I think Jay meant getting rid of
> > scala
> > > > > classes
> > > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > > >> > > did
> > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > >> > > > > > > as a stop-gap thing when adding the
> new
> > > > > requests
> > > > > > > for
> > > > > > > > > the
> > > > > > > > > > > > >> consumers.
> > > > > > > > > > > > >> > > > > > > However, the long term plan is to get
> > rid
> > > of
> > > > > all
> > > > > > > > those
> > > > > > > > > > and
> > > > > > > > > > > > >> just
> > > > > > > > > > > > >> > > reuse
> > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > >> > > > > > > java request/response in the client.
> > Since
> > > > > this
> > > > > > > KIP
> > > > > > > > > > proposes
> > > > > > > > > > > > >> to
> > > > > > > > > > > > >> > > add a
> > > > > > > > > > > > >> > > > > > > significant number of new requests,
> > > perhaps
> > > > we
> > > > > > > > should
> > > > > > > > > > bite
> > > > > > > > > > > > the
> > > > > > > > > > > > >> > > bullet
> > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > >> > > > > > > clean up the existing scala requests
> > first
> > > > > before
> > > > > > > > > > adding new
> > > > > > > > > > > > >> ones?
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM,
> Andrii
> > > > > Biletskyi
> > > > > > > <
> > > > > > > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > As said above - I list again all
> > > comments
> > > > > from
> > > > > > > > this
> > > > > > > > > > thread
> > > > > > > > > > > > >> so we
> > > > > > > > > > > > >> > > > > > > > can see what's left and finalize all
> > > > pending
> > > > > > > > issues.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > > >> > > > > > > > 1. This is much needed
> functionality,
> > > but
> > > > > there
> > > > > > > > are
> > > > > > > > > a
> > > > > > > > > > lot
> > > > > > > > > > > > >> of the
> > > > > > > > > > > > >> > > so
> > > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > > >> > > > > > > > really think these protocols
> through.
> > We
> > > > > really
> > > > > > > > want
> > > > > > > > > > to
> > > > > > > > > > > > end
> > > > > > > > > > > > >> up
> > > > > > > > > > > > >> > > > with a
> > > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > > >> > > > > > > > of well thought-out, orthoganol
> apis.
> > > For
> > > > > this
> > > > > > > > > reason
> > > > > > > > > > I
> > > > > > > > > > > > >> think it
> > > > > > > > > > > > >> > > is
> > > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > > >> > > > > > > > important to think through the end
> > state
> > > > > even if
> > > > > > > > > that
> > > > > > > > > > > > >> includes
> > > > > > > > > > > > >> > > APIs
> > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > >> > > > > > > > won't implement in the first phase.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: Definitely behind this. Would
> > > > appreciate
> > > > > if
> > > > > > > > there
> > > > > > > > > > are
> > > > > > > > > > > > >> concrete
> > > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 2. Let's please please please wait
> > until
> > > > we
> > > > > have
> > > > > > > > > > switched
> > > > > > > > > > > > >> the
> > > > > > > > > > > > >> > > > server
> > > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > > >> > > > > > > > to the new java protocol
> definitions.
> > If
> > > > we
> > > > > add
> > > > > > > > > upteen
> > > > > > > > > > > > more
> > > > > > > > > > > > >> ad
> > > > > > > > > > > > >> > > hoc
> > > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > > >> > > > > > > > objects that is just generating more
> > > work
> > > > > for
> > > > > > > the
> > > > > > > > > > > > >> conversion we
> > > > > > > > > > > > >> > > > know
> > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> removed
> > > > scala
> > > > > > > > > protocol
> > > > > > > > > > > > >> classes.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 3. This proposal introduces a new
> type
> > > of
> > > > > > > optional
> > > > > > > > > > > > >> parameter.
> > > > > > > > > > > > >> > > This
> > > > > > > > > > > > >> > > > is
> > > > > > > > > > > > >> > > > > > > > inconsistent with everything else in
> > the
> > > > > > > protocol
> > > > > > > > > > where we
> > > > > > > > > > > > >> use -1
> > > > > > > > > > > > >> > > > or
> > > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > > >> > > > > > > > other marker value. You could argue
> > > either
> > > > > way
> > > > > > > but
> > > > > > > > > > let's
> > > > > > > > > > > > >> stick
> > > > > > > > > > > > >> > > with
> > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > >> > > > > > > > for consistency. For clients that
> > > > > implemented
> > > > > > > the
> > > > > > > > > > protocol
> > > > > > > > > > > > >> in a
> > > > > > > > > > > > >> > > > > better
> > > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > > >> > > > > > > > than our scala code these basic
> > > primitives
> > > > > are
> > > > > > > > hard
> > > > > > > > > to
> > > > > > > > > > > > >> change.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> removed
> > > > > MaybeOf
> > > > > > > > type
> > > > > > > > > > and
> > > > > > > > > > > > >> changed
> > > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This seems to
> > > > duplicate
> > > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > > >> > > > > which
> > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > >> > > > > > > > brokers, topics, and partitions. I
> > think
> > > > we
> > > > > > > should
> > > > > > > > > > rename
> > > > > > > > > > > > >> that
> > > > > > > > > > > > >> > > > > request
> > > > > > > > > > > > >> > > > > > > > ClusterMetadataRequest (or just
> > > > > MetadataRequest)
> > > > > > > > and
> > > > > > > > > > > > >> include the
> > > > > > > > > > > > >> > > id
> > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > >> > > > > > > > controller. Or are there other
> things
> > we
> > > > > could
> > > > > > > add
> > > > > > > > > > here?
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: I agree. Updated the KIP. Let's
> > > extends
> > > > > > > > > > TopicMetadata
> > > > > > > > > > > > to
> > > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 5. We have a tendency to try to
> make a
> > > lot
> > > > > of
> > > > > > > > > requests
> > > > > > > > > > > > that
> > > > > > > > > > > > >> can
> > > > > > > > > > > > >> > > > only
> > > > > > > > > > > > >> > > > > go
> > > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > > >> > > > > > > > particular nodes. This adds a lot of
> > > > burden
> > > > > for
> > > > > > > > > client
> > > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > > >> > > > > > > > sounds easy but each discovery can
> > fail
> > > in
> > > > > many
> > > > > > > > > parts
> > > > > > > > > > so
> > > > > > > > > > > > it
> > > > > > > > > > > > >> ends
> > > > > > > > > > > > >> > > up
> > > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > > >> > > > > > > > full state machine to do right). I
> > think
> > > > we
> > > > > > > should
> > > > > > > > > > > > consider
> > > > > > > > > > > > >> > > making
> > > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > > >> > > > > > > > commands and ideally as many of the
> > > other
> > > > > apis
> > > > > > > as
> > > > > > > > > > possible
> > > > > > > > > > > > >> > > > available
> > > > > > > > > > > > >> > > > > on
> > > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > > >> > > > > > > > brokers and just redirect to the
> > > > controller
> > > > > on
> > > > > > > the
> > > > > > > > > > broker
> > > > > > > > > > > > >> side.
> > > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > > >> > > > > > > > there would be a general way to
> > > > encapsulate
> > > > > this
> > > > > > > > > > > > re-routing
> > > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: It's a very interesting idea, but
> > > seems
> > > > > there
> > > > > > > > are
> > > > > > > > > > some
> > > > > > > > > > > > >> > > concerns
> > > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > > >> > > > > > > > feature (like performance
> > > considerations,
> > > > > how
> > > > > > > this
> > > > > > > > > > will
> > > > > > > > > > > > >> > > complicate
> > > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > > >> > > > > > > > I believe this shouldn't be a
> blocker.
> > > If
> > > > > this
> > > > > > > > > > feature is
> > > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > > >> > > > > > > > point it won't affect Admin changes
> -
> > at
> > > > > least
> > > > > > > no
> > > > > > > > > > changes
> > > > > > > > > > > > to
> > > > > > > > > > > > >> > > public
> > > > > > > > > > > > >> > > > > API
> > > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 6. We should probably normalize the
> > key
> > > > > value
> > > > > > > > pairs
> > > > > > > > > > used
> > > > > > > > > > > > for
> > > > > > > > > > > > >> > > > configs
> > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > >> > > > > > > > than embedding a new formatting. So
> > two
> > > > > strings
> > > > > > > > > rather
> > > > > > > > > > > > than
> > > > > > > > > > > > >> one
> > > > > > > > > > > > >> > > > with
> > > > > > > > > > > > >> > > > > an
> > > > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > > normalized
> > > > > > > configs
> > > > > > > > > and
> > > > > > > > > > > > >> changed
> > > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 7. Is the postcondition of these
> APIs
> > > that
> > > > > the
> > > > > > > > > > command has
> > > > > > > > > > > > >> begun
> > > > > > > > > > > > >> > > or
> > > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > > >> > > > > > > > the command has been completed? It
> is
> > a
> > > > lot
> > > > > more
> > > > > > > > > > usable if
> > > > > > > > > > > > >> the
> > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > > >> > > > > > > > been completed so you know that if
> you
> > > > > create a
> > > > > > > > > topic
> > > > > > > > > > and
> > > > > > > > > > > > >> then
> > > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > >> > > > > > > > it you won't get an exception about
> > > there
> > > > > being
> > > > > > > no
> > > > > > > > > > such
> > > > > > > > > > > > >> topic.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: For long running requests (like
> > > > reassign
> > > > > > > > > > partitions) -
> > > > > > > > > > > > >> the
> > > > > > > > > > > > >> > > post
> > > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > > >> > > > > > > > command has begun - so we don't
> block
> > > the
> > > > > > > client.
> > > > > > > > In
> > > > > > > > > > case
> > > > > > > > > > > > >> of your
> > > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > > >> > > > > > > > topic commands, this will be
> > refactored
> > > > and
> > > > > > > topic
> > > > > > > > > > commands
> > > > > > > > > > > > >> will
> > > > > > > > > > > > >> > > be
> > > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > > >> > > > > > > > immediately, since the Controller
> will
> > > > serve
> > > > > > > Admin
> > > > > > > > > > > > requests
> > > > > > > > > > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 8. Describe topic and list topics
> > > > duplicate
> > > > > a
> > > > > > > lot
> > > > > > > > of
> > > > > > > > > > stuff
> > > > > > > > > > > > >> in the
> > > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > > >> > > > > > > > request. Is there a reason to give
> > back
> > > > > topics
> > > > > > > > > marked
> > > > > > > > > > for
> > > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > > >> > > > > > > > like if we just make the
> > post-condition
> > > of
> > > > > the
> > > > > > > > > delete
> > > > > > > > > > > > >> command be
> > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > >> > > > > > > > topic is deleted that will get rid
> of
> > > the
> > > > > need
> > > > > > > for
> > > > > > > > > > this
> > > > > > > > > > > > >> right?
> > > > > > > > > > > > >> > > And
> > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> removed
> > > > > topics
> > > > > > > > marked
> > > > > > > > > > for
> > > > > > > > > > > > >> deletion
> > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 9. Should we consider batching these
> > > > > requests?
> > > > > > > We
> > > > > > > > > have
> > > > > > > > > > > > >> generally
> > > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > >> > > > > > > > allow multiple operations to be
> > batched.
> > > > My
> > > > > > > > > suspicion
> > > > > > > > > > is
> > > > > > > > > > > > >> that
> > > > > > > > > > > > >> > > > without
> > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > >> > > > > > > > we will get a lot of code that does
> > > > > something
> > > > > > > like
> > > > > > > > > > > > >> > > > > > > >    for(topic:
> > adminClient.listTopics())
> > > > > > > > > > > > >> > > > > > > >
>  adminClient.describeTopic(topic)
> > > > > > > > > > > > >> > > > > > > > this code will work great when you
> > test
> > > > on 5
> > > > > > > > topics
> > > > > > > > > > but
> > > > > > > > > > > > not
> > > > > > > > > > > > >> do as
> > > > > > > > > > > > >> > > > > well
> > > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check
> > "Topic
> > > > > Admin
> > > > > > > > > Schema"
> > > > > > > > > > > > >> section.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 10. I think we should also discuss
> how
> > > we
> > > > > want
> > > > > > > to
> > > > > > > > > > expose a
> > > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > > >> > > > > > > > client api for these operations.
> > > Currently
> > > > > > > people
> > > > > > > > > > rely on
> > > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > > >> > > > > > > > is totally sketchy. I think we
> > probably
> > > > need
> > > > > > > > another
> > > > > > > > > > > > client
> > > > > > > > > > > > >> under
> > > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > > >> > > > > > > > that exposes administrative
> > > functionality.
> > > > > We
> > > > > > > will
> > > > > > > > > > need
> > > > > > > > > > > > >> this just
> > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > >> > > > > > > > properly test the new apis, I
> suspect.
> > > We
> > > > > should
> > > > > > > > > > figure
> > > > > > > > > > > > out
> > > > > > > > > > > > >> that
> > > > > > > > > > > > >> > > > API.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check
> > "Admin
> > > > > Client"
> > > > > > > > > > section
> > > > > > > > > > > > >> with an
> > > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 11. The other information that would
> > be
> > > > > really
> > > > > > > > > useful
> > > > > > > > > > to
> > > > > > > > > > > > get
> > > > > > > > > > > > >> > > would
> > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > >> > > > > > > > information about partitions--how
> much
> > > > data
> > > > > is
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > > >> partition,
> > > > > > > > > > > > >> > > > what
> > > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > > >> > > > > > > > the segment offsets, what is the
> > log-end
> > > > > offset
> > > > > > > > > (i.e.
> > > > > > > > > > last
> > > > > > > > > > > > >> > > offset),
> > > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > > >> > > > > > > > the compaction point, etc. I think
> > that
> > > > done
> > > > > > > right
> > > > > > > > > > this
> > > > > > > > > > > > >> would be
> > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > >> > > > > > > > successor to the very awkward
> > > > OffsetRequest
> > > > > we
> > > > > > > > have
> > > > > > > > > > today.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: I removed
> > ConsumerGroupOffsetsRequest
> > > > in
> > > > > the
> > > > > > > > > latest
> > > > > > > > > > > > >> patch. I
> > > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > > >> > > > > > > > be resolved in a separate KIP / jira
> > > > ticket.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 12. Generally we can do good error
> > > > handling
> > > > > > > > without
> > > > > > > > > > > > needing
> > > > > > > > > > > > >> > > custom
> > > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > > >> > > > > > > > messages. I.e. generally the client
> > has
> > > > the
> > > > > > > > context
> > > > > > > > > to
> > > > > > > > > > > > know
> > > > > > > > > > > > >> that
> > > > > > > > > > > > >> > > if
> > > > > > > > > > > > >> > > > > it
> > > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > > >> > > > > > > > an error that the topic doesn't
> exist
> > to
> > > > say
> > > > > > > > "Topic
> > > > > > > > > X
> > > > > > > > > > > > >> doesn't
> > > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > > >> > > > > > > > than "error code 14" (or whatever).
> > > Maybe
> > > > > there
> > > > > > > > are
> > > > > > > > > > > > specific
> > > > > > > > > > > > >> > > cases
> > > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > > >> > > > > > > > this is hard? If we want to add
> > > > server-side
> > > > > > > error
> > > > > > > > > > messages
> > > > > > > > > > > > >> we
> > > > > > > > > > > > >> > > > really
> > > > > > > > > > > > >> > > > > do
> > > > > > > > > > > > >> > > > > > > > need to do this in a consistent way
> > > across
> > > > > the
> > > > > > > > > > protocol.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check
> > > > "Protocol
> > > > > > > > Errors"
> > > > > > > > > > > > >> section. I
> > > > > > > > > > > > >> > > > added
> > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > >> > > > > > > > comprehensive, fine-grained list of
> > > error
> > > > > codes.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > > > > >> > > > > > > > 13. Describe topic request: it would
> > be
> > > > > great to
> > > > > > > > go
> > > > > > > > > > beyond
> > > > > > > > > > > > >> just
> > > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > >> > > > > > > > topic name regex for this request.
> For
> > > > > example,
> > > > > > > a
> > > > > > > > > very
> > > > > > > > > > > > >> common use
> > > > > > > > > > > > >> > > > > case
> > > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > > >> > > > > > > > the topic command is to list all
> > topics
> > > > > whose
> > > > > > > > config
> > > > > > > > > > A's
> > > > > > > > > > > > >> value is
> > > > > > > > > > > > >> > > > B.
> > > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > > >> > > > > > > > topic name regex then we have to
> first
> > > > > retrieve
> > > > > > > > > > __all__
> > > > > > > > > > > > >> topics's
> > > > > > > > > > > > >> > > > > > > > description info and then filter at
> > the
> > > > > client
> > > > > > > > end,
> > > > > > > > > > which
> > > > > > > > > > > > >> will
> > > > > > > > > > > > >> > > be a
> > > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in create topic:
> this
> > is
> > > > > related
> > > > > > > > to
> > > > > > > > > > the
> > > > > > > > > > > > >> previous
> > > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > > >> > > > > > > > maybe we can add another metadata
> K-V
> > or
> > > > > just a
> > > > > > > > > > metadata
> > > > > > > > > > > > >> string
> > > > > > > > > > > > >> > > > along
> > > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > > >> > > > > > > > with config K-V in create topic like
> > we
> > > > did
> > > > > for
> > > > > > > > > offset
> > > > > > > > > > > > >> commit
> > > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > > >> > > > > > > > field can be quite useful in storing
> > > > > information
> > > > > > > > > like
> > > > > > > > > > > > >> "owner" of
> > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > > >> > > > > > > > who issue the create command, etc,
> > which
> > > > is
> > > > > > > quite
> > > > > > > > > > > > important
> > > > > > > > > > > > >> for a
> > > > > > > > > > > > >> > > > > > > > multi-tenant setting. Then in the
> > > describe
> > > > > topic
> > > > > > > > > > request
> > > > > > > > > > > > we
> > > > > > > > > > > > >> can
> > > > > > > > > > > > >> > > > also
> > > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > > >> > > > > > > > on regex of the metadata field.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: As discussed it is very
> interesting
> > > but
> > > > > can
> > > > > > > be
> > > > > > > > > > > > >> implemented
> > > > > > > > > > > > >> > > later
> > > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > > >> > > > > > > > we have some basic functionality
> > there.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > 15. Today all the admin operations
> are
> > > > > async in
> > > > > > > > the
> > > > > > > > > > sense
> > > > > > > > > > > > >> that
> > > > > > > > > > > > >> > > > > command
> > > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > > >> > > > > > > > return once it is written in ZK, and
> > > that
> > > > > is why
> > > > > > > > we
> > > > > > > > > > need
> > > > > > > > > > > > >> extra
> > > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > > >> > > > > > > > like testUtil.waitForTopicCreated()
> /
> > > > verify
> > > > > > > > > partition
> > > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > > >> > > > > > > > request, etc. With admin requests we
> > > could
> > > > > add a
> > > > > > > > > flag
> > > > > > > > > > to
> > > > > > > > > > > > >> enable /
> > > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > > >> > > > > > > > synchronous requests; when it is
> > turned
> > > > on,
> > > > > the
> > > > > > > > > > response
> > > > > > > > > > > > >> will not
> > > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > > >> > > > > > > > until the request has been
> completed.
> > > And
> > > > > for
> > > > > > > > async
> > > > > > > > > > > > >> requests we
> > > > > > > > > > > > >> > > can
> > > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > > >> > > > > > > > "token" field in the response, and
> > then
> > > > only
> > > > > > > need
> > > > > > > > a
> > > > > > > > > > > > general
> > > > > > > > > > > > >> > > "admin
> > > > > > > > > > > > >> > > > > > > > verification request" with the given
> > > token
> > > > > to
> > > > > > > > check
> > > > > > > > > > if the
> > > > > > > > > > > > >> async
> > > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: I see your point. My idea was to
> > > > provide
> > > > > > > > specific
> > > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > > >> > > > > > > > long running request, where needed.
> We
> > > can
> > > > > do it
> > > > > > > > the
> > > > > > > > > > way
> > > > > > > > > > > > you
> > > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > > >> > > > > > > > concern is that introducing a token
> we
> > > > again
> > > > > > > will
> > > > > > > > > make
> > > > > > > > > > > > >> schema
> > > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > > >> > > > > > > > to do similar thing introducing
> single
> > > > > > > > AdminRequest
> > > > > > > > > > for
> > > > > > > > > > > > all
> > > > > > > > > > > > >> topic
> > > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > > >> > > > > > > > this idea because we wanted to have
> > > schema
> > > > > > > > defined.
> > > > > > > > > So
> > > > > > > > > > > > this
> > > > > > > > > > > > >> is
> > > > > > > > > > > > >> > > > more a
> > > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > > >> > > > > > > > a) have fixed schema but introduce
> > each
> > > > > time new
> > > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > > > >> > > > > > > > b) use one request for verification
> > but
> > > > > > > generalize
> > > > > > > > > it
> > > > > > > > > > with
> > > > > > > > > > > > >> token
> > > > > > > > > > > > >> > > > > > > > I'm fine with whatever decision
> > > community
> > > > > come
> > > > > > > to.
> > > > > > > > > > Just
> > > > > > > > > > > > let
> > > > > > > > > > > > >> me
> > > > > > > > > > > > >> > > know
> > > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > > >> > > > > > > > 16. Specifically for ownership, I
> > think
> > > > the
> > > > > plan
> > > > > > > > is
> > > > > > > > > > to add
> > > > > > > > > > > > >> ACL
> > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > > >> > > > > > > > like you are describing ACL) via an
> > > > external
> > > > > > > > system
> > > > > > > > > > > > (Argus,
> > > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > > >> > > > > > > > I remember KIP-11 described this,
> but
> > I
> > > > > can't
> > > > > > > find
> > > > > > > > > > the KIP
> > > > > > > > > > > > >> any
> > > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > A: Okay, no problem. Not sure though
> > how
> > > > we
> > > > > are
> > > > > > > > > going
> > > > > > > > > > to
> > > > > > > > > > > > >> handle
> > > > > > > > > > > > >> > > it.
> > > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > > >> > > > > > > > will be committed first and include
> > > > changes
> > > > > to
> > > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > > >> > > > the
> > > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > > >> > > > > > > > Anyway, I added this note to "Open
> > > > > Questions"
> > > > > > > > > section
> > > > > > > > > > so
> > > > > > > > > > > > we
> > > > > > > > > > > > >> don't
> > > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM,
> > Andrii
> > > > > > > > Biletskyi <
> > > > > > > > > > > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > >> > > > > > > > > Today I uploaded the patch that
> > covers
> > > > > some of
> > > > > > > > the
> > > > > > > > > > > > >> discussed
> > > > > > > > > > > > >> > > and
> > > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > > >> > > > > > > > > - removed MaybeOf optional type
> > > > > > > > > > > > >> > > > > > > > > - switched to java protocol
> > > definitions
> > > > > > > > > > > > >> > > > > > > > > - simplified messages (normalized
> > > > configs,
> > > > > > > > removed
> > > > > > > > > > topic
> > > > > > > > > > > > >> marked
> > > > > > > > > > > > >> > > > for
> > > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > >> > > > > > > > > I also updated the KIP-4 with
> > > respective
> > > > > > > changes
> > > > > > > > > and
> > > > > > > > > > > > >> wrote down
> > > > > > > > > > > > >> > > > my
> > > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > > >> > > > > > > > > - Batch Admin Operations ->
> updated
> > > Wire
> > > > > > > > Protocol
> > > > > > > > > > schema
> > > > > > > > > > > > >> > > proposal
> > > > > > > > > > > > >> > > > > > > > > - Remove ClusterMetadata ->
> changed
> > to
> > > > > extend
> > > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > > >> > > > > > > > > - Admin Client -> updated my
> initial
> > > > > proposal
> > > > > > > to
> > > > > > > > > > reflect
> > > > > > > > > > > > >> > > batching
> > > > > > > > > > > > >> > > > > > > > > - Error codes -> proposed
> > fine-grained
> > > > > error
> > > > > > > > code
> > > > > > > > > > > > instead
> > > > > > > > > > > > >> of
> > > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > >> > > > > > > > > I will also send a separate email
> to
> > > > > cover all
> > > > > > > > > > comments
> > > > > > > > > > > > >> from
> > > > > > > > > > > > >> > > this
> > > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM,
> > Gwen
> > > > > Shapira
> > > > > > > <
> > > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >>
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > > >> > > > > > > > >> It actually specifies changes to
> > the
> > > > > Metadata
> > > > > > > > > > protocol,
> > > > > > > > > > > > >> so
> > > > > > > > > > > > >> > > > making
> > > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > > >> > > > > > > > >> both KIPs are consistent in this
> > > regard
> > > > > will
> > > > > > > be
> > > > > > > > > > good.
> > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM,
> > > Gwen
> > > > > > > Shapira
> > > > > > > > <
> > > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > >> > > > > > > > >> > Specifically for ownership, I
> > think
> > > > the
> > > > > > > plan
> > > > > > > > is
> > > > > > > > > > to
> > > > > > > > > > > > add
> > > > > > > > > > > > >> ACL
> > > > > > > > > > > > >> > > (it
> > > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > > >> > > > > > > > >> > like you are describing ACL)
> via
> > an
> > > > > > > external
> > > > > > > > > > system
> > > > > > > > > > > > >> (Argus,
> > > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > > >> > > > > > > > >> > I remember KIP-11 described
> this,
> > > > but I
> > > > > > > can't
> > > > > > > > > > find
> > > > > > > > > > > > the
> > > > > > > > > > > > >> KIP
> > > > > > > > > > > > >> > > any
> > > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > >> > > > > > > > >> > Regardless, I think KIP-4
> focuses
> > > on
> > > > > > > getting
> > > > > > > > > > > > >> information
> > > > > > > > > > > > >> > > that
> > > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > > >> > > > > > > > >> > exists from Kafka brokers, not
> on
> > > > > adding
> > > > > > > > > > information
> > > > > > > > > > > > >> that
> > > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > > >> > > > > > > > >> > should exist but doesn't yet?
> > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37
> AM,
> > > > > Guozhang
> > > > > > > > Wang
> > > > > > > > > <
> > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > >> > > > > > > > >> >> Just want to elaborate a bit
> > more
> > > on
> > > > > the
> > > > > > > > > > > > create-topic
> > > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > > >> > > > > > > > >> >> describe-topic based on
> config /
> > > > > metadata
> > > > > > > in
> > > > > > > > > my
> > > > > > > > > > > > >> previous
> > > > > > > > > > > > >> > > > email
> > > > > > > > > > > > >> > > > > as
> > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The main
> > motivation
> > > > is
> > > > > to
> > > > > > > > have
> > > > > > > > > > some
> > > > > > > > > > > > >> sort of
> > > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > > >> > > > > > > > >> >> mechanisms, which I think is
> > quite
> > > > > > > important
> > > > > > > > > in
> > > > > > > > > > a
> > > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > > >> > > > > /
> > > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > > >> > > > > > > > >> >> architecture: today anyone can
> > > > create
> > > > > > > topics
> > > > > > > > > in
> > > > > > > > > > a
> > > > > > > > > > > > >> shared
> > > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > > >> > > > > > > > >> >> there is no concept or
> > "ownership"
> > > > of
> > > > > > > topics
> > > > > > > > > > that
> > > > > > > > > > > > are
> > > > > > > > > > > > >> > > created
> > > > > > > > > > > > >> > > > > by
> > > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > > >> > > > > > > > >> >> users. For example, at
> LinkedIn
> > we
> > > > > > > basically
> > > > > > > > > > > > >> distinguish
> > > > > > > > > > > > >> > > > topic
> > > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > > >> > > > > > > > >> >> some casual topic name prefix,
> > > which
> > > > > is a
> > > > > > > > bit
> > > > > > > > > > > > awkward
> > > > > > > > > > > > >> and
> > > > > > > > > > > > >> > > > does
> > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > > >> > > > > > > > >> >> we scale our customers. It
> would
> > > be
> > > > > great
> > > > > > > to
> > > > > > > > > use
> > > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics that is
> > > created
> > > > > by me.
> > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics whose
> > > retention
> > > > > time
> > > > > > > is
> > > > > > > > > > > > overriden
> > > > > > > > > > > > >> to X.
> > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > >> > > > > > > > >> >> Describe all topics whose
> > writable
> > > > > group
> > > > > > > > > include
> > > > > > > > > > > > user
> > > > > > > > > > > > >> Y
> > > > > > > > > > > > >> > > (this
> > > > > > > > > > > > >> > > > > is
> > > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > >> > > > > > > > >> >> One possible way to achieve
> this
> > > is
> > > > to
> > > > > > > add a
> > > > > > > > > > > > metadata
> > > > > > > > > > > > >> file
> > > > > > > > > > > > >> > > in
> > > > > > > > > > > > >> > > > > the
> > > > > > > > > > > > >> > > > > > > > >> >> create-topic request, whose
> > value
> > > > will
> > > > > > > also
> > > > > > > > be
> > > > > > > > > > > > >> written ZK
> > > > > > > > > > > > >> > > as
> > > > > > > > > > > > >> > > > we
> > > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > > >> > > > > > > > >> >> topic; then describe-topics
> can
> > > > > choose to
> > > > > > > > > batch
> > > > > > > > > > > > topics
> > > > > > > > > > > > >> > > based
> > > > > > > > > > > > >> > > > on
> > > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > > >> > > > > > > > >> >> regex, 2) config K-V matching,
> > 3)
> > > > > metadata
> > > > > > > > > > regex,
> > > > > > > > > > > > etc.
> > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37
> PM,
> > > > > Guozhang
> > > > > > > > Wang
> > > > > > > > > <
> > > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > > >> > > > > > > > >> >>> Thanks for the updated wiki.
> A
> > > few
> > > > > > > comments
> > > > > > > > > > below:
> > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > >> > > > > > > > >> >>> 1. Error description in
> > > response: I
> > > > > think
> > > > > > > > if
> > > > > > > > > > some
> > > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > > >> > > > > > > > >> >>> several different error cases
> > > then
> > > > we
> > > > > > > > should
> > > > > > > > > > really
> > > > > > > > > > > > >> change
> > > > > > > > > > > > >> > > > it
> > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > > >> > > > > > > > >> >>> codes. In general the
> errorCode
> > > > > itself
> > > > > > > > would
> > > > > > > > > be
> > > > > > > > > > > > >> precise
> > > > > > > > > > > > >> > > and
> > > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > > >> > > > > > > > >> >>> describing the server side
> > > errors.
> > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic request: it
> > > would
> > > > > be
> > > > > > > > great
> > > > > > > > > > to go
> > > > > > > > > > > > >> beyond
> > > > > > > > > > > > >> > > > just
> > > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex for this
> > > request.
> > > > > For
> > > > > > > > > > example, a
> > > > > > > > > > > > >> very
> > > > > > > > > > > > >> > > > common
> > > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > > >> > > > > > > > >> >>> the topic command is to list
> > all
> > > > > topics
> > > > > > > > whose
> > > > > > > > > > > > config
> > > > > > > > > > > > >> A's
> > > > > > > > > > > > >> > > > value
> > > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > > >> > > > > > > > >> >>> topic name regex then we have
> > to
> > > > > first
> > > > > > > > > retrieve
> > > > > > > > > > > > >> __all__
> > > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > > >> > > > > > > > >> >>> description info and then
> > filter
> > > at
> > > > > the
> > > > > > > > > client
> > > > > > > > > > end,
> > > > > > > > > > > > >> which
> > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in create
> topic:
> > > > this
> > > > > is
> > > > > > > > > > related to
> > > > > > > > > > > > >> the
> > > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > > >> > > > > > > > >> >>> maybe we can add another
> > metadata
> > > > > K-V or
> > > > > > > > > just a
> > > > > > > > > > > > >> metadata
> > > > > > > > > > > > >> > > > > string
> > > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > > >> > > > > > > > >> >>> with config K-V in create
> topic
> > > > like
> > > > > we
> > > > > > > did
> > > > > > > > > for
> > > > > > > > > > > > >> offset
> > > > > > > > > > > > >> > > > commit
> > > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > > >> > > > > > > > >> >>> field can be quite useful in
> > > > storing
> > > > > > > > > > information
> > > > > > > > > > > > like
> > > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > > >> > > > > of
> > > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > > >> > > > > > > > >> >>> who issue the create command,
> > > etc,
> > > > > which
> > > > > > > is
> > > > > > > > > > quite
> > > > > > > > > > > > >> > > important
> > > > > > > > > > > > >> > > > > for
> > > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant setting. Then in
> > the
> > > > > > > describe
> > > > > > > > > > topic
> > > > > > > > > > > > >> request
> > > > > > > > > > > > >> > > we
> > > > > > > > > > > > >> > > > > can
> > > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > > >> > > > > > > > >> >>> on regex of the metadata
> field.
> > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all the admin
> > operations
> > > > are
> > > > > > > async
> > > > > > > > > in
> > > > > > > > > > the
> > > > > > > > > > > > >> sense
> > > > > > > > > > > > >> > > > that
> > > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > > >> > > > > > > > >> >>> return once it is written in
> > ZK,
> > > > and
> > > > > that
> > > > > > > > is
> > > > > > > > > > why we
> > > > > > > > > > > > >> need
> > > > > > > > > > > > >> > > > extra
> > > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > > >> > > > > > > > >> >>> like
> > > > testUtil.waitForTopicCreated() /
> > > > > > > > verify
> > > > > > > > > > > > >> partition
> > > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > > >> > > > > > > > >> >>> request, etc. With admin
> > requests
> > > > we
> > > > > > > could
> > > > > > > > > add
> > > > > > > > > > a
> > > > > > > > > > > > >> flag to
> > > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > > >> > > > > > > > >> >>> synchronous requests; when it
> > is
> > > > > turned
> > > > > > > on,
> > > > > > > > > the
> > > > > > > > > > > > >> response
> > > > > > > > > > > > >> > > > will
> > > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > > >> > > > > > > > >> >>> until the request has been
> > > > > completed. And
> > > > > > > > for
> > > > > > > > > > async
> > > > > > > > > > > > >> > > requests
> > > > > > > > > > > > >> > > > > we
> > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > > >> > > > > > > > >> >>> "token" field in the
> response,
> > > and
> > > > > then
> > > > > > > > only
> > > > > > > > > > need a
> > > > > > > > > > > > >> > > general
> > > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > > >> > > > > > > > >> >>> verification request" with
> the
> > > > given
> > > > > > > token
> > > > > > > > to
> > > > > > > > > > check
> > > > > > > > > > > > >> if the
> > > > > > > > > > > > >> > > > > async
> > > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for extending Metadata
> > > > request
> > > > > to
> > > > > > > > > include
> > > > > > > > > > > > >> > > controller /
> > > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > > >> > > > > > > > >> >>> information, and then we can
> > > remove
> > > > > the
> > > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23
> > AM,
> > > > Joel
> > > > > > > > Koshy <
> > > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for sending that out
> > Joe
> > > -
> > > > I
> > > > > > > don't
> > > > > > > > > > think I
> > > > > > > > > > > > >> will be
> > > > > > > > > > > > >> > > > > able
> > > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > > >> > > > > > > > >> >>>> it today, so if notes can be
> > > sent
> > > > > out
> > > > > > > > > > afterward
> > > > > > > > > > > > that
> > > > > > > > > > > > >> > > would
> > > > > > > > > > > > >> > > > be
> > > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at
> > > 09:16:13AM
> > > > > > > -0800,
> > > > > > > > > Gwen
> > > > > > > > > > > > >> Shapira
> > > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for sending this
> out
> > > Joe.
> > > > > > > Looking
> > > > > > > > > > forward
> > > > > > > > > > > > >> to
> > > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at
> 6:46
> > > AM,
> > > > > Joe
> > > > > > > > Stein
> > > > > > > > > <
> > > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just sent out a
> > > google
> > > > > > > hangout
> > > > > > > > > > invite
> > > > > > > > > > > > to
> > > > > > > > > > > > >> all
> > > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I found working
> > on
> > > a
> > > > > KIP.
> > > > > > > If
> > > > > > > > I
> > > > > > > > > > missed
> > > > > > > > > > > > >> anyone
> > > > > > > > > > > > >> > > > in
> > > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > > >> > > > > > > > >> >>>> > > let me know and can
> update
> > > it,
> > > > > np.
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > > We should do this every
> > > > Tuesday
> > > > > @
> > > > > > > 2pm
> > > > > > > > > > Eastern
> > > > > > > > > > > > >> Time.
> > > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > > >> > > > > > > > >> >>>> > > help to make a google
> > > account
> > > > > so we
> > > > > > > > can
> > > > > > > > > > manage
> > > > > > > > > > > > >> > > better?
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > > >> > > > > >
> > > > > > > > > > > > >> > > > >
> > > > > > > > > > > > >> > > >
> > > > > > > > > > > > >> > >
> > > > > > > > > > > > >>
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > > >> > > > > > > > >> >>>> > > in progress and related
> > JIRA
> > > > > that
> > > > > > > are
> > > > > > > > > > > > >> interdependent
> > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at
> > 2:59
> > > > > PM, Jay
> > > > > > > > > > Kreps <
> > > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on Google
> > > hangouts
> > > > > that
> > > > > > > > will
> > > > > > > > > > also
> > > > > > > > > > > > >> record
> > > > > > > > > > > > >> > > > and
> > > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at
> > > 11:49
> > > > > AM,
> > > > > > > > Jeff
> > > > > > > > > > > > Holoman
> > > > > > > > > > > > >> <
> > > > > > > > > > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're happy to send
> > out a
> > > > > Webex
> > > > > > > for
> > > > > > > > > > this
> > > > > > > > > > > > >> purpose.
> > > > > > > > > > > > >> > > We
> > > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions if there is
> > > > > interest and
> > > > > > > > > > publish
> > > > > > > > > > > > >> them
> > > > > > > > > > > > >> > > out.
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015
> at
> > > > > 11:28 AM,
> > > > > > > > Jay
> > > > > > > > > > > > Kreps <
> > > > > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try to get
> the
> > > > > technical
> > > > > > > > > > hang-ups
> > > > > > > > > > > > >> sorted
> > > > > > > > > > > > >> > > > out,
> > > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there is some
> benefit
> > > to
> > > > > live
> > > > > > > > > > discussion
> > > > > > > > > > > > vs
> > > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post
> instructions
> > > and
> > > > > give
> > > > > > > > > > ourselves a
> > > > > > > > > > > > >> few
> > > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that
> time
> > > > would
> > > > > work
> > > > > > > > for
> > > > > > > > > > > > >> me...any
> > > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24,
> 2015
> > at
> > > > > 8:18
> > > > > > > AM,
> > > > > > > > > Joe
> > > > > > > > > > > > Stein
> > > > > > > > > > > > >> <
> > > > > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly would be
> > great
> > > > > maybe
> > > > > > > > like
> > > > > > > > > > every
> > > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't mind
> google
> > > > > hangout
> > > > > > > but
> > > > > > > > > > there
> > > > > > > > > > > > is
> > > > > > > > > > > > >> > > always
> > > > > > > > > > > > >> > > > > some
> > > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we know the
> apache
> > > irc
> > > > > > > channel
> > > > > > > > > > works.
> > > > > > > > > > > > We
> > > > > > > > > > > > >> can
> > > > > > > > > > > > >> > > > start
> > > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull
> > > > > transcripts
> > > > > > > > too
> > > > > > > > > > and
> > > > > > > > > > > > >> > > associate
> > > > > > > > > > > > >> > > > to
> > > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it helpful for
> > > things.
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24,
> > 2015
> > > at
> > > > > 11:10
> > > > > > > > AM,
> > > > > > > > > > Jay
> > > > > > > > > > > > >> Kreps <
> > > > > > > > > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked
> about
> > > > > doing a
> > > > > > > > > Google
> > > > > > > > > > > > >> Hangout to
> > > > > > > > > > > > >> > > > chat
> > > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > generalizing
> > that a
> > > > > little
> > > > > > > > > > > > further...I
> > > > > > > > > > > > >> > > > actually
> > > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > everyone
> > spending a
> > > > > > > > reasonable
> > > > > > > > > > chunk
> > > > > > > > > > > > of
> > > > > > > > > > > > >> > > their
> > > > > > > > > > > > >> > > > > week
> > > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync up once a
> > > week.
> > > > I
> > > > > > > think
> > > > > > > > we
> > > > > > > > > > could
> > > > > > > > > > > > >> use
> > > > > > > > > > > > >> > > time
> > > > > > > > > > > > >> > > > > to
> > > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make
> sure
> > we
> > > > > are on
> > > > > > > > top
> > > > > > > > > of
> > > > > > > > > > > > code
> > > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We can make it
> > > > publicly
> > > > > > > > > > available so
> > > > > > > > > > > > >> that
> > > > > > > > > > > > >> > > any
> > > > > > > > > > > > >> > > > > one
> > > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any interest in
> > > doing
> > > > > this?
> > > > > > > > If
> > > > > > > > > so
> > > > > > > > > > > > I'll
> > > > > > > > > > > > >> try
> > > > > > > > > > > > >> > > to
> > > > > > > > > > > > >> > > > > set
> > > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24,
> > > 2015
> > > > at
> > > > > > > 3:57
> > > > > > > > > AM,
> > > > > > > > > > > > Andrii
> > > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > wrote:
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > I've updated
> > KIP
> > > > > page,
> > > > > > > > fixed
> > > > > > > > > /
> > > > > > > > > > > > >> aligned
> > > > > > > > > > > > >> > > > > document
> > > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > >
> > ...
> >
> > [Message clipped]
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Great.
I want to elaborate this a bit more, to see we are on the same page
concerning the client code.

So with all topic commands being async a client (AdminClient in our
case or any other other client people would like to implement) to support
a blocking operation (which seems to be a natural use-case e.g. for topic
creation): would have to do:
1. issue CreateTopicRequest
2. if successful, in a "while" loop send DescribeTopicRequest and
break the loop once all topics are returned in response (or upon timeout).
3. if unsuccessful throw exception
Would it be okay?

Thanks,
Andrii Biletskyi


On Fri, Mar 20, 2015 at 6:21 PM, Jun Rao <ju...@confluent.io> wrote:

> Andrii,
>
> I think you are right. It seems that only ReassignPartitions needs a
> separate verification request.
>
> Thanks,
>
> Jun
>
> On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Guys,
> > I like this idea too. Let's stick with that. I'll update KIP accordingly.
> >
> > I was also thinking we can avoid adding dedicated status check
> > requests for topic commands. - We have everything in DescribeTopic
> > for that! E.g.:
> > User issued CreateTopic - to check the status client sends DescribeTopic
> > and checks whether is something returned for that topic. The same for
> > alteration, deletion.
> > Btw, PreferredReplica status can be also checked with
> DescribeTopicRequest
> > (head of assigned replicas list == leader).
> > For ReassignPartitions as discussed we'll need to have a separate
> Verify...
> > request.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> >
> > On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> > > +1 on broker writing to ZK for async handling. I was thinking that in
> the
> > > end state the admin requests would be eventually sent to controller
> > either
> > > through re-routing or clients discovering them, instead of letting
> > > controller listen on ZK admin path. But thinking about it a second
> time,
> > I
> > > think it is actually simpler to let controller manage
> > > incoming queued-up admin requests through ZK.
> > >
> > > Guozhang
> > >
> > >
> > > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <jj...@gmail.com>
> wrote:
> > >
> > > > +1 as well. I think it helps to keep the rerouting approach
> orthogonal
> > > > to this KIP.
> > > >
> > > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps wrote:
> > > > > I'm +1 on Jun's suggestion as long as it can work for all the
> > requests.
> > > > >
> > > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Andrii,
> > > > > >
> > > > > > I think we agreed on the following.
> > > > > >
> > > > > > (a) Admin requests can be sent to and handled by any broker.
> > > > > > (b) Admin requests are processed asynchronously, at least for
> now.
> > > > That is,
> > > > > > when the client gets a response, it just means that the request
> is
> > > > > > initiated, but not necessarily completed. Then, it's up to the
> > client
> > > > to
> > > > > > issue another request to check the status for completion.
> > > > > >
> > > > > > To support (a), we were thinking of doing request forwarding to
> the
> > > > > > controller (utilizing KAFKA-1912). I am making an alternative
> > > proposal.
> > > > > > Basically, the broker can just write to ZooKeeper to inform the
> > > > controller
> > > > > > about the request. For example, to handle partitionReassignment,
> > the
> > > > broker
> > > > > > will just write the requested partitions to
> > > /admin/reassign_partitions
> > > > > > (like what AdminUtils currently does) and then send a response to
> > the
> > > > > > client. This shouldn't take long and the implementation will be
> > > simpler
> > > > > > than forwarding the requests to the controller through RPC.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Jun,
> > > > > > >
> > > > > > > I might be wrong but didn't we agree we will let any broker
> from
> > > the
> > > > > > > cluster handle *long-running* admin requests (at this time
> > > > > > preferredReplica
> > > > > > > and
> > > > > > > reassignPartitions), via zk admin path. Thus CreateTopics etc
> > > should
> > > > be
> > > > > > > sent
> > > > > > > only to the controller.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > > >
> > > > > > > > Joel, Andril,
> > > > > > > >
> > > > > > > > I think we agreed that those admin requests can be issued to
> > any
> > > > > > broker.
> > > > > > > > Because of that, there doesn't seem to be a strong need to
> know
> > > the
> > > > > > > > controller. So, perhaps we can proceed by not making any
> change
> > > to
> > > > the
> > > > > > > > format of TMR right now. When we start using create topic
> > request
> > > > in
> > > > > > the
> > > > > > > > producer, we will need a new version of TMR that doesn't
> > trigger
> > > > auto
> > > > > > > topic
> > > > > > > > creation. But that can be done later.
> > > > > > > >
> > > > > > > > As a first cut implementation, I think the broker can just
> > write
> > > > to ZK
> > > > > > > > directly for
> > > > > > > >
> > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > > requests, instead of forwarding them to the controller. This
> > will
> > > > > > > simplify
> > > > > > > > the implementation on the broker side.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > >
> > > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> > > jjkoshy.w@gmail.com>
> > > > > > > wrote:
> > > > > > > >
> > > > > > > > > (Thanks Andrii for the summary)
> > > > > > > > >
> > > > > > > > > For (1) yes we will circle back on that shortly after
> syncing
> > > up
> > > > in
> > > > > > > > > person. I think it is close to getting committed although
> > > > development
> > > > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > > > >
> > > > > > > > > There is one more item we covered at the hangout. i.e.,
> > whether
> > > > we
> > > > > > > > > want to add the coordinator to the topic metadata response
> or
> > > > provide
> > > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > > >
> > > > > > > > > There are two reasons I think we should try and avoid
> adding
> > > the
> > > > > > > > > field:
> > > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > > - If we finally do request rerouting in Kafka then the
> field
> > > > would
> > > > > > add
> > > > > > > > >   little to no value. (It still helps to have a separate
> > > > > > > > >   ClusterMetadataRequest to query for cluster-wide
> > information
> > > > such
> > > > > > as
> > > > > > > > >   'which broker is the controller?' as Joe mentioned.)
> > > > > > > > >
> > > > > > > > > I think it would be cleaner to have an explicit
> > > > > > ClusterMetadataRequest
> > > > > > > > > that you can send to any broker in order to obtain the
> > > controller
> > > > > > (and
> > > > > > > > > in the future possibly other cluster-wide information). I
> > think
> > > > the
> > > > > > > > > main argument against doing this and instead adding it to
> the
> > > > topic
> > > > > > > > > metadata response was convenience - i.e., you don't have to
> > > > discover
> > > > > > > > > the controller in advance. However, I don't see much actual
> > > > > > > > > benefit/convenience in this and in fact think it is a
> > > non-issue.
> > > > Let
> > > > > > > > > me know if I'm overlooking something here.
> > > > > > > > >
> > > > > > > > > As an example, say we need to initiate partition
> reassignment
> > > by
> > > > > > > > > issuing the new ReassignPartitionsRequest to the controller
> > > > (assume
> > > > > > we
> > > > > > > > > already have the desired manual partition assignment).  If
> we
> > > > are to
> > > > > > > > > augment topic metadata response then the flow be something
> > like
> > > > this
> > > > > > :
> > > > > > > > >
> > > > > > > > > - Issue topic metadata request to any broker (and discover
> > the
> > > > > > > > >   controller
> > > > > > > > > - Connect to controller if required (i.e., if the broker
> > above
> > > !=
> > > > > > > > >   controller)
> > > > > > > > > - Issue the partition reassignment request to the
> controller.
> > > > > > > > >
> > > > > > > > > With an explicit cluster metadata request it would be:
> > > > > > > > > - Issue cluster metadata request to any broker
> > > > > > > > > - Connect to controller if required (i.e., if the broker
> > above
> > > !=
> > > > > > > > >   controller)
> > > > > > > > > - Issue the partition reassignment request
> > > > > > > > >
> > > > > > > > > So it seems to add little practical value and bloats topic
> > > > metadata
> > > > > > > > > response with an irrelevant detail.
> > > > > > > > >
> > > > > > > > > The other angle to this is the following - is it a matter
> of
> > > > naming?
> > > > > > > > > Should we just rename topic metadata request/response to
> just
> > > > > > > > > MetadataRequest/Response and add cluster metadata to it? By
> > > that
> > > > same
> > > > > > > > > token should we also allow querying for the consumer
> > > coordinator
> > > > (and
> > > > > > > > > in future transaction coordinator) as well? This leads to a
> > > > bloated
> > > > > > > > > request which isn't very appealing and altogether
> confusing.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Joel
> > > > > > > > >
> > > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > > > > > > > > > Andri,
> > > > > > > > > >
> > > > > > > > > > Thanks for the summary.
> > > > > > > > > >
> > > > > > > > > > 1. I just realized that in order to start working on
> > > > KAFKA-1927, we
> > > > > > > > will
> > > > > > > > > > need to merge the changes to OffsetCommitRequest (from
> > 0.8.2)
> > > > to
> > > > > > > trunk.
> > > > > > > > > > This is planned to be done as part of KAFKA-1634. So, we
> > will
> > > > need
> > > > > > > > > Guozhang
> > > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > > >
> > > > > > > > > > 2. Thinking about this a bit more, if the semantic of
> those
> > > > "write"
> > > > > > > > > > requests is async (i.e., after the client gets a
> response,
> > it
> > > > just
> > > > > > > > means
> > > > > > > > > > that the operation is initiated, but not necessarily
> > > > completed), we
> > > > > > > > don't
> > > > > > > > > > really need to forward the requests to the controller.
> > > > Instead, the
> > > > > > > > > > receiving broker can just write the operation to ZK as
> the
> > > > admin
> > > > > > > > command
> > > > > > > > > > line tool previously does. This will simplify the
> > > > implementation.
> > > > > > > > > >
> > > > > > > > > > 8. There is another implementation detail for describe
> > topic.
> > > > > > > Ideally,
> > > > > > > > we
> > > > > > > > > > want to read the topic config from the broker cache,
> > instead
> > > of
> > > > > > > > > ZooKeeper.
> > > > > > > > > > Currently, every broker reads the topic-level config for
> > all
> > > > > > topics.
> > > > > > > > > > However, it ignores those for topics not hosted on
> itself.
> > > So,
> > > > we
> > > > > > may
> > > > > > > > > need
> > > > > > > > > > to change TopicConfigManager a bit so that it caches the
> > > > configs
> > > > > > for
> > > > > > > > all
> > > > > > > > > > topics.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > >
> > > > > > > > > > Jun
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > >
> > > > > > > > > > > Guys,
> > > > > > > > > > >
> > > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > > Here are the actions points:
> > > > > > > > > > >
> > > > > > > > > > > 1. Q: Get rid of all scala requests objects, use java
> > > > protocol
> > > > > > > > > definitions.
> > > > > > > > > > >     A: Gwen kindly took that (KAFKA-1927). It's
> important
> > > to
> > > > > > speed
> > > > > > > up
> > > > > > > > > > > review procedure
> > > > > > > > > > >          there since this ticket blocks other important
> > > > changes.
> > > > > > > > > > >
> > > > > > > > > > > 2. Q: Generic re-reroute facility vs client maintaining
> > > > cluster
> > > > > > > > state.
> > > > > > > > > > >     A: Jay has added pseudo code to KAFKA-1912 - need
> to
> > > > consider
> > > > > > > > > whether
> > > > > > > > > > > this will be
> > > > > > > > > > >         easy to implement as a server-side feature
> > > (comments
> > > > are
> > > > > > > > > > > welcomed!).
> > > > > > > > > > >
> > > > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > > > >     A: This might be useful for clients, add this to
> > > > > > > > > TopicMetadataResponse
> > > > > > > > > > > (already in KIP).
> > > > > > > > > > >
> > > > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > > > >     A: I will add proposed by Jun solution (using
> > clientId
> > > > for
> > > > > > > that)
> > > > > > > > > to the
> > > > > > > > > > > KIP.
> > > > > > > > > > >
> > > > > > > > > > > 5. Q: Bumping new versions of TMR vs grabbing all
> > protocol
> > > > > > changes
> > > > > > > in
> > > > > > > > > one
> > > > > > > > > > > version.
> > > > > > > > > > >     A: It was decided to try to gather all changes to
> > > > protocol
> > > > > > > > (before
> > > > > > > > > > > release).
> > > > > > > > > > >         In case of TMR it worth checking: KAFKA-2020
> and
> > > > KIP-13
> > > > > > > > > (quotas)
> > > > > > > > > > >
> > > > > > > > > > > 6. Q: JSON lib is needed to deserialize user's input in
> > CLI
> > > > tool.
> > > > > > > > > > >     A: Use jackson for that, /tools project is a
> separate
> > > > jar so
> > > > > > > > > shouldn't
> > > > > > > > > > > be a big deal.
> > > > > > > > > > >
> > > > > > > > > > > 7.  Q: VerifyReassingPartitions vs generic status check
> > > > command.
> > > > > > > > > > >      A: For long-running requests like reassign
> > partitions
> > > > > > > *progress*
> > > > > > > > > check
> > > > > > > > > > > request is useful,
> > > > > > > > > > >          it makes sense to introduce it.
> > > > > > > > > > >
> > > > > > > > > > >  Please add, correct me if I missed something.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Joel,
> > > > > > > > > > > >
> > > > > > > > > > > > You are right, I removed ClusterMetadata because we
> > have
> > > > > > > partially
> > > > > > > > > > > > what we need in TopicMetadata. Also, as Jay pointed
> out
> > > > > > earlier,
> > > > > > > we
> > > > > > > > > > > > would like to have "orthogonal" API, but at the same
> > time
> > > > we
> > > > > > need
> > > > > > > > > > > > to be backward compatible.
> > > > > > > > > > > >
> > > > > > > > > > > > But I like your idea and even have some other
> arguments
> > > for
> > > > > > this
> > > > > > > > > option:
> > > > > > > > > > > > There is also DescribeTopicRequest which was proposed
> > in
> > > > this
> > > > > > > KIP,
> > > > > > > > > > > > it returns topic configs, partitions, replication
> > factor
> > > > plus
> > > > > > > > > partition
> > > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > > leader replica. The later part is really already
> there
> > in
> > > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > > So again we'll have to add stuff to TMR, not to
> > duplicate
> > > > some
> > > > > > > info
> > > > > > > > > in
> > > > > > > > > > > > newly added requests. However, this way we'll end up
> > with
> > > > > > > "monster"
> > > > > > > > > > > > request which returns cluster metadata, topic
> > replication
> > > > and
> > > > > > > > config
> > > > > > > > > info
> > > > > > > > > > > > plus partition replication data. Seems logical to
> split
> > > > TMR to
> > > > > > > > > > > > - ClusterMetadata (brokers + controller, maybe smth
> > else)
> > > > > > > > > > > > - TopicMetadata (topic info + partition details)
> > > > > > > > > > > > But since current TMR is involved in lots of places
> > > > (including
> > > > > > > > > network
> > > > > > > > > > > > client,
> > > > > > > > > > > > as I understand) this might be very serious change
> and
> > it
> > > > > > > probably
> > > > > > > > > makes
> > > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks,
> > > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <
> > > > > > jjkoshy.w@gmail.com
> > > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > >> I may be missing some context but hopefully this
> will
> > > > also be
> > > > > > > > > covered
> > > > > > > > > > > >> today: I thought the earlier proposal where there
> was
> > an
> > > > > > > explicit
> > > > > > > > > > > >> ClusterMetadata request was clearer and explicit.
> > During
> > > > the
> > > > > > > > course
> > > > > > > > > of
> > > > > > > > > > > >> this thread I think the conclusion was that the main
> > > need
> > > > was
> > > > > > > for
> > > > > > > > > > > >> controller information and that can be rolled into
> the
> > > > topic
> > > > > > > > > metadata
> > > > > > > > > > > >> response but that seems a bit irrelevant to topic
> > > > metadata.
> > > > > > > FWIW I
> > > > > > > > > > > >> think the full broker-list is also irrelevant to
> topic
> > > > > > metadata,
> > > > > > > > but
> > > > > > > > > > > >> it is already there and in use. I think there is
> still
> > > > room
> > > > > > for
> > > > > > > an
> > > > > > > > > > > >> explicit ClusterMetadata request since there may be
> > > other
> > > > > > > > > > > >> cluster-level information that we may want to add
> over
> > > > time
> > > > > > (and
> > > > > > > > > that
> > > > > > > > > > > >> have nothing to do with topic metadata).
> > > > > > > > > > > >>
> > > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii
> > > Biletskyi
> > > > > > > wrote:
> > > > > > > > > > > >> > Jun,
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 101. Okay, if you say that such use case is
> > > important. I
> > > > > > also
> > > > > > > > > think
> > > > > > > > > > > >> > using clientId for these purposes is fine - if we
> > > > already
> > > > > > have
> > > > > > > > > this
> > > > > > > > > > > >> field
> > > > > > > > > > > >> > as part of all Wire protocol messages, why not use
> > > that.
> > > > > > > > > > > >> > I will update KIP-4 page if nobody has other ideas
> > > > (which
> > > > > > may
> > > > > > > > > come up
> > > > > > > > > > > >> > during the call today).
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > 102.1 Agree, I'll update the KIP accordingly. I
> > think
> > > > we can
> > > > > > > add
> > > > > > > > > new,
> > > > > > > > > > > >> > fine-grained error codes if some error code
> received
> > > in
> > > > > > > specific
> > > > > > > > > case
> > > > > > > > > > > >> > won't give enough context to return a descriptive
> > > error
> > > > > > > message
> > > > > > > > > for
> > > > > > > > > > > >> user.
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > Look forward to discussing all outstanding issues
> in
> > > > detail
> > > > > > > > today
> > > > > > > > > > > during
> > > > > > > > > > > >> > the call.
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > Thanks,
> > > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > > >> >
> > > > > > > > > > > >> >
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <
> > > > jun@confluent.io
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > > 101. There may be a use case where you only want
> > the
> > > > > > topics
> > > > > > > to
> > > > > > > > > be
> > > > > > > > > > > >> created
> > > > > > > > > > > >> > > manually by admins. Currently, you can do that
> by
> > > > > > disabling
> > > > > > > > auto
> > > > > > > > > > > topic
> > > > > > > > > > > >> > > creation and issue topic creation from the
> > > > TopicCommand.
> > > > > > If
> > > > > > > we
> > > > > > > > > > > >> disable auto
> > > > > > > > > > > >> > > topic creation completely on the broker and
> don't
> > > > have a
> > > > > > way
> > > > > > > > to
> > > > > > > > > > > >> distinguish
> > > > > > > > > > > >> > > between topic creation requests from the regular
> > > > clients
> > > > > > and
> > > > > > > > the
> > > > > > > > > > > >> admin, we
> > > > > > > > > > > >> > > can't support manual topic creation any more. I
> > was
> > > > > > thinking
> > > > > > > > > that
> > > > > > > > > > > >> another
> > > > > > > > > > > >> > > way of distinguishing the clients making the
> topic
> > > > > > creation
> > > > > > > > > requests
> > > > > > > > > > > >> is
> > > > > > > > > > > >> > > using clientId. For example, the admin tool can
> > set
> > > > it to
> > > > > > > > > something
> > > > > > > > > > > >> like
> > > > > > > > > > > >> > > admin and the broker can treat that clientId
> > > > specially.
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > Also, there is a related discussion in
> KAFKA-2020.
> > > > > > > Currently,
> > > > > > > > > we do
> > > > > > > > > > > >> the
> > > > > > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > 1. If leader is not available, we set the
> > partition
> > > > level
> > > > > > > > error
> > > > > > > > > code
> > > > > > > > > > > >> to
> > > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > > >> > > 2. If a non-leader replica is not available, we
> > take
> > > > that
> > > > > > > > > replica
> > > > > > > > > > > out
> > > > > > > > > > > >> of
> > > > > > > > > > > >> > > the assigned replica list and isr in the
> response.
> > > As
> > > > an
> > > > > > > > > indication
> > > > > > > > > > > >> for
> > > > > > > > > > > >> > > doing that, we set the partition level error
> code
> > to
> > > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > This has a few problems. First,
> > ReplicaNotAvailable
> > > > > > probably
> > > > > > > > > > > >> shouldn't be
> > > > > > > > > > > >> > > an error, at least for the normal
> > producer/consumer
> > > > > > clients
> > > > > > > > that
> > > > > > > > > > > just
> > > > > > > > > > > >> want
> > > > > > > > > > > >> > > to find out the leader. Second, it can happen
> that
> > > > both
> > > > > > the
> > > > > > > > > leader
> > > > > > > > > > > and
> > > > > > > > > > > >> > > another replica are not available at the same
> > time.
> > > > There
> > > > > > is
> > > > > > > > no
> > > > > > > > > > > error
> > > > > > > > > > > >> code
> > > > > > > > > > > >> > > to indicate both. Third, even if a replica is
> not
> > > > > > available,
> > > > > > > > > it's
> > > > > > > > > > > >> still
> > > > > > > > > > > >> > > useful to return its replica id since some
> clients
> > > > (e.g.
> > > > > > > admin
> > > > > > > > > tool)
> > > > > > > > > > > >> may
> > > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > One way to address this issue is to always
> return
> > > the
> > > > > > > replica
> > > > > > > > > id for
> > > > > > > > > > > >> > > leader, assigned replicas, and isr regardless of
> > > > whether
> > > > > > the
> > > > > > > > > > > >> corresponding
> > > > > > > > > > > >> > > broker is live or not. Since we also return the
> > list
> > > > of
> > > > > > live
> > > > > > > > > > > brokers,
> > > > > > > > > > > >> the
> > > > > > > > > > > >> > > client can figure out whether a leader or a
> > replica
> > > is
> > > > > > live
> > > > > > > or
> > > > > > > > > not
> > > > > > > > > > > >> and act
> > > > > > > > > > > >> > > accordingly. This way, we don't need to set the
> > > > partition
> > > > > > > > level
> > > > > > > > > > > error
> > > > > > > > > > > >> code
> > > > > > > > > > > >> > > when the leader or a replica is not available.
> > This
> > > > > > doesn't
> > > > > > > > > change
> > > > > > > > > > > >> the wire
> > > > > > > > > > > >> > > protocol, but does change the semantics. Since
> we
> > > are
> > > > > > > evolving
> > > > > > > > > the
> > > > > > > > > > > >> protocol
> > > > > > > > > > > >> > > of TopicMetadataRequest here, we can potentially
> > > > piggyback
> > > > > > > the
> > > > > > > > > > > change.
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > 102.1 For those types of errors due to invalid
> > > input,
> > > > > > > > shouldn't
> > > > > > > > > we
> > > > > > > > > > > >> just
> > > > > > > > > > > >> > > guard it at parameter validation time and throw
> > > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > > >> > > without even sending the request to the broker?
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > Thanks,
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > Jun
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii
> > Biletskyi <
> > > > > > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > >> > >
> > > > > > > > > > > >> > > > Jun,
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > 101. If I understand you correctly, you are
> > saying
> > > > > > future
> > > > > > > > > producer
> > > > > > > > > > > >> > > versions
> > > > > > > > > > > >> > > > (which
> > > > > > > > > > > >> > > > will be ported to TMR_V1) won't be able to
> > > > automatically
> > > > > > > > > create
> > > > > > > > > > > >> topic (if
> > > > > > > > > > > >> > > > we
> > > > > > > > > > > >> > > > unconditionally remove topic creation from
> > there).
> > > > But
> > > > > > we
> > > > > > > > > need to
> > > > > > > > > > > >> this
> > > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > > >> > > > Ok, about your proposal: I'm not a big fan
> too,
> > > > when it
> > > > > > > > comes
> > > > > > > > > to
> > > > > > > > > > > >> > > > differentiating
> > > > > > > > > > > >> > > > clients directly in protocol schema. And also
> > I'm
> > > > not
> > > > > > > sure I
> > > > > > > > > > > >> understand
> > > > > > > > > > > >> > > at
> > > > > > > > > > > >> > > > all why
> > > > > > > > > > > >> > > > auto.create.topics.enable is a server side
> > > > > > configuration.
> > > > > > > > Can
> > > > > > > > > we
> > > > > > > > > > > >> > > deprecate
> > > > > > > > > > > >> > > > this setting
> > > > > > > > > > > >> > > > in future versions, add this setting to
> producer
> > > and
> > > > > > based
> > > > > > > > on
> > > > > > > > > that
> > > > > > > > > > > >> upon
> > > > > > > > > > > >> > > > receiving
> > > > > > > > > > > >> > > > UnknownTopic create topic explicitly by a
> > separate
> > > > > > > producer
> > > > > > > > > call
> > > > > > > > > > > via
> > > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > 102.1. Hm, yes. It's because we want to
> support
> > > > batching
> > > > > > > and
> > > > > > > > > at
> > > > > > > > > > > the
> > > > > > > > > > > >> same
> > > > > > > > > > > >> > > > time we
> > > > > > > > > > > >> > > > want to give descriptive error messages for
> > > clients.
> > > > > > Since
> > > > > > > > > > > >> AdminClient
> > > > > > > > > > > >> > > > holds the context
> > > > > > > > > > > >> > > > to construct such messages (e.g. AdminClient
> > layer
> > > > can
> > > > > > > know
> > > > > > > > > that
> > > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > > >> > > > means two cases: either invalid number - e.g.
> > -1;
> > > or
> > > > > > > > > > > >> replication-factor
> > > > > > > > > > > >> > > was
> > > > > > > > > > > >> > > > provided while
> > > > > > > > > > > >> > > > partitions argument wasn't) - I wrapped
> > responses
> > > in
> > > > > > > > > Exceptions.
> > > > > > > > > > > >> But I'm
> > > > > > > > > > > >> > > > open to any
> > > > > > > > > > > >> > > > other ideas, this was just initial version.
> > > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change that to
> > probably
> > > > some
> > > > > > > other
> > > > > > > > > dto.
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <
> > > > > > > jun@confluent.io>
> > > > > > > > > > > wrote:
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > 101. That's what I was thinking too, but it
> > may
> > > > not be
> > > > > > > > that
> > > > > > > > > > > >> simple. In
> > > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > > >> > > > > we can let it not trigger auto topic
> creation.
> > > > Then,
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > >> producer
> > > > > > > > > > > >> > > > side,
> > > > > > > > > > > >> > > > > if it gets an UnknownTopicException, it can
> > > > explicitly
> > > > > > > > > issue a
> > > > > > > > > > > >> > > > > createTopicRequest for auto topic creation.
> On
> > > the
> > > > > > > > consumer
> > > > > > > > > > > side,
> > > > > > > > > > > >> it
> > > > > > > > > > > >> > > will
> > > > > > > > > > > >> > > > > never issue createTopicRequest. This works
> > when
> > > > auto
> > > > > > > topic
> > > > > > > > > > > >> creation is
> > > > > > > > > > > >> > > > > enabled on the broker side. However, I am
> not
> > > > sure how
> > > > > > > > > things
> > > > > > > > > > > >> will work
> > > > > > > > > > > >> > > > > when auto topic creation is disabled on the
> > > broker
> > > > > > side.
> > > > > > > > In
> > > > > > > > > this
> > > > > > > > > > > >> case,
> > > > > > > > > > > >> > > we
> > > > > > > > > > > >> > > > > want to have a way to manually create a
> topic,
> > > > > > > potentially
> > > > > > > > > > > through
> > > > > > > > > > > >> > > admin
> > > > > > > > > > > >> > > > > commands. However, then we need a way to
> > > > distinguish
> > > > > > > > > > > >> createTopicRequest
> > > > > > > > > > > >> > > > > issued from the producer clients and the
> admin
> > > > tools.
> > > > > > > May
> > > > > > > > > be we
> > > > > > > > > > > >> can
> > > > > > > > > > > >> > > add a
> > > > > > > > > > > >> > > > > new field in createTopicRequest and set it
> > > > differently
> > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > >> producer
> > > > > > > > > > > >> > > > > client and the admin client. However, I am
> not
> > > > sure if
> > > > > > > > > that's
> > > > > > > > > > > the
> > > > > > > > > > > >> best
> > > > > > > > > > > >> > > > > approach.
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > 2. Yes, refactoring existing requests is a
> > > > non-trivial
> > > > > > > > > amount of
> > > > > > > > > > > >> work.
> > > > > > > > > > > >> > > I
> > > > > > > > > > > >> > > > > posted some comments in KAFKA-1927. We will
> > > > probably
> > > > > > > have
> > > > > > > > > to fix
> > > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > > >> > > > > first, before adding the new logic in
> > > KAFKA-1694.
> > > > > > > > > Otherwise, the
> > > > > > > > > > > >> > > changes
> > > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > > >> > > > > 102.1. It's a bit weird that we return
> > exception
> > > > in
> > > > > > the
> > > > > > > > > api. It
> > > > > > > > > > > >> seems
> > > > > > > > > > > >> > > > that
> > > > > > > > > > > >> > > > > we should either return error code or throw
> an
> > > > > > exception
> > > > > > > > > when
> > > > > > > > > > > >> getting
> > > > > > > > > > > >> > > the
> > > > > > > > > > > >> > > > > response state.
> > > > > > > > > > > >> > > > > 102.2. We probably shouldn't explicitly use
> > the
> > > > > > request
> > > > > > > > > object
> > > > > > > > > > > in
> > > > > > > > > > > >> the
> > > > > > > > > > > >> > > > api.
> > > > > > > > > > > >> > > > > Not every request evolution requires an api
> > > > change.
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > Jun
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii
> > > Biletskyi
> > > > <
> > > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > > Thanks for you comments. Answers inline:
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > > 100. There are a few fields such as
> > > > > > ReplicaAssignment,
> > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> > > represented
> > > > as a
> > > > > > > > > string,
> > > > > > > > > > > but
> > > > > > > > > > > >> > > > contain
> > > > > > > > > > > >> > > > > > > composite structures in json. Could we
> > > flatten
> > > > > > them
> > > > > > > > out
> > > > > > > > > > > >> directly in
> > > > > > > > > > > >> > > > the
> > > > > > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > > Yes, now with Admin Client this looks a
> bit
> > > > weird.
> > > > > > My
> > > > > > > > > initial
> > > > > > > > > > > >> > > > motivation
> > > > > > > > > > > >> > > > > > was:
> > > > > > > > > > > >> > > > > > ReassignPartitionCommand accepts input in
> > > json,
> > > > we
> > > > > > > want
> > > > > > > > to
> > > > > > > > > > > >> remain
> > > > > > > > > > > >> > > > tools'
> > > > > > > > > > > >> > > > > > interfaces unchanged, where possible.
> > > > > > > > > > > >> > > > > > If we port it to deserialized format, in
> CLI
> > > > (/tools
> > > > > > > > > project)
> > > > > > > > > > > >> we will
> > > > > > > > > > > >> > > > > have
> > > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > > >> > > > > > json library since /tools is written in
> java
> > > and
> > > > > > we'll
> > > > > > > > > need to
> > > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > > >> > > > > > json file
> > > > > > > > > > > >> > > > > > provided by a user. Can we quickly agree
> on
> > > what
> > > > > > this
> > > > > > > > > library
> > > > > > > > > > > >> should
> > > > > > > > > > > >> > > be
> > > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > > 101. Does TopicMetadataRequest v1 still
> > > trigger
> > > > auto
> > > > > > > > topic
> > > > > > > > > > > >> creation?
> > > > > > > > > > > >> > > > This
> > > > > > > > > > > >> > > > > > > will be a bit weird now that we have a
> > > > separate
> > > > > > > topic
> > > > > > > > > > > >> creation api.
> > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > createTopicRequest
> > > > > > and
> > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > >> > > > > > > v1 will be used in the producer/consumer
> > > > client,
> > > > > > in
> > > > > > > > > addition
> > > > > > > > > > > >> to
> > > > > > > > > > > >> > > admin
> > > > > > > > > > > >> > > > > > > tools? For example, ideally, we don't
> want
> > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > >> > > from
> > > > > > > > > > > >> > > > > the
> > > > > > > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > > I agree, this strange logic should be
> fixed.
> > > > I'm not
> > > > > > > > > confident
> > > > > > > > > > > >> in
> > > > > > > > > > > >> > > this
> > > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > > >> > > > > > correct me if I'm wrong, but it doesn't
> look
> > > > like a
> > > > > > > hard
> > > > > > > > > thing
> > > > > > > > > > > >> to
> > > > > > > > > > > >> > > do, I
> > > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > > >> > > > > > leverage AdminClient for that in Producer
> > and
> > > > > > > > > unconditionally
> > > > > > > > > > > >> remove
> > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > >> > > > > > creation from the TopicMetadataRequest_V1.
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > > 2. I think Jay meant getting rid of scala
> > > > classes
> > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > >> > > did
> > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > >> > > > > > > as a stop-gap thing when adding the new
> > > > requests
> > > > > > for
> > > > > > > > the
> > > > > > > > > > > >> consumers.
> > > > > > > > > > > >> > > > > > > However, the long term plan is to get
> rid
> > of
> > > > all
> > > > > > > those
> > > > > > > > > and
> > > > > > > > > > > >> just
> > > > > > > > > > > >> > > reuse
> > > > > > > > > > > >> > > > > the
> > > > > > > > > > > >> > > > > > > java request/response in the client.
> Since
> > > > this
> > > > > > KIP
> > > > > > > > > proposes
> > > > > > > > > > > >> to
> > > > > > > > > > > >> > > add a
> > > > > > > > > > > >> > > > > > > significant number of new requests,
> > perhaps
> > > we
> > > > > > > should
> > > > > > > > > bite
> > > > > > > > > > > the
> > > > > > > > > > > >> > > bullet
> > > > > > > > > > > >> > > > > to
> > > > > > > > > > > >> > > > > > > clean up the existing scala requests
> first
> > > > before
> > > > > > > > > adding new
> > > > > > > > > > > >> ones?
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > > Yes, looks like I misunderstood the point
> of
> > > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > > >> > > > > > will
> > > > > > > > > > > >> > > > > > rework that. The only thing is that I
> don't
> > > see
> > > > any
> > > > > > > > > example
> > > > > > > > > > > how
> > > > > > > > > > > >> it
> > > > > > > > > > > >> > > was
> > > > > > > > > > > >> > > > > done
> > > > > > > > > > > >> > > > > > for at
> > > > > > > > > > > >> > > > > > least one existing protocol message. Thus,
> > as
> > > I
> > > > > > > > > understand, I
> > > > > > > > > > > >> have to
> > > > > > > > > > > >> > > > > think
> > > > > > > > > > > >> > > > > > how we
> > > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > > >> > > > > > Re porting all existing RQ/RP in this
> patch.
> > > > Sounds
> > > > > > > > > > > reasonable,
> > > > > > > > > > > >> but
> > > > > > > > > > > >> > > if
> > > > > > > > > > > >> > > > > it's
> > > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > > >> > > > > > requirement to have Admin KIP done, I'm
> > afraid
> > > > this
> > > > > > > can
> > > > > > > > > be a
> > > > > > > > > > > >> serious
> > > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > > >> > > > > > There are 13 protocol messages and all
> that
> > > > would
> > > > > > > > require
> > > > > > > > > not
> > > > > > > > > > > >> only
> > > > > > > > > > > >> > > unit
> > > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > > >> > > > > > intensive manual testing, no? I'm afraid
> I'm
> > > > not the
> > > > > > > > > right guy
> > > > > > > > > > > >> to
> > > > > > > > > > > >> > > cover
> > > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > > >> > > > > > Kafka core internals :). Let me know your
> > > > thoughts
> > > > > > on
> > > > > > > > this
> > > > > > > > > > > >> item. Btw
> > > > > > > > > > > >> > > > > there
> > > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > > >> > >
> https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > > >> > > > ).
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <
> > > > > > > > > jun@confluent.io>
> > > > > > > > > > > >> wrote:
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > > 100. There are a few fields such as
> > > > > > > ReplicaAssignment,
> > > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> > > represented
> > > > as a
> > > > > > > > > string,
> > > > > > > > > > > but
> > > > > > > > > > > >> > > > contain
> > > > > > > > > > > >> > > > > > > composite structures in json. Could we
> > > flatten
> > > > > > them
> > > > > > > > out
> > > > > > > > > > > >> directly in
> > > > > > > > > > > >> > > > the
> > > > > > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > > 101. Does TopicMetadataRequest v1 still
> > > > trigger
> > > > > > auto
> > > > > > > > > topic
> > > > > > > > > > > >> > > creation?
> > > > > > > > > > > >> > > > > This
> > > > > > > > > > > >> > > > > > > will be a bit weird now that we have a
> > > > separate
> > > > > > > topic
> > > > > > > > > > > >> creation api.
> > > > > > > > > > > >> > > > > Have
> > > > > > > > > > > >> > > > > > > you thought about how the new
> > > > createTopicRequest
> > > > > > and
> > > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > > >> > > > > > > v1 will be used in the producer/consumer
> > > > client,
> > > > > > in
> > > > > > > > > addition
> > > > > > > > > > > >> to
> > > > > > > > > > > >> > > admin
> > > > > > > > > > > >> > > > > > > tools? For example, ideally, we don't
> want
> > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > >> > > from
> > > > > > > > > > > >> > > > > the
> > > > > > > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > > 2. I think Jay meant getting rid of
> scala
> > > > classes
> > > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > > >> > > did
> > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > >> > > > > > > as a stop-gap thing when adding the new
> > > > requests
> > > > > > for
> > > > > > > > the
> > > > > > > > > > > >> consumers.
> > > > > > > > > > > >> > > > > > > However, the long term plan is to get
> rid
> > of
> > > > all
> > > > > > > those
> > > > > > > > > and
> > > > > > > > > > > >> just
> > > > > > > > > > > >> > > reuse
> > > > > > > > > > > >> > > > > the
> > > > > > > > > > > >> > > > > > > java request/response in the client.
> Since
> > > > this
> > > > > > KIP
> > > > > > > > > proposes
> > > > > > > > > > > >> to
> > > > > > > > > > > >> > > add a
> > > > > > > > > > > >> > > > > > > significant number of new requests,
> > perhaps
> > > we
> > > > > > > should
> > > > > > > > > bite
> > > > > > > > > > > the
> > > > > > > > > > > >> > > bullet
> > > > > > > > > > > >> > > > > to
> > > > > > > > > > > >> > > > > > > clean up the existing scala requests
> first
> > > > before
> > > > > > > > > adding new
> > > > > > > > > > > >> ones?
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii
> > > > Biletskyi
> > > > > > <
> > > > > > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > As said above - I list again all
> > comments
> > > > from
> > > > > > > this
> > > > > > > > > thread
> > > > > > > > > > > >> so we
> > > > > > > > > > > >> > > > > > > > can see what's left and finalize all
> > > pending
> > > > > > > issues.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > > >> > > > > > > > 1. This is much needed functionality,
> > but
> > > > there
> > > > > > > are
> > > > > > > > a
> > > > > > > > > lot
> > > > > > > > > > > >> of the
> > > > > > > > > > > >> > > so
> > > > > > > > > > > >> > > > > > let's
> > > > > > > > > > > >> > > > > > > > really think these protocols through.
> We
> > > > really
> > > > > > > want
> > > > > > > > > to
> > > > > > > > > > > end
> > > > > > > > > > > >> up
> > > > > > > > > > > >> > > > with a
> > > > > > > > > > > >> > > > > > set
> > > > > > > > > > > >> > > > > > > > of well thought-out, orthoganol apis.
> > For
> > > > this
> > > > > > > > reason
> > > > > > > > > I
> > > > > > > > > > > >> think it
> > > > > > > > > > > >> > > is
> > > > > > > > > > > >> > > > > > > really
> > > > > > > > > > > >> > > > > > > > important to think through the end
> state
> > > > even if
> > > > > > > > that
> > > > > > > > > > > >> includes
> > > > > > > > > > > >> > > APIs
> > > > > > > > > > > >> > > > > we
> > > > > > > > > > > >> > > > > > > > won't implement in the first phase.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: Definitely behind this. Would
> > > appreciate
> > > > if
> > > > > > > there
> > > > > > > > > are
> > > > > > > > > > > >> concrete
> > > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 2. Let's please please please wait
> until
> > > we
> > > > have
> > > > > > > > > switched
> > > > > > > > > > > >> the
> > > > > > > > > > > >> > > > server
> > > > > > > > > > > >> > > > > > over
> > > > > > > > > > > >> > > > > > > > to the new java protocol definitions.
> If
> > > we
> > > > add
> > > > > > > > upteen
> > > > > > > > > > > more
> > > > > > > > > > > >> ad
> > > > > > > > > > > >> > > hoc
> > > > > > > > > > > >> > > > > > scala
> > > > > > > > > > > >> > > > > > > > objects that is just generating more
> > work
> > > > for
> > > > > > the
> > > > > > > > > > > >> conversion we
> > > > > > > > > > > >> > > > know
> > > > > > > > > > > >> > > > > we
> > > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed
> > > scala
> > > > > > > > protocol
> > > > > > > > > > > >> classes.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 3. This proposal introduces a new type
> > of
> > > > > > optional
> > > > > > > > > > > >> parameter.
> > > > > > > > > > > >> > > This
> > > > > > > > > > > >> > > > is
> > > > > > > > > > > >> > > > > > > > inconsistent with everything else in
> the
> > > > > > protocol
> > > > > > > > > where we
> > > > > > > > > > > >> use -1
> > > > > > > > > > > >> > > > or
> > > > > > > > > > > >> > > > > > some
> > > > > > > > > > > >> > > > > > > > other marker value. You could argue
> > either
> > > > way
> > > > > > but
> > > > > > > > > let's
> > > > > > > > > > > >> stick
> > > > > > > > > > > >> > > with
> > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > >> > > > > > > > for consistency. For clients that
> > > > implemented
> > > > > > the
> > > > > > > > > protocol
> > > > > > > > > > > >> in a
> > > > > > > > > > > >> > > > > better
> > > > > > > > > > > >> > > > > > > way
> > > > > > > > > > > >> > > > > > > > than our scala code these basic
> > primitives
> > > > are
> > > > > > > hard
> > > > > > > > to
> > > > > > > > > > > >> change.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed
> > > > MaybeOf
> > > > > > > type
> > > > > > > > > and
> > > > > > > > > > > >> changed
> > > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This seems to
> > > duplicate
> > > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > > >> > > > > which
> > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > >> > > > > > > > brokers, topics, and partitions. I
> think
> > > we
> > > > > > should
> > > > > > > > > rename
> > > > > > > > > > > >> that
> > > > > > > > > > > >> > > > > request
> > > > > > > > > > > >> > > > > > > > ClusterMetadataRequest (or just
> > > > MetadataRequest)
> > > > > > > and
> > > > > > > > > > > >> include the
> > > > > > > > > > > >> > > id
> > > > > > > > > > > >> > > > > of
> > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > >> > > > > > > > controller. Or are there other things
> we
> > > > could
> > > > > > add
> > > > > > > > > here?
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: I agree. Updated the KIP. Let's
> > extends
> > > > > > > > > TopicMetadata
> > > > > > > > > > > to
> > > > > > > > > > > >> > > > version 2
> > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 5. We have a tendency to try to make a
> > lot
> > > > of
> > > > > > > > requests
> > > > > > > > > > > that
> > > > > > > > > > > >> can
> > > > > > > > > > > >> > > > only
> > > > > > > > > > > >> > > > > go
> > > > > > > > > > > >> > > > > > > to
> > > > > > > > > > > >> > > > > > > > particular nodes. This adds a lot of
> > > burden
> > > > for
> > > > > > > > client
> > > > > > > > > > > >> > > > > implementations
> > > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > > >> > > > > > > > sounds easy but each discovery can
> fail
> > in
> > > > many
> > > > > > > > parts
> > > > > > > > > so
> > > > > > > > > > > it
> > > > > > > > > > > >> ends
> > > > > > > > > > > >> > > up
> > > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > > >> > > > > > > > full state machine to do right). I
> think
> > > we
> > > > > > should
> > > > > > > > > > > consider
> > > > > > > > > > > >> > > making
> > > > > > > > > > > >> > > > > > admin
> > > > > > > > > > > >> > > > > > > > commands and ideally as many of the
> > other
> > > > apis
> > > > > > as
> > > > > > > > > possible
> > > > > > > > > > > >> > > > available
> > > > > > > > > > > >> > > > > on
> > > > > > > > > > > >> > > > > > > all
> > > > > > > > > > > >> > > > > > > > brokers and just redirect to the
> > > controller
> > > > on
> > > > > > the
> > > > > > > > > broker
> > > > > > > > > > > >> side.
> > > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > > >> > > > > > > > there would be a general way to
> > > encapsulate
> > > > this
> > > > > > > > > > > re-routing
> > > > > > > > > > > >> > > > behavior.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: It's a very interesting idea, but
> > seems
> > > > there
> > > > > > > are
> > > > > > > > > some
> > > > > > > > > > > >> > > concerns
> > > > > > > > > > > >> > > > > > about
> > > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > > >> > > > > > > > feature (like performance
> > considerations,
> > > > how
> > > > > > this
> > > > > > > > > will
> > > > > > > > > > > >> > > complicate
> > > > > > > > > > > >> > > > > > server
> > > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > > >> > > > > > > > I believe this shouldn't be a blocker.
> > If
> > > > this
> > > > > > > > > feature is
> > > > > > > > > > > >> > > > implemented
> > > > > > > > > > > >> > > > > > at
> > > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > > >> > > > > > > > point it won't affect Admin changes -
> at
> > > > least
> > > > > > no
> > > > > > > > > changes
> > > > > > > > > > > to
> > > > > > > > > > > >> > > public
> > > > > > > > > > > >> > > > > API
> > > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 6. We should probably normalize the
> key
> > > > value
> > > > > > > pairs
> > > > > > > > > used
> > > > > > > > > > > for
> > > > > > > > > > > >> > > > configs
> > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > >> > > > > > > > than embedding a new formatting. So
> two
> > > > strings
> > > > > > > > rather
> > > > > > > > > > > than
> > > > > > > > > > > >> one
> > > > > > > > > > > >> > > > with
> > > > > > > > > > > >> > > > > an
> > > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> > normalized
> > > > > > configs
> > > > > > > > and
> > > > > > > > > > > >> changed
> > > > > > > > > > > >> > > > > protocol
> > > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 7. Is the postcondition of these APIs
> > that
> > > > the
> > > > > > > > > command has
> > > > > > > > > > > >> begun
> > > > > > > > > > > >> > > or
> > > > > > > > > > > >> > > > > > that
> > > > > > > > > > > >> > > > > > > > the command has been completed? It is
> a
> > > lot
> > > > more
> > > > > > > > > usable if
> > > > > > > > > > > >> the
> > > > > > > > > > > >> > > > > command
> > > > > > > > > > > >> > > > > > > has
> > > > > > > > > > > >> > > > > > > > been completed so you know that if you
> > > > create a
> > > > > > > > topic
> > > > > > > > > and
> > > > > > > > > > > >> then
> > > > > > > > > > > >> > > > > publish
> > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > >> > > > > > > > it you won't get an exception about
> > there
> > > > being
> > > > > > no
> > > > > > > > > such
> > > > > > > > > > > >> topic.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: For long running requests (like
> > > reassign
> > > > > > > > > partitions) -
> > > > > > > > > > > >> the
> > > > > > > > > > > >> > > post
> > > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > > >> > > > > > > > command has begun - so we don't block
> > the
> > > > > > client.
> > > > > > > In
> > > > > > > > > case
> > > > > > > > > > > >> of your
> > > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > > >> > > > > > > > topic commands, this will be
> refactored
> > > and
> > > > > > topic
> > > > > > > > > commands
> > > > > > > > > > > >> will
> > > > > > > > > > > >> > > be
> > > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > > >> > > > > > > > immediately, since the Controller will
> > > serve
> > > > > > Admin
> > > > > > > > > > > requests
> > > > > > > > > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 8. Describe topic and list topics
> > > duplicate
> > > > a
> > > > > > lot
> > > > > > > of
> > > > > > > > > stuff
> > > > > > > > > > > >> in the
> > > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > > >> > > > > > > > request. Is there a reason to give
> back
> > > > topics
> > > > > > > > marked
> > > > > > > > > for
> > > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > > >> > > > > > > > like if we just make the
> post-condition
> > of
> > > > the
> > > > > > > > delete
> > > > > > > > > > > >> command be
> > > > > > > > > > > >> > > > that
> > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > >> > > > > > > > topic is deleted that will get rid of
> > the
> > > > need
> > > > > > for
> > > > > > > > > this
> > > > > > > > > > > >> right?
> > > > > > > > > > > >> > > And
> > > > > > > > > > > >> > > > it
> > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed
> > > > topics
> > > > > > > marked
> > > > > > > > > for
> > > > > > > > > > > >> deletion
> > > > > > > > > > > >> > > > in
> > > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 9. Should we consider batching these
> > > > requests?
> > > > > > We
> > > > > > > > have
> > > > > > > > > > > >> generally
> > > > > > > > > > > >> > > > > tried
> > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > >> > > > > > > > allow multiple operations to be
> batched.
> > > My
> > > > > > > > suspicion
> > > > > > > > > is
> > > > > > > > > > > >> that
> > > > > > > > > > > >> > > > without
> > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > >> > > > > > > > we will get a lot of code that does
> > > > something
> > > > > > like
> > > > > > > > > > > >> > > > > > > >    for(topic:
> adminClient.listTopics())
> > > > > > > > > > > >> > > > > > > >       adminClient.describeTopic(topic)
> > > > > > > > > > > >> > > > > > > > this code will work great when you
> test
> > > on 5
> > > > > > > topics
> > > > > > > > > but
> > > > > > > > > > > not
> > > > > > > > > > > >> do as
> > > > > > > > > > > >> > > > > well
> > > > > > > > > > > >> > > > > > if
> > > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check
> "Topic
> > > > Admin
> > > > > > > > Schema"
> > > > > > > > > > > >> section.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 10. I think we should also discuss how
> > we
> > > > want
> > > > > > to
> > > > > > > > > expose a
> > > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > > >> > > > > > > > client api for these operations.
> > Currently
> > > > > > people
> > > > > > > > > rely on
> > > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > > >> > > > > > > which
> > > > > > > > > > > >> > > > > > > > is totally sketchy. I think we
> probably
> > > need
> > > > > > > another
> > > > > > > > > > > client
> > > > > > > > > > > >> under
> > > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > > >> > > > > > > > that exposes administrative
> > functionality.
> > > > We
> > > > > > will
> > > > > > > > > need
> > > > > > > > > > > >> this just
> > > > > > > > > > > >> > > > to
> > > > > > > > > > > >> > > > > > > > properly test the new apis, I suspect.
> > We
> > > > should
> > > > > > > > > figure
> > > > > > > > > > > out
> > > > > > > > > > > >> that
> > > > > > > > > > > >> > > > API.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check
> "Admin
> > > > Client"
> > > > > > > > > section
> > > > > > > > > > > >> with an
> > > > > > > > > > > >> > > > > > initial
> > > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 11. The other information that would
> be
> > > > really
> > > > > > > > useful
> > > > > > > > > to
> > > > > > > > > > > get
> > > > > > > > > > > >> > > would
> > > > > > > > > > > >> > > > be
> > > > > > > > > > > >> > > > > > > > information about partitions--how much
> > > data
> > > > is
> > > > > > in
> > > > > > > > the
> > > > > > > > > > > >> partition,
> > > > > > > > > > > >> > > > what
> > > > > > > > > > > >> > > > > > are
> > > > > > > > > > > >> > > > > > > > the segment offsets, what is the
> log-end
> > > > offset
> > > > > > > > (i.e.
> > > > > > > > > last
> > > > > > > > > > > >> > > offset),
> > > > > > > > > > > >> > > > > > what
> > > > > > > > > > > >> > > > > > > is
> > > > > > > > > > > >> > > > > > > > the compaction point, etc. I think
> that
> > > done
> > > > > > right
> > > > > > > > > this
> > > > > > > > > > > >> would be
> > > > > > > > > > > >> > > > the
> > > > > > > > > > > >> > > > > > > > successor to the very awkward
> > > OffsetRequest
> > > > we
> > > > > > > have
> > > > > > > > > today.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: I removed
> ConsumerGroupOffsetsRequest
> > > in
> > > > the
> > > > > > > > latest
> > > > > > > > > > > >> patch. I
> > > > > > > > > > > >> > > > > believe
> > > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > > >> > > > > > > > be resolved in a separate KIP / jira
> > > ticket.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 12. Generally we can do good error
> > > handling
> > > > > > > without
> > > > > > > > > > > needing
> > > > > > > > > > > >> > > custom
> > > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > > >> > > > > > > > messages. I.e. generally the client
> has
> > > the
> > > > > > > context
> > > > > > > > to
> > > > > > > > > > > know
> > > > > > > > > > > >> that
> > > > > > > > > > > >> > > if
> > > > > > > > > > > >> > > > > it
> > > > > > > > > > > >> > > > > > > got
> > > > > > > > > > > >> > > > > > > > an error that the topic doesn't exist
> to
> > > say
> > > > > > > "Topic
> > > > > > > > X
> > > > > > > > > > > >> doesn't
> > > > > > > > > > > >> > > > exist"
> > > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > > >> > > > > > > > than "error code 14" (or whatever).
> > Maybe
> > > > there
> > > > > > > are
> > > > > > > > > > > specific
> > > > > > > > > > > >> > > cases
> > > > > > > > > > > >> > > > > > where
> > > > > > > > > > > >> > > > > > > > this is hard? If we want to add
> > > server-side
> > > > > > error
> > > > > > > > > messages
> > > > > > > > > > > >> we
> > > > > > > > > > > >> > > > really
> > > > > > > > > > > >> > > > > do
> > > > > > > > > > > >> > > > > > > > need to do this in a consistent way
> > across
> > > > the
> > > > > > > > > protocol.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check
> > > "Protocol
> > > > > > > Errors"
> > > > > > > > > > > >> section. I
> > > > > > > > > > > >> > > > added
> > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > >> > > > > > > > comprehensive, fine-grained list of
> > error
> > > > codes.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > > > >> > > > > > > > 13. Describe topic request: it would
> be
> > > > great to
> > > > > > > go
> > > > > > > > > beyond
> > > > > > > > > > > >> just
> > > > > > > > > > > >> > > > > > batching
> > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > >> > > > > > > > topic name regex for this request. For
> > > > example,
> > > > > > a
> > > > > > > > very
> > > > > > > > > > > >> common use
> > > > > > > > > > > >> > > > > case
> > > > > > > > > > > >> > > > > > of
> > > > > > > > > > > >> > > > > > > > the topic command is to list all
> topics
> > > > whose
> > > > > > > config
> > > > > > > > > A's
> > > > > > > > > > > >> value is
> > > > > > > > > > > >> > > > B.
> > > > > > > > > > > >> > > > > > With
> > > > > > > > > > > >> > > > > > > > topic name regex then we have to first
> > > > retrieve
> > > > > > > > > __all__
> > > > > > > > > > > >> topics's
> > > > > > > > > > > >> > > > > > > > description info and then filter at
> the
> > > > client
> > > > > > > end,
> > > > > > > > > which
> > > > > > > > > > > >> will
> > > > > > > > > > > >> > > be a
> > > > > > > > > > > >> > > > > > huge
> > > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > > >> > > > > > > > 14. Config K-Vs in create topic: this
> is
> > > > related
> > > > > > > to
> > > > > > > > > the
> > > > > > > > > > > >> previous
> > > > > > > > > > > >> > > > > point;
> > > > > > > > > > > >> > > > > > > > maybe we can add another metadata K-V
> or
> > > > just a
> > > > > > > > > metadata
> > > > > > > > > > > >> string
> > > > > > > > > > > >> > > > along
> > > > > > > > > > > >> > > > > > > side
> > > > > > > > > > > >> > > > > > > > with config K-V in create topic like
> we
> > > did
> > > > for
> > > > > > > > offset
> > > > > > > > > > > >> commit
> > > > > > > > > > > >> > > > > request.
> > > > > > > > > > > >> > > > > > > This
> > > > > > > > > > > >> > > > > > > > field can be quite useful in storing
> > > > information
> > > > > > > > like
> > > > > > > > > > > >> "owner" of
> > > > > > > > > > > >> > > > the
> > > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > > >> > > > > > > > who issue the create command, etc,
> which
> > > is
> > > > > > quite
> > > > > > > > > > > important
> > > > > > > > > > > >> for a
> > > > > > > > > > > >> > > > > > > > multi-tenant setting. Then in the
> > describe
> > > > topic
> > > > > > > > > request
> > > > > > > > > > > we
> > > > > > > > > > > >> can
> > > > > > > > > > > >> > > > also
> > > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > > >> > > > > > > > on regex of the metadata field.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: As discussed it is very interesting
> > but
> > > > can
> > > > > > be
> > > > > > > > > > > >> implemented
> > > > > > > > > > > >> > > later
> > > > > > > > > > > >> > > > > > after
> > > > > > > > > > > >> > > > > > > > we have some basic functionality
> there.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > 15. Today all the admin operations are
> > > > async in
> > > > > > > the
> > > > > > > > > sense
> > > > > > > > > > > >> that
> > > > > > > > > > > >> > > > > command
> > > > > > > > > > > >> > > > > > > will
> > > > > > > > > > > >> > > > > > > > return once it is written in ZK, and
> > that
> > > > is why
> > > > > > > we
> > > > > > > > > need
> > > > > > > > > > > >> extra
> > > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > > >> > > > > > > > like testUtil.waitForTopicCreated() /
> > > verify
> > > > > > > > partition
> > > > > > > > > > > >> > > reassignment
> > > > > > > > > > > >> > > > > > > > request, etc. With admin requests we
> > could
> > > > add a
> > > > > > > > flag
> > > > > > > > > to
> > > > > > > > > > > >> enable /
> > > > > > > > > > > >> > > > > > disable
> > > > > > > > > > > >> > > > > > > > synchronous requests; when it is
> turned
> > > on,
> > > > the
> > > > > > > > > response
> > > > > > > > > > > >> will not
> > > > > > > > > > > >> > > > > > return
> > > > > > > > > > > >> > > > > > > > until the request has been completed.
> > And
> > > > for
> > > > > > > async
> > > > > > > > > > > >> requests we
> > > > > > > > > > > >> > > can
> > > > > > > > > > > >> > > > > > add a
> > > > > > > > > > > >> > > > > > > > "token" field in the response, and
> then
> > > only
> > > > > > need
> > > > > > > a
> > > > > > > > > > > general
> > > > > > > > > > > >> > > "admin
> > > > > > > > > > > >> > > > > > > > verification request" with the given
> > token
> > > > to
> > > > > > > check
> > > > > > > > > if the
> > > > > > > > > > > >> async
> > > > > > > > > > > >> > > > > > request
> > > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: I see your point. My idea was to
> > > provide
> > > > > > > specific
> > > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > > >> > > > > > per
> > > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > > >> > > > > > > > long running request, where needed. We
> > can
> > > > do it
> > > > > > > the
> > > > > > > > > way
> > > > > > > > > > > you
> > > > > > > > > > > >> > > > suggest.
> > > > > > > > > > > >> > > > > > The
> > > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > > >> > > > > > > > concern is that introducing a token we
> > > again
> > > > > > will
> > > > > > > > make
> > > > > > > > > > > >> schema
> > > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > > >> > > > > > > We
> > > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > > >> > > > > > > > to do similar thing introducing single
> > > > > > > AdminRequest
> > > > > > > > > for
> > > > > > > > > > > all
> > > > > > > > > > > >> topic
> > > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > > >> > > > > > > > this idea because we wanted to have
> > schema
> > > > > > > defined.
> > > > > > > > So
> > > > > > > > > > > this
> > > > > > > > > > > >> is
> > > > > > > > > > > >> > > > more a
> > > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > > >> > > > > > > > a) have fixed schema but introduce
> each
> > > > time new
> > > > > > > > > > > >> Verify...Request
> > > > > > > > > > > >> > > > for
> > > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > > >> > > > > > > > b) use one request for verification
> but
> > > > > > generalize
> > > > > > > > it
> > > > > > > > > with
> > > > > > > > > > > >> token
> > > > > > > > > > > >> > > > > > > > I'm fine with whatever decision
> > community
> > > > come
> > > > > > to.
> > > > > > > > > Just
> > > > > > > > > > > let
> > > > > > > > > > > >> me
> > > > > > > > > > > >> > > know
> > > > > > > > > > > >> > > > > > your
> > > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > > >> > > > > > > > 16. Specifically for ownership, I
> think
> > > the
> > > > plan
> > > > > > > is
> > > > > > > > > to add
> > > > > > > > > > > >> ACL
> > > > > > > > > > > >> > > (it
> > > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > > >> > > > > > > > like you are describing ACL) via an
> > > external
> > > > > > > system
> > > > > > > > > > > (Argus,
> > > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > > >> > > > > > > > I remember KIP-11 described this, but
> I
> > > > can't
> > > > > > find
> > > > > > > > > the KIP
> > > > > > > > > > > >> any
> > > > > > > > > > > >> > > > > longer.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > A: Okay, no problem. Not sure though
> how
> > > we
> > > > are
> > > > > > > > going
> > > > > > > > > to
> > > > > > > > > > > >> handle
> > > > > > > > > > > >> > > it.
> > > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > > >> > > > > > > > will be committed first and include
> > > changes
> > > > to
> > > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > > >> > > > the
> > > > > > > > > > > >> > > > > > > later
> > > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > > >> > > > > > > > Anyway, I added this note to "Open
> > > > Questions"
> > > > > > > > section
> > > > > > > > > so
> > > > > > > > > > > we
> > > > > > > > > > > >> don't
> > > > > > > > > > > >> > > > > miss
> > > > > > > > > > > >> > > > > > > this
> > > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM,
> Andrii
> > > > > > > Biletskyi <
> > > > > > > > > > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > >> > > > > > > > > Today I uploaded the patch that
> covers
> > > > some of
> > > > > > > the
> > > > > > > > > > > >> discussed
> > > > > > > > > > > >> > > and
> > > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > > >> > > > > > > > > - removed MaybeOf optional type
> > > > > > > > > > > >> > > > > > > > > - switched to java protocol
> > definitions
> > > > > > > > > > > >> > > > > > > > > - simplified messages (normalized
> > > configs,
> > > > > > > removed
> > > > > > > > > topic
> > > > > > > > > > > >> marked
> > > > > > > > > > > >> > > > for
> > > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > >> > > > > > > > > I also updated the KIP-4 with
> > respective
> > > > > > changes
> > > > > > > > and
> > > > > > > > > > > >> wrote down
> > > > > > > > > > > >> > > > my
> > > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > > >> > > > > > > > > - Batch Admin Operations -> updated
> > Wire
> > > > > > > Protocol
> > > > > > > > > schema
> > > > > > > > > > > >> > > proposal
> > > > > > > > > > > >> > > > > > > > > - Remove ClusterMetadata -> changed
> to
> > > > extend
> > > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > > >> > > > > > > > > - Admin Client -> updated my initial
> > > > proposal
> > > > > > to
> > > > > > > > > reflect
> > > > > > > > > > > >> > > batching
> > > > > > > > > > > >> > > > > > > > > - Error codes -> proposed
> fine-grained
> > > > error
> > > > > > > code
> > > > > > > > > > > instead
> > > > > > > > > > > >> of
> > > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > >> > > > > > > > > I will also send a separate email to
> > > > cover all
> > > > > > > > > comments
> > > > > > > > > > > >> from
> > > > > > > > > > > >> > > this
> > > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM,
> Gwen
> > > > Shapira
> > > > > > <
> > > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > >
> > > > > > > > > > > >>
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > > >> > > > > > > > >> It actually specifies changes to
> the
> > > > Metadata
> > > > > > > > > protocol,
> > > > > > > > > > > >> so
> > > > > > > > > > > >> > > > making
> > > > > > > > > > > >> > > > > > sure
> > > > > > > > > > > >> > > > > > > > >> both KIPs are consistent in this
> > regard
> > > > will
> > > > > > be
> > > > > > > > > good.
> > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM,
> > Gwen
> > > > > > Shapira
> > > > > > > <
> > > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > >> > > > > > > > >> > Specifically for ownership, I
> think
> > > the
> > > > > > plan
> > > > > > > is
> > > > > > > > > to
> > > > > > > > > > > add
> > > > > > > > > > > >> ACL
> > > > > > > > > > > >> > > (it
> > > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > > >> > > > > > > > >> > like you are describing ACL) via
> an
> > > > > > external
> > > > > > > > > system
> > > > > > > > > > > >> (Argus,
> > > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > > >> > > > > > > > >> > I remember KIP-11 described this,
> > > but I
> > > > > > can't
> > > > > > > > > find
> > > > > > > > > > > the
> > > > > > > > > > > >> KIP
> > > > > > > > > > > >> > > any
> > > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > >> > > > > > > > >> > Regardless, I think KIP-4 focuses
> > on
> > > > > > getting
> > > > > > > > > > > >> information
> > > > > > > > > > > >> > > that
> > > > > > > > > > > >> > > > > > > already
> > > > > > > > > > > >> > > > > > > > >> > exists from Kafka brokers, not on
> > > > adding
> > > > > > > > > information
> > > > > > > > > > > >> that
> > > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > > >> > > > > > > > >> > should exist but doesn't yet?
> > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM,
> > > > Guozhang
> > > > > > > Wang
> > > > > > > > <
> > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > >> > > > > > > > >> >> Just want to elaborate a bit
> more
> > on
> > > > the
> > > > > > > > > > > create-topic
> > > > > > > > > > > >> > > > metadata
> > > > > > > > > > > >> > > > > > and
> > > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > > >> > > > > > > > >> >> describe-topic based on config /
> > > > metadata
> > > > > > in
> > > > > > > > my
> > > > > > > > > > > >> previous
> > > > > > > > > > > >> > > > email
> > > > > > > > > > > >> > > > > as
> > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The main
> motivation
> > > is
> > > > to
> > > > > > > have
> > > > > > > > > some
> > > > > > > > > > > >> sort of
> > > > > > > > > > > >> > > > > topic
> > > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > > >> > > > > > > > >> >> mechanisms, which I think is
> quite
> > > > > > important
> > > > > > > > in
> > > > > > > > > a
> > > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > > >> > > > > /
> > > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > > >> > > > > > > > >> >> architecture: today anyone can
> > > create
> > > > > > topics
> > > > > > > > in
> > > > > > > > > a
> > > > > > > > > > > >> shared
> > > > > > > > > > > >> > > > Kafka
> > > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > > >> > > > > > > > >> >> there is no concept or
> "ownership"
> > > of
> > > > > > topics
> > > > > > > > > that
> > > > > > > > > > > are
> > > > > > > > > > > >> > > created
> > > > > > > > > > > >> > > > > by
> > > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > > >> > > > > > > > >> >> users. For example, at LinkedIn
> we
> > > > > > basically
> > > > > > > > > > > >> distinguish
> > > > > > > > > > > >> > > > topic
> > > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > > >> > > > > > > > >> >> some casual topic name prefix,
> > which
> > > > is a
> > > > > > > bit
> > > > > > > > > > > awkward
> > > > > > > > > > > >> and
> > > > > > > > > > > >> > > > does
> > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > > >> > > > > > > > >> >> we scale our customers. It would
> > be
> > > > great
> > > > > > to
> > > > > > > > use
> > > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > > >> > > > > > > such
> > > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > >> > > > > > > > >> >> Describe all topics that is
> > created
> > > > by me.
> > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > >> > > > > > > > >> >> Describe all topics whose
> > retention
> > > > time
> > > > > > is
> > > > > > > > > > > overriden
> > > > > > > > > > > >> to X.
> > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > >> > > > > > > > >> >> Describe all topics whose
> writable
> > > > group
> > > > > > > > include
> > > > > > > > > > > user
> > > > > > > > > > > >> Y
> > > > > > > > > > > >> > > (this
> > > > > > > > > > > >> > > > > is
> > > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > >> > > > > > > > >> >> One possible way to achieve this
> > is
> > > to
> > > > > > add a
> > > > > > > > > > > metadata
> > > > > > > > > > > >> file
> > > > > > > > > > > >> > > in
> > > > > > > > > > > >> > > > > the
> > > > > > > > > > > >> > > > > > > > >> >> create-topic request, whose
> value
> > > will
> > > > > > also
> > > > > > > be
> > > > > > > > > > > >> written ZK
> > > > > > > > > > > >> > > as
> > > > > > > > > > > >> > > > we
> > > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > > >> > > > > > > > >> >> topic; then describe-topics can
> > > > choose to
> > > > > > > > batch
> > > > > > > > > > > topics
> > > > > > > > > > > >> > > based
> > > > > > > > > > > >> > > > on
> > > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > > >> > > > > > > > >> >> regex, 2) config K-V matching,
> 3)
> > > > metadata
> > > > > > > > > regex,
> > > > > > > > > > > etc.
> > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM,
> > > > Guozhang
> > > > > > > Wang
> > > > > > > > <
> > > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > > >> > > > > > > > >> >>> Thanks for the updated wiki. A
> > few
> > > > > > comments
> > > > > > > > > below:
> > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > >> > > > > > > > >> >>> 1. Error description in
> > response: I
> > > > think
> > > > > > > if
> > > > > > > > > some
> > > > > > > > > > > >> > > errorCode
> > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > > >> > > > > > > > >> >>> several different error cases
> > then
> > > we
> > > > > > > should
> > > > > > > > > really
> > > > > > > > > > > >> change
> > > > > > > > > > > >> > > > it
> > > > > > > > > > > >> > > > > to
> > > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > > >> > > > > > > > >> >>> codes. In general the errorCode
> > > > itself
> > > > > > > would
> > > > > > > > be
> > > > > > > > > > > >> precise
> > > > > > > > > > > >> > > and
> > > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > > >> > > > > > > > >> >>> describing the server side
> > errors.
> > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic request: it
> > would
> > > > be
> > > > > > > great
> > > > > > > > > to go
> > > > > > > > > > > >> beyond
> > > > > > > > > > > >> > > > just
> > > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > > >> > > > > > > > >> >>> topic name regex for this
> > request.
> > > > For
> > > > > > > > > example, a
> > > > > > > > > > > >> very
> > > > > > > > > > > >> > > > common
> > > > > > > > > > > >> > > > > > use
> > > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > > >> > > > > > > > >> >>> the topic command is to list
> all
> > > > topics
> > > > > > > whose
> > > > > > > > > > > config
> > > > > > > > > > > >> A's
> > > > > > > > > > > >> > > > value
> > > > > > > > > > > >> > > > > > is
> > > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > > >> > > > > > > > >> >>> topic name regex then we have
> to
> > > > first
> > > > > > > > retrieve
> > > > > > > > > > > >> __all__
> > > > > > > > > > > >> > > > > topics's
> > > > > > > > > > > >> > > > > > > > >> >>> description info and then
> filter
> > at
> > > > the
> > > > > > > > client
> > > > > > > > > end,
> > > > > > > > > > > >> which
> > > > > > > > > > > >> > > > will
> > > > > > > > > > > >> > > > > > be
> > > > > > > > > > > >> > > > > > > a
> > > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in create topic:
> > > this
> > > > is
> > > > > > > > > related to
> > > > > > > > > > > >> the
> > > > > > > > > > > >> > > > > previous
> > > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > > >> > > > > > > > >> >>> maybe we can add another
> metadata
> > > > K-V or
> > > > > > > > just a
> > > > > > > > > > > >> metadata
> > > > > > > > > > > >> > > > > string
> > > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > > >> > > > > > > > >> >>> with config K-V in create topic
> > > like
> > > > we
> > > > > > did
> > > > > > > > for
> > > > > > > > > > > >> offset
> > > > > > > > > > > >> > > > commit
> > > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > > >> > > > > > > > >> >>> field can be quite useful in
> > > storing
> > > > > > > > > information
> > > > > > > > > > > like
> > > > > > > > > > > >> > > > "owner"
> > > > > > > > > > > >> > > > > of
> > > > > > > > > > > >> > > > > > > the
> > > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > > >> > > > > > > > >> >>> who issue the create command,
> > etc,
> > > > which
> > > > > > is
> > > > > > > > > quite
> > > > > > > > > > > >> > > important
> > > > > > > > > > > >> > > > > for
> > > > > > > > > > > >> > > > > > a
> > > > > > > > > > > >> > > > > > > > >> >>> multi-tenant setting. Then in
> the
> > > > > > describe
> > > > > > > > > topic
> > > > > > > > > > > >> request
> > > > > > > > > > > >> > > we
> > > > > > > > > > > >> > > > > can
> > > > > > > > > > > >> > > > > > > also
> > > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > > >> > > > > > > > >> >>> on regex of the metadata field.
> > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > >> > > > > > > > >> >>> 4. Today all the admin
> operations
> > > are
> > > > > > async
> > > > > > > > in
> > > > > > > > > the
> > > > > > > > > > > >> sense
> > > > > > > > > > > >> > > > that
> > > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > > >> > > > > > > > >> >>> return once it is written in
> ZK,
> > > and
> > > > that
> > > > > > > is
> > > > > > > > > why we
> > > > > > > > > > > >> need
> > > > > > > > > > > >> > > > extra
> > > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > > >> > > > > > > > >> >>> like
> > > testUtil.waitForTopicCreated() /
> > > > > > > verify
> > > > > > > > > > > >> partition
> > > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > > >> > > > > > > > >> >>> request, etc. With admin
> requests
> > > we
> > > > > > could
> > > > > > > > add
> > > > > > > > > a
> > > > > > > > > > > >> flag to
> > > > > > > > > > > >> > > > > enable
> > > > > > > > > > > >> > > > > > /
> > > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > > >> > > > > > > > >> >>> synchronous requests; when it
> is
> > > > turned
> > > > > > on,
> > > > > > > > the
> > > > > > > > > > > >> response
> > > > > > > > > > > >> > > > will
> > > > > > > > > > > >> > > > > > not
> > > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > > >> > > > > > > > >> >>> until the request has been
> > > > completed. And
> > > > > > > for
> > > > > > > > > async
> > > > > > > > > > > >> > > requests
> > > > > > > > > > > >> > > > > we
> > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > > >> > > > > > > > >> >>> "token" field in the response,
> > and
> > > > then
> > > > > > > only
> > > > > > > > > need a
> > > > > > > > > > > >> > > general
> > > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > > >> > > > > > > > >> >>> verification request" with the
> > > given
> > > > > > token
> > > > > > > to
> > > > > > > > > check
> > > > > > > > > > > >> if the
> > > > > > > > > > > >> > > > > async
> > > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for extending Metadata
> > > request
> > > > to
> > > > > > > > include
> > > > > > > > > > > >> > > controller /
> > > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > > >> > > > > > > > >> >>> information, and then we can
> > remove
> > > > the
> > > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23
> AM,
> > > Joel
> > > > > > > Koshy <
> > > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > > >> > > > > > > > >> >>>> Thanks for sending that out
> Joe
> > -
> > > I
> > > > > > don't
> > > > > > > > > think I
> > > > > > > > > > > >> will be
> > > > > > > > > > > >> > > > > able
> > > > > > > > > > > >> > > > > > to
> > > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > > >> > > > > > > > >> >>>> it today, so if notes can be
> > sent
> > > > out
> > > > > > > > > afterward
> > > > > > > > > > > that
> > > > > > > > > > > >> > > would
> > > > > > > > > > > >> > > > be
> > > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at
> > 09:16:13AM
> > > > > > -0800,
> > > > > > > > Gwen
> > > > > > > > > > > >> Shapira
> > > > > > > > > > > >> > > > wrote:
> > > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for sending this out
> > Joe.
> > > > > > Looking
> > > > > > > > > forward
> > > > > > > > > > > >> to
> > > > > > > > > > > >> > > > > chatting
> > > > > > > > > > > >> > > > > > > with
> > > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46
> > AM,
> > > > Joe
> > > > > > > Stein
> > > > > > > > <
> > > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just sent out a
> > google
> > > > > > hangout
> > > > > > > > > invite
> > > > > > > > > > > to
> > > > > > > > > > > >> all
> > > > > > > > > > > >> > > > pmc,
> > > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I found working
> on
> > a
> > > > KIP.
> > > > > > If
> > > > > > > I
> > > > > > > > > missed
> > > > > > > > > > > >> anyone
> > > > > > > > > > > >> > > > in
> > > > > > > > > > > >> > > > > > the
> > > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > > >> > > > > > > > >> >>>> > > let me know and can update
> > it,
> > > > np.
> > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > > We should do this every
> > > Tuesday
> > > > @
> > > > > > 2pm
> > > > > > > > > Eastern
> > > > > > > > > > > >> Time.
> > > > > > > > > > > >> > > > Maybe
> > > > > > > > > > > >> > > > > > we
> > > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > > >> > > > > > > > >> >>>> > > help to make a google
> > account
> > > > so we
> > > > > > > can
> > > > > > > > > manage
> > > > > > > > > > > >> > > better?
> > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > > >> > > > > > > >
> > > > > > > > > > > >> > > > > > >
> > > > > > > > > > > >> > > > > >
> > > > > > > > > > > >> > > > >
> > > > > > > > > > > >> > > >
> > > > > > > > > > > >> > >
> > > > > > > > > > > >>
> > > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > > >> > > > > > > > >> >>>> > > in progress and related
> JIRA
> > > > that
> > > > > > are
> > > > > > > > > > > >> interdependent
> > > > > > > > > > > >> > > > and
> > > > > > > > > > > >> > > > > > > common
> > > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at
> 2:59
> > > > PM, Jay
> > > > > > > > > Kreps <
> > > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on Google
> > hangouts
> > > > that
> > > > > > > will
> > > > > > > > > also
> > > > > > > > > > > >> record
> > > > > > > > > > > >> > > > and
> > > > > > > > > > > >> > > > > > make
> > > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at
> > 11:49
> > > > AM,
> > > > > > > Jeff
> > > > > > > > > > > Holoman
> > > > > > > > > > > >> <
> > > > > > > > > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're happy to send
> out a
> > > > Webex
> > > > > > for
> > > > > > > > > this
> > > > > > > > > > > >> purpose.
> > > > > > > > > > > >> > > We
> > > > > > > > > > > >> > > > > > could
> > > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions if there is
> > > > interest and
> > > > > > > > > publish
> > > > > > > > > > > >> them
> > > > > > > > > > > >> > > out.
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at
> > > > 11:28 AM,
> > > > > > > Jay
> > > > > > > > > > > Kreps <
> > > > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try to get the
> > > > technical
> > > > > > > > > hang-ups
> > > > > > > > > > > >> sorted
> > > > > > > > > > > >> > > > out,
> > > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there is some benefit
> > to
> > > > live
> > > > > > > > > discussion
> > > > > > > > > > > vs
> > > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > > >> > > > > > > am
> > > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post instructions
> > and
> > > > give
> > > > > > > > > ourselves a
> > > > > > > > > > > >> few
> > > > > > > > > > > >> > > > > attempts
> > > > > > > > > > > >> > > > > > > we
> > > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that time
> > > would
> > > > work
> > > > > > > for
> > > > > > > > > > > >> me...any
> > > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015
> at
> > > > 8:18
> > > > > > AM,
> > > > > > > > Joe
> > > > > > > > > > > Stein
> > > > > > > > > > > >> <
> > > > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly would be
> great
> > > > maybe
> > > > > > > like
> > > > > > > > > every
> > > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't mind google
> > > > hangout
> > > > > > but
> > > > > > > > > there
> > > > > > > > > > > is
> > > > > > > > > > > >> > > always
> > > > > > > > > > > >> > > > > some
> > > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we know the apache
> > irc
> > > > > > channel
> > > > > > > > > works.
> > > > > > > > > > > We
> > > > > > > > > > > >> can
> > > > > > > > > > > >> > > > start
> > > > > > > > > > > >> > > > > > > there
> > > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull
> > > > transcripts
> > > > > > > too
> > > > > > > > > and
> > > > > > > > > > > >> > > associate
> > > > > > > > > > > >> > > > to
> > > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it helpful for
> > things.
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24,
> 2015
> > at
> > > > 11:10
> > > > > > > AM,
> > > > > > > > > Jay
> > > > > > > > > > > >> Kreps <
> > > > > > > > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked about
> > > > doing a
> > > > > > > > Google
> > > > > > > > > > > >> Hangout to
> > > > > > > > > > > >> > > > chat
> > > > > > > > > > > >> > > > > > > about
> > > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > generalizing
> that a
> > > > little
> > > > > > > > > > > further...I
> > > > > > > > > > > >> > > > actually
> > > > > > > > > > > >> > > > > > > think
> > > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > everyone
> spending a
> > > > > > > reasonable
> > > > > > > > > chunk
> > > > > > > > > > > of
> > > > > > > > > > > >> > > their
> > > > > > > > > > > >> > > > > week
> > > > > > > > > > > >> > > > > > > on
> > > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync up once a
> > week.
> > > I
> > > > > > think
> > > > > > > we
> > > > > > > > > could
> > > > > > > > > > > >> use
> > > > > > > > > > > >> > > time
> > > > > > > > > > > >> > > > > to
> > > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure
> we
> > > > are on
> > > > > > > top
> > > > > > > > of
> > > > > > > > > > > code
> > > > > > > > > > > >> > > > reviews,
> > > > > > > > > > > >> > > > > > talk
> > > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We can make it
> > > publicly
> > > > > > > > > available so
> > > > > > > > > > > >> that
> > > > > > > > > > > >> > > any
> > > > > > > > > > > >> > > > > one
> > > > > > > > > > > >> > > > > > > can
> > > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any interest in
> > doing
> > > > this?
> > > > > > > If
> > > > > > > > so
> > > > > > > > > > > I'll
> > > > > > > > > > > >> try
> > > > > > > > > > > >> > > to
> > > > > > > > > > > >> > > > > set
> > > > > > > > > > > >> > > > > > it
> > > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24,
> > 2015
> > > at
> > > > > > 3:57
> > > > > > > > AM,
> > > > > > > > > > > Andrii
> > > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > > >> > > > > > <
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > wrote:
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > I've updated
> KIP
> > > > page,
> > > > > > > fixed
> > > > > > > > /
> > > > > > > > > > > >> aligned
> > > > > > > > > > > >> > > > > document
> > > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> >
> ...
>
> [Message clipped]

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
Andrii,

I think you are right. It seems that only ReassignPartitions needs a
separate verification request.

Thanks,

Jun

On Thu, Mar 19, 2015 at 9:22 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Guys,
> I like this idea too. Let's stick with that. I'll update KIP accordingly.
>
> I was also thinking we can avoid adding dedicated status check
> requests for topic commands. - We have everything in DescribeTopic
> for that! E.g.:
> User issued CreateTopic - to check the status client sends DescribeTopic
> and checks whether is something returned for that topic. The same for
> alteration, deletion.
> Btw, PreferredReplica status can be also checked with DescribeTopicRequest
> (head of assigned replicas list == leader).
> For ReassignPartitions as discussed we'll need to have a separate Verify...
> request.
>
> Thanks,
> Andrii Biletskyi
>
>
> On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
> > +1 on broker writing to ZK for async handling. I was thinking that in the
> > end state the admin requests would be eventually sent to controller
> either
> > through re-routing or clients discovering them, instead of letting
> > controller listen on ZK admin path. But thinking about it a second time,
> I
> > think it is actually simpler to let controller manage
> > incoming queued-up admin requests through ZK.
> >
> > Guozhang
> >
> >
> > On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <jj...@gmail.com> wrote:
> >
> > > +1 as well. I think it helps to keep the rerouting approach orthogonal
> > > to this KIP.
> > >
> > > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps wrote:
> > > > I'm +1 on Jun's suggestion as long as it can work for all the
> requests.
> > > >
> > > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Andrii,
> > > > >
> > > > > I think we agreed on the following.
> > > > >
> > > > > (a) Admin requests can be sent to and handled by any broker.
> > > > > (b) Admin requests are processed asynchronously, at least for now.
> > > That is,
> > > > > when the client gets a response, it just means that the request is
> > > > > initiated, but not necessarily completed. Then, it's up to the
> client
> > > to
> > > > > issue another request to check the status for completion.
> > > > >
> > > > > To support (a), we were thinking of doing request forwarding to the
> > > > > controller (utilizing KAFKA-1912). I am making an alternative
> > proposal.
> > > > > Basically, the broker can just write to ZooKeeper to inform the
> > > controller
> > > > > about the request. For example, to handle partitionReassignment,
> the
> > > broker
> > > > > will just write the requested partitions to
> > /admin/reassign_partitions
> > > > > (like what AdminUtils currently does) and then send a response to
> the
> > > > > client. This shouldn't take long and the implementation will be
> > simpler
> > > > > than forwarding the requests to the controller through RPC.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Jun,
> > > > > >
> > > > > > I might be wrong but didn't we agree we will let any broker from
> > the
> > > > > > cluster handle *long-running* admin requests (at this time
> > > > > preferredReplica
> > > > > > and
> > > > > > reassignPartitions), via zk admin path. Thus CreateTopics etc
> > should
> > > be
> > > > > > sent
> > > > > > only to the controller.
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > > >
> > > > > > > Joel, Andril,
> > > > > > >
> > > > > > > I think we agreed that those admin requests can be issued to
> any
> > > > > broker.
> > > > > > > Because of that, there doesn't seem to be a strong need to know
> > the
> > > > > > > controller. So, perhaps we can proceed by not making any change
> > to
> > > the
> > > > > > > format of TMR right now. When we start using create topic
> request
> > > in
> > > > > the
> > > > > > > producer, we will need a new version of TMR that doesn't
> trigger
> > > auto
> > > > > > topic
> > > > > > > creation. But that can be done later.
> > > > > > >
> > > > > > > As a first cut implementation, I think the broker can just
> write
> > > to ZK
> > > > > > > directly for
> > > > > > >
> > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > > requests, instead of forwarding them to the controller. This
> will
> > > > > > simplify
> > > > > > > the implementation on the broker side.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> > jjkoshy.w@gmail.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > > (Thanks Andrii for the summary)
> > > > > > > >
> > > > > > > > For (1) yes we will circle back on that shortly after syncing
> > up
> > > in
> > > > > > > > person. I think it is close to getting committed although
> > > development
> > > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > > >
> > > > > > > > There is one more item we covered at the hangout. i.e.,
> whether
> > > we
> > > > > > > > want to add the coordinator to the topic metadata response or
> > > provide
> > > > > > > > a clearer ClusterMetadataRequest.
> > > > > > > >
> > > > > > > > There are two reasons I think we should try and avoid adding
> > the
> > > > > > > > field:
> > > > > > > > - It is irrelevant to topic metadata
> > > > > > > > - If we finally do request rerouting in Kafka then the field
> > > would
> > > > > add
> > > > > > > >   little to no value. (It still helps to have a separate
> > > > > > > >   ClusterMetadataRequest to query for cluster-wide
> information
> > > such
> > > > > as
> > > > > > > >   'which broker is the controller?' as Joe mentioned.)
> > > > > > > >
> > > > > > > > I think it would be cleaner to have an explicit
> > > > > ClusterMetadataRequest
> > > > > > > > that you can send to any broker in order to obtain the
> > controller
> > > > > (and
> > > > > > > > in the future possibly other cluster-wide information). I
> think
> > > the
> > > > > > > > main argument against doing this and instead adding it to the
> > > topic
> > > > > > > > metadata response was convenience - i.e., you don't have to
> > > discover
> > > > > > > > the controller in advance. However, I don't see much actual
> > > > > > > > benefit/convenience in this and in fact think it is a
> > non-issue.
> > > Let
> > > > > > > > me know if I'm overlooking something here.
> > > > > > > >
> > > > > > > > As an example, say we need to initiate partition reassignment
> > by
> > > > > > > > issuing the new ReassignPartitionsRequest to the controller
> > > (assume
> > > > > we
> > > > > > > > already have the desired manual partition assignment).  If we
> > > are to
> > > > > > > > augment topic metadata response then the flow be something
> like
> > > this
> > > > > :
> > > > > > > >
> > > > > > > > - Issue topic metadata request to any broker (and discover
> the
> > > > > > > >   controller
> > > > > > > > - Connect to controller if required (i.e., if the broker
> above
> > !=
> > > > > > > >   controller)
> > > > > > > > - Issue the partition reassignment request to the controller.
> > > > > > > >
> > > > > > > > With an explicit cluster metadata request it would be:
> > > > > > > > - Issue cluster metadata request to any broker
> > > > > > > > - Connect to controller if required (i.e., if the broker
> above
> > !=
> > > > > > > >   controller)
> > > > > > > > - Issue the partition reassignment request
> > > > > > > >
> > > > > > > > So it seems to add little practical value and bloats topic
> > > metadata
> > > > > > > > response with an irrelevant detail.
> > > > > > > >
> > > > > > > > The other angle to this is the following - is it a matter of
> > > naming?
> > > > > > > > Should we just rename topic metadata request/response to just
> > > > > > > > MetadataRequest/Response and add cluster metadata to it? By
> > that
> > > same
> > > > > > > > token should we also allow querying for the consumer
> > coordinator
> > > (and
> > > > > > > > in future transaction coordinator) as well? This leads to a
> > > bloated
> > > > > > > > request which isn't very appealing and altogether confusing.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Joel
> > > > > > > >
> > > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > > > > > > > > Andri,
> > > > > > > > >
> > > > > > > > > Thanks for the summary.
> > > > > > > > >
> > > > > > > > > 1. I just realized that in order to start working on
> > > KAFKA-1927, we
> > > > > > > will
> > > > > > > > > need to merge the changes to OffsetCommitRequest (from
> 0.8.2)
> > > to
> > > > > > trunk.
> > > > > > > > > This is planned to be done as part of KAFKA-1634. So, we
> will
> > > need
> > > > > > > > Guozhang
> > > > > > > > > and Joel's help to wrap this up.
> > > > > > > > >
> > > > > > > > > 2. Thinking about this a bit more, if the semantic of those
> > > "write"
> > > > > > > > > requests is async (i.e., after the client gets a response,
> it
> > > just
> > > > > > > means
> > > > > > > > > that the operation is initiated, but not necessarily
> > > completed), we
> > > > > > > don't
> > > > > > > > > really need to forward the requests to the controller.
> > > Instead, the
> > > > > > > > > receiving broker can just write the operation to ZK as the
> > > admin
> > > > > > > command
> > > > > > > > > line tool previously does. This will simplify the
> > > implementation.
> > > > > > > > >
> > > > > > > > > 8. There is another implementation detail for describe
> topic.
> > > > > > Ideally,
> > > > > > > we
> > > > > > > > > want to read the topic config from the broker cache,
> instead
> > of
> > > > > > > > ZooKeeper.
> > > > > > > > > Currently, every broker reads the topic-level config for
> all
> > > > > topics.
> > > > > > > > > However, it ignores those for topics not hosted on itself.
> > So,
> > > we
> > > > > may
> > > > > > > > need
> > > > > > > > > to change TopicConfigManager a bit so that it caches the
> > > configs
> > > > > for
> > > > > > > all
> > > > > > > > > topics.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > >
> > > > > > > > > Jun
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >
> > > > > > > > > > Guys,
> > > > > > > > > >
> > > > > > > > > > Thanks for a great discussion!
> > > > > > > > > > Here are the actions points:
> > > > > > > > > >
> > > > > > > > > > 1. Q: Get rid of all scala requests objects, use java
> > > protocol
> > > > > > > > definitions.
> > > > > > > > > >     A: Gwen kindly took that (KAFKA-1927). It's important
> > to
> > > > > speed
> > > > > > up
> > > > > > > > > > review procedure
> > > > > > > > > >          there since this ticket blocks other important
> > > changes.
> > > > > > > > > >
> > > > > > > > > > 2. Q: Generic re-reroute facility vs client maintaining
> > > cluster
> > > > > > > state.
> > > > > > > > > >     A: Jay has added pseudo code to KAFKA-1912 - need to
> > > consider
> > > > > > > > whether
> > > > > > > > > > this will be
> > > > > > > > > >         easy to implement as a server-side feature
> > (comments
> > > are
> > > > > > > > > > welcomed!).
> > > > > > > > > >
> > > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > > >     A: This might be useful for clients, add this to
> > > > > > > > TopicMetadataResponse
> > > > > > > > > > (already in KIP).
> > > > > > > > > >
> > > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > > >     A: I will add proposed by Jun solution (using
> clientId
> > > for
> > > > > > that)
> > > > > > > > to the
> > > > > > > > > > KIP.
> > > > > > > > > >
> > > > > > > > > > 5. Q: Bumping new versions of TMR vs grabbing all
> protocol
> > > > > changes
> > > > > > in
> > > > > > > > one
> > > > > > > > > > version.
> > > > > > > > > >     A: It was decided to try to gather all changes to
> > > protocol
> > > > > > > (before
> > > > > > > > > > release).
> > > > > > > > > >         In case of TMR it worth checking: KAFKA-2020 and
> > > KIP-13
> > > > > > > > (quotas)
> > > > > > > > > >
> > > > > > > > > > 6. Q: JSON lib is needed to deserialize user's input in
> CLI
> > > tool.
> > > > > > > > > >     A: Use jackson for that, /tools project is a separate
> > > jar so
> > > > > > > > shouldn't
> > > > > > > > > > be a big deal.
> > > > > > > > > >
> > > > > > > > > > 7.  Q: VerifyReassingPartitions vs generic status check
> > > command.
> > > > > > > > > >      A: For long-running requests like reassign
> partitions
> > > > > > *progress*
> > > > > > > > check
> > > > > > > > > > request is useful,
> > > > > > > > > >          it makes sense to introduce it.
> > > > > > > > > >
> > > > > > > > > >  Please add, correct me if I missed something.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Andrii Biletskyi
> > > > > > > > > >
> > > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > >
> > > > > > > > > > > Joel,
> > > > > > > > > > >
> > > > > > > > > > > You are right, I removed ClusterMetadata because we
> have
> > > > > > partially
> > > > > > > > > > > what we need in TopicMetadata. Also, as Jay pointed out
> > > > > earlier,
> > > > > > we
> > > > > > > > > > > would like to have "orthogonal" API, but at the same
> time
> > > we
> > > > > need
> > > > > > > > > > > to be backward compatible.
> > > > > > > > > > >
> > > > > > > > > > > But I like your idea and even have some other arguments
> > for
> > > > > this
> > > > > > > > option:
> > > > > > > > > > > There is also DescribeTopicRequest which was proposed
> in
> > > this
> > > > > > KIP,
> > > > > > > > > > > it returns topic configs, partitions, replication
> factor
> > > plus
> > > > > > > > partition
> > > > > > > > > > > ISR, ASR,
> > > > > > > > > > > leader replica. The later part is really already there
> in
> > > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > > So again we'll have to add stuff to TMR, not to
> duplicate
> > > some
> > > > > > info
> > > > > > > > in
> > > > > > > > > > > newly added requests. However, this way we'll end up
> with
> > > > > > "monster"
> > > > > > > > > > > request which returns cluster metadata, topic
> replication
> > > and
> > > > > > > config
> > > > > > > > info
> > > > > > > > > > > plus partition replication data. Seems logical to split
> > > TMR to
> > > > > > > > > > > - ClusterMetadata (brokers + controller, maybe smth
> else)
> > > > > > > > > > > - TopicMetadata (topic info + partition details)
> > > > > > > > > > > But since current TMR is involved in lots of places
> > > (including
> > > > > > > > network
> > > > > > > > > > > client,
> > > > > > > > > > > as I understand) this might be very serious change and
> it
> > > > > > probably
> > > > > > > > makes
> > > > > > > > > > > sense to stick with current approach.
> > > > > > > > > > >
> > > > > > > > > > > Thanks,
> > > > > > > > > > > Andrii Biletskyi
> > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <
> > > > > jjkoshy.w@gmail.com
> > > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > >> I may be missing some context but hopefully this will
> > > also be
> > > > > > > > covered
> > > > > > > > > > >> today: I thought the earlier proposal where there was
> an
> > > > > > explicit
> > > > > > > > > > >> ClusterMetadata request was clearer and explicit.
> During
> > > the
> > > > > > > course
> > > > > > > > of
> > > > > > > > > > >> this thread I think the conclusion was that the main
> > need
> > > was
> > > > > > for
> > > > > > > > > > >> controller information and that can be rolled into the
> > > topic
> > > > > > > > metadata
> > > > > > > > > > >> response but that seems a bit irrelevant to topic
> > > metadata.
> > > > > > FWIW I
> > > > > > > > > > >> think the full broker-list is also irrelevant to topic
> > > > > metadata,
> > > > > > > but
> > > > > > > > > > >> it is already there and in use. I think there is still
> > > room
> > > > > for
> > > > > > an
> > > > > > > > > > >> explicit ClusterMetadata request since there may be
> > other
> > > > > > > > > > >> cluster-level information that we may want to add over
> > > time
> > > > > (and
> > > > > > > > that
> > > > > > > > > > >> have nothing to do with topic metadata).
> > > > > > > > > > >>
> > > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii
> > Biletskyi
> > > > > > wrote:
> > > > > > > > > > >> > Jun,
> > > > > > > > > > >> >
> > > > > > > > > > >> > 101. Okay, if you say that such use case is
> > important. I
> > > > > also
> > > > > > > > think
> > > > > > > > > > >> > using clientId for these purposes is fine - if we
> > > already
> > > > > have
> > > > > > > > this
> > > > > > > > > > >> field
> > > > > > > > > > >> > as part of all Wire protocol messages, why not use
> > that.
> > > > > > > > > > >> > I will update KIP-4 page if nobody has other ideas
> > > (which
> > > > > may
> > > > > > > > come up
> > > > > > > > > > >> > during the call today).
> > > > > > > > > > >> >
> > > > > > > > > > >> > 102.1 Agree, I'll update the KIP accordingly. I
> think
> > > we can
> > > > > > add
> > > > > > > > new,
> > > > > > > > > > >> > fine-grained error codes if some error code received
> > in
> > > > > > specific
> > > > > > > > case
> > > > > > > > > > >> > won't give enough context to return a descriptive
> > error
> > > > > > message
> > > > > > > > for
> > > > > > > > > > >> user.
> > > > > > > > > > >> >
> > > > > > > > > > >> > Look forward to discussing all outstanding issues in
> > > detail
> > > > > > > today
> > > > > > > > > > during
> > > > > > > > > > >> > the call.
> > > > > > > > > > >> >
> > > > > > > > > > >> > Thanks,
> > > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > > > > > > > > > >> >
> > > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <
> > > jun@confluent.io
> > > > > >
> > > > > > > > wrote:
> > > > > > > > > > >> >
> > > > > > > > > > >> > > 101. There may be a use case where you only want
> the
> > > > > topics
> > > > > > to
> > > > > > > > be
> > > > > > > > > > >> created
> > > > > > > > > > >> > > manually by admins. Currently, you can do that by
> > > > > disabling
> > > > > > > auto
> > > > > > > > > > topic
> > > > > > > > > > >> > > creation and issue topic creation from the
> > > TopicCommand.
> > > > > If
> > > > > > we
> > > > > > > > > > >> disable auto
> > > > > > > > > > >> > > topic creation completely on the broker and don't
> > > have a
> > > > > way
> > > > > > > to
> > > > > > > > > > >> distinguish
> > > > > > > > > > >> > > between topic creation requests from the regular
> > > clients
> > > > > and
> > > > > > > the
> > > > > > > > > > >> admin, we
> > > > > > > > > > >> > > can't support manual topic creation any more. I
> was
> > > > > thinking
> > > > > > > > that
> > > > > > > > > > >> another
> > > > > > > > > > >> > > way of distinguishing the clients making the topic
> > > > > creation
> > > > > > > > requests
> > > > > > > > > > >> is
> > > > > > > > > > >> > > using clientId. For example, the admin tool can
> set
> > > it to
> > > > > > > > something
> > > > > > > > > > >> like
> > > > > > > > > > >> > > admin and the broker can treat that clientId
> > > specially.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Also, there is a related discussion in KAFKA-2020.
> > > > > > Currently,
> > > > > > > > we do
> > > > > > > > > > >> the
> > > > > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > 1. If leader is not available, we set the
> partition
> > > level
> > > > > > > error
> > > > > > > > code
> > > > > > > > > > >> to
> > > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > > >> > > 2. If a non-leader replica is not available, we
> take
> > > that
> > > > > > > > replica
> > > > > > > > > > out
> > > > > > > > > > >> of
> > > > > > > > > > >> > > the assigned replica list and isr in the response.
> > As
> > > an
> > > > > > > > indication
> > > > > > > > > > >> for
> > > > > > > > > > >> > > doing that, we set the partition level error code
> to
> > > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > This has a few problems. First,
> ReplicaNotAvailable
> > > > > probably
> > > > > > > > > > >> shouldn't be
> > > > > > > > > > >> > > an error, at least for the normal
> producer/consumer
> > > > > clients
> > > > > > > that
> > > > > > > > > > just
> > > > > > > > > > >> want
> > > > > > > > > > >> > > to find out the leader. Second, it can happen that
> > > both
> > > > > the
> > > > > > > > leader
> > > > > > > > > > and
> > > > > > > > > > >> > > another replica are not available at the same
> time.
> > > There
> > > > > is
> > > > > > > no
> > > > > > > > > > error
> > > > > > > > > > >> code
> > > > > > > > > > >> > > to indicate both. Third, even if a replica is not
> > > > > available,
> > > > > > > > it's
> > > > > > > > > > >> still
> > > > > > > > > > >> > > useful to return its replica id since some clients
> > > (e.g.
> > > > > > admin
> > > > > > > > tool)
> > > > > > > > > > >> may
> > > > > > > > > > >> > > still make use of it.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > One way to address this issue is to always return
> > the
> > > > > > replica
> > > > > > > > id for
> > > > > > > > > > >> > > leader, assigned replicas, and isr regardless of
> > > whether
> > > > > the
> > > > > > > > > > >> corresponding
> > > > > > > > > > >> > > broker is live or not. Since we also return the
> list
> > > of
> > > > > live
> > > > > > > > > > brokers,
> > > > > > > > > > >> the
> > > > > > > > > > >> > > client can figure out whether a leader or a
> replica
> > is
> > > > > live
> > > > > > or
> > > > > > > > not
> > > > > > > > > > >> and act
> > > > > > > > > > >> > > accordingly. This way, we don't need to set the
> > > partition
> > > > > > > level
> > > > > > > > > > error
> > > > > > > > > > >> code
> > > > > > > > > > >> > > when the leader or a replica is not available.
> This
> > > > > doesn't
> > > > > > > > change
> > > > > > > > > > >> the wire
> > > > > > > > > > >> > > protocol, but does change the semantics. Since we
> > are
> > > > > > evolving
> > > > > > > > the
> > > > > > > > > > >> protocol
> > > > > > > > > > >> > > of TopicMetadataRequest here, we can potentially
> > > piggyback
> > > > > > the
> > > > > > > > > > change.
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > 102.1 For those types of errors due to invalid
> > input,
> > > > > > > shouldn't
> > > > > > > > we
> > > > > > > > > > >> just
> > > > > > > > > > >> > > guard it at parameter validation time and throw
> > > > > > > > > > >> InvalidArgumentException
> > > > > > > > > > >> > > without even sending the request to the broker?
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Thanks,
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > Jun
> > > > > > > > > > >> > >
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii
> Biletskyi <
> > > > > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > >> > >
> > > > > > > > > > >> > > > Jun,
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > 101. If I understand you correctly, you are
> saying
> > > > > future
> > > > > > > > producer
> > > > > > > > > > >> > > versions
> > > > > > > > > > >> > > > (which
> > > > > > > > > > >> > > > will be ported to TMR_V1) won't be able to
> > > automatically
> > > > > > > > create
> > > > > > > > > > >> topic (if
> > > > > > > > > > >> > > > we
> > > > > > > > > > >> > > > unconditionally remove topic creation from
> there).
> > > But
> > > > > we
> > > > > > > > need to
> > > > > > > > > > >> this
> > > > > > > > > > >> > > > preserve logic.
> > > > > > > > > > >> > > > Ok, about your proposal: I'm not a big fan too,
> > > when it
> > > > > > > comes
> > > > > > > > to
> > > > > > > > > > >> > > > differentiating
> > > > > > > > > > >> > > > clients directly in protocol schema. And also
> I'm
> > > not
> > > > > > sure I
> > > > > > > > > > >> understand
> > > > > > > > > > >> > > at
> > > > > > > > > > >> > > > all why
> > > > > > > > > > >> > > > auto.create.topics.enable is a server side
> > > > > configuration.
> > > > > > > Can
> > > > > > > > we
> > > > > > > > > > >> > > deprecate
> > > > > > > > > > >> > > > this setting
> > > > > > > > > > >> > > > in future versions, add this setting to producer
> > and
> > > > > based
> > > > > > > on
> > > > > > > > that
> > > > > > > > > > >> upon
> > > > > > > > > > >> > > > receiving
> > > > > > > > > > >> > > > UnknownTopic create topic explicitly by a
> separate
> > > > > > producer
> > > > > > > > call
> > > > > > > > > > via
> > > > > > > > > > >> > > > adminClient?
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > 102.1. Hm, yes. It's because we want to support
> > > batching
> > > > > > and
> > > > > > > > at
> > > > > > > > > > the
> > > > > > > > > > >> same
> > > > > > > > > > >> > > > time we
> > > > > > > > > > >> > > > want to give descriptive error messages for
> > clients.
> > > > > Since
> > > > > > > > > > >> AdminClient
> > > > > > > > > > >> > > > holds the context
> > > > > > > > > > >> > > > to construct such messages (e.g. AdminClient
> layer
> > > can
> > > > > > know
> > > > > > > > that
> > > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > > >> > > > means two cases: either invalid number - e.g.
> -1;
> > or
> > > > > > > > > > >> replication-factor
> > > > > > > > > > >> > > was
> > > > > > > > > > >> > > > provided while
> > > > > > > > > > >> > > > partitions argument wasn't) - I wrapped
> responses
> > in
> > > > > > > > Exceptions.
> > > > > > > > > > >> But I'm
> > > > > > > > > > >> > > > open to any
> > > > > > > > > > >> > > > other ideas, this was just initial version.
> > > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change that to
> probably
> > > some
> > > > > > other
> > > > > > > > dto.
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > Thanks,
> > > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <
> > > > > > jun@confluent.io>
> > > > > > > > > > wrote:
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > > > > Andrii,
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > 101. That's what I was thinking too, but it
> may
> > > not be
> > > > > > > that
> > > > > > > > > > >> simple. In
> > > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > > >> > > > > we can let it not trigger auto topic creation.
> > > Then,
> > > > > in
> > > > > > > the
> > > > > > > > > > >> producer
> > > > > > > > > > >> > > > side,
> > > > > > > > > > >> > > > > if it gets an UnknownTopicException, it can
> > > explicitly
> > > > > > > > issue a
> > > > > > > > > > >> > > > > createTopicRequest for auto topic creation. On
> > the
> > > > > > > consumer
> > > > > > > > > > side,
> > > > > > > > > > >> it
> > > > > > > > > > >> > > will
> > > > > > > > > > >> > > > > never issue createTopicRequest. This works
> when
> > > auto
> > > > > > topic
> > > > > > > > > > >> creation is
> > > > > > > > > > >> > > > > enabled on the broker side. However, I am not
> > > sure how
> > > > > > > > things
> > > > > > > > > > >> will work
> > > > > > > > > > >> > > > > when auto topic creation is disabled on the
> > broker
> > > > > side.
> > > > > > > In
> > > > > > > > this
> > > > > > > > > > >> case,
> > > > > > > > > > >> > > we
> > > > > > > > > > >> > > > > want to have a way to manually create a topic,
> > > > > > potentially
> > > > > > > > > > through
> > > > > > > > > > >> > > admin
> > > > > > > > > > >> > > > > commands. However, then we need a way to
> > > distinguish
> > > > > > > > > > >> createTopicRequest
> > > > > > > > > > >> > > > > issued from the producer clients and the admin
> > > tools.
> > > > > > May
> > > > > > > > be we
> > > > > > > > > > >> can
> > > > > > > > > > >> > > add a
> > > > > > > > > > >> > > > > new field in createTopicRequest and set it
> > > differently
> > > > > > in
> > > > > > > > the
> > > > > > > > > > >> producer
> > > > > > > > > > >> > > > > client and the admin client. However, I am not
> > > sure if
> > > > > > > > that's
> > > > > > > > > > the
> > > > > > > > > > >> best
> > > > > > > > > > >> > > > > approach.
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > 2. Yes, refactoring existing requests is a
> > > non-trivial
> > > > > > > > amount of
> > > > > > > > > > >> work.
> > > > > > > > > > >> > > I
> > > > > > > > > > >> > > > > posted some comments in KAFKA-1927. We will
> > > probably
> > > > > > have
> > > > > > > > to fix
> > > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > > >> > > > > first, before adding the new logic in
> > KAFKA-1694.
> > > > > > > > Otherwise, the
> > > > > > > > > > >> > > changes
> > > > > > > > > > >> > > > > will be too big.
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > > >> > > > > 102.1. It's a bit weird that we return
> exception
> > > in
> > > > > the
> > > > > > > > api. It
> > > > > > > > > > >> seems
> > > > > > > > > > >> > > > that
> > > > > > > > > > >> > > > > we should either return error code or throw an
> > > > > exception
> > > > > > > > when
> > > > > > > > > > >> getting
> > > > > > > > > > >> > > the
> > > > > > > > > > >> > > > > response state.
> > > > > > > > > > >> > > > > 102.2. We probably shouldn't explicitly use
> the
> > > > > request
> > > > > > > > object
> > > > > > > > > > in
> > > > > > > > > > >> the
> > > > > > > > > > >> > > > api.
> > > > > > > > > > >> > > > > Not every request evolution requires an api
> > > change.
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > Thanks,
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > Jun
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii
> > Biletskyi
> > > <
> > > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > > > > Jun,
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > Thanks for you comments. Answers inline:
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > 100. There are a few fields such as
> > > > > ReplicaAssignment,
> > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> > represented
> > > as a
> > > > > > > > string,
> > > > > > > > > > but
> > > > > > > > > > >> > > > contain
> > > > > > > > > > >> > > > > > > composite structures in json. Could we
> > flatten
> > > > > them
> > > > > > > out
> > > > > > > > > > >> directly in
> > > > > > > > > > >> > > > the
> > > > > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > Yes, now with Admin Client this looks a bit
> > > weird.
> > > > > My
> > > > > > > > initial
> > > > > > > > > > >> > > > motivation
> > > > > > > > > > >> > > > > > was:
> > > > > > > > > > >> > > > > > ReassignPartitionCommand accepts input in
> > json,
> > > we
> > > > > > want
> > > > > > > to
> > > > > > > > > > >> remain
> > > > > > > > > > >> > > > tools'
> > > > > > > > > > >> > > > > > interfaces unchanged, where possible.
> > > > > > > > > > >> > > > > > If we port it to deserialized format, in CLI
> > > (/tools
> > > > > > > > project)
> > > > > > > > > > >> we will
> > > > > > > > > > >> > > > > have
> > > > > > > > > > >> > > > > > to add some
> > > > > > > > > > >> > > > > > json library since /tools is written in java
> > and
> > > > > we'll
> > > > > > > > need to
> > > > > > > > > > >> > > > > deserialize
> > > > > > > > > > >> > > > > > json file
> > > > > > > > > > >> > > > > > provided by a user. Can we quickly agree on
> > what
> > > > > this
> > > > > > > > library
> > > > > > > > > > >> should
> > > > > > > > > > >> > > be
> > > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > 101. Does TopicMetadataRequest v1 still
> > trigger
> > > auto
> > > > > > > topic
> > > > > > > > > > >> creation?
> > > > > > > > > > >> > > > This
> > > > > > > > > > >> > > > > > > will be a bit weird now that we have a
> > > separate
> > > > > > topic
> > > > > > > > > > >> creation api.
> > > > > > > > > > >> > > > > Have
> > > > > > > > > > >> > > > > > > you thought about how the new
> > > createTopicRequest
> > > > > and
> > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > >> > > > > > > v1 will be used in the producer/consumer
> > > client,
> > > > > in
> > > > > > > > addition
> > > > > > > > > > >> to
> > > > > > > > > > >> > > admin
> > > > > > > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > >> > > from
> > > > > > > > > > >> > > > > the
> > > > > > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > I agree, this strange logic should be fixed.
> > > I'm not
> > > > > > > > confident
> > > > > > > > > > >> in
> > > > > > > > > > >> > > this
> > > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > > >> > > > > > correct me if I'm wrong, but it doesn't look
> > > like a
> > > > > > hard
> > > > > > > > thing
> > > > > > > > > > >> to
> > > > > > > > > > >> > > do, I
> > > > > > > > > > >> > > > > > think we can
> > > > > > > > > > >> > > > > > leverage AdminClient for that in Producer
> and
> > > > > > > > unconditionally
> > > > > > > > > > >> remove
> > > > > > > > > > >> > > > > topic
> > > > > > > > > > >> > > > > > creation from the TopicMetadataRequest_V1.
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > 2. I think Jay meant getting rid of scala
> > > classes
> > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > >> > > did
> > > > > > > > > > >> > > > > > that
> > > > > > > > > > >> > > > > > > as a stop-gap thing when adding the new
> > > requests
> > > > > for
> > > > > > > the
> > > > > > > > > > >> consumers.
> > > > > > > > > > >> > > > > > > However, the long term plan is to get rid
> of
> > > all
> > > > > > those
> > > > > > > > and
> > > > > > > > > > >> just
> > > > > > > > > > >> > > reuse
> > > > > > > > > > >> > > > > the
> > > > > > > > > > >> > > > > > > java request/response in the client. Since
> > > this
> > > > > KIP
> > > > > > > > proposes
> > > > > > > > > > >> to
> > > > > > > > > > >> > > add a
> > > > > > > > > > >> > > > > > > significant number of new requests,
> perhaps
> > we
> > > > > > should
> > > > > > > > bite
> > > > > > > > > > the
> > > > > > > > > > >> > > bullet
> > > > > > > > > > >> > > > > to
> > > > > > > > > > >> > > > > > > clean up the existing scala requests first
> > > before
> > > > > > > > adding new
> > > > > > > > > > >> ones?
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > Yes, looks like I misunderstood the point of
> > > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > > >> > > > Okay, I
> > > > > > > > > > >> > > > > > will
> > > > > > > > > > >> > > > > > rework that. The only thing is that I don't
> > see
> > > any
> > > > > > > > example
> > > > > > > > > > how
> > > > > > > > > > >> it
> > > > > > > > > > >> > > was
> > > > > > > > > > >> > > > > done
> > > > > > > > > > >> > > > > > for at
> > > > > > > > > > >> > > > > > least one existing protocol message. Thus,
> as
> > I
> > > > > > > > understand, I
> > > > > > > > > > >> have to
> > > > > > > > > > >> > > > > think
> > > > > > > > > > >> > > > > > how we
> > > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > > >> > > > > > Re porting all existing RQ/RP in this patch.
> > > Sounds
> > > > > > > > > > reasonable,
> > > > > > > > > > >> but
> > > > > > > > > > >> > > if
> > > > > > > > > > >> > > > > it's
> > > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > > >> > > > > > requirement to have Admin KIP done, I'm
> afraid
> > > this
> > > > > > can
> > > > > > > > be a
> > > > > > > > > > >> serious
> > > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > > >> > > > > > There are 13 protocol messages and all that
> > > would
> > > > > > > require
> > > > > > > > not
> > > > > > > > > > >> only
> > > > > > > > > > >> > > unit
> > > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > > >> > > > > > intensive manual testing, no? I'm afraid I'm
> > > not the
> > > > > > > > right guy
> > > > > > > > > > >> to
> > > > > > > > > > >> > > cover
> > > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > > >> > > > > > Kafka core internals :). Let me know your
> > > thoughts
> > > > > on
> > > > > > > this
> > > > > > > > > > >> item. Btw
> > > > > > > > > > >> > > > > there
> > > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > > >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > > >> > > > ).
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <
> > > > > > > > jun@confluent.io>
> > > > > > > > > > >> wrote:
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > 100. There are a few fields such as
> > > > > > ReplicaAssignment,
> > > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> > represented
> > > as a
> > > > > > > > string,
> > > > > > > > > > but
> > > > > > > > > > >> > > > contain
> > > > > > > > > > >> > > > > > > composite structures in json. Could we
> > flatten
> > > > > them
> > > > > > > out
> > > > > > > > > > >> directly in
> > > > > > > > > > >> > > > the
> > > > > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > 101. Does TopicMetadataRequest v1 still
> > > trigger
> > > > > auto
> > > > > > > > topic
> > > > > > > > > > >> > > creation?
> > > > > > > > > > >> > > > > This
> > > > > > > > > > >> > > > > > > will be a bit weird now that we have a
> > > separate
> > > > > > topic
> > > > > > > > > > >> creation api.
> > > > > > > > > > >> > > > > Have
> > > > > > > > > > >> > > > > > > you thought about how the new
> > > createTopicRequest
> > > > > and
> > > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > > >> > > > > > > v1 will be used in the producer/consumer
> > > client,
> > > > > in
> > > > > > > > addition
> > > > > > > > > > >> to
> > > > > > > > > > >> > > admin
> > > > > > > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > >> > > from
> > > > > > > > > > >> > > > > the
> > > > > > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > 2. I think Jay meant getting rid of scala
> > > classes
> > > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > > >> > > did
> > > > > > > > > > >> > > > > > that
> > > > > > > > > > >> > > > > > > as a stop-gap thing when adding the new
> > > requests
> > > > > for
> > > > > > > the
> > > > > > > > > > >> consumers.
> > > > > > > > > > >> > > > > > > However, the long term plan is to get rid
> of
> > > all
> > > > > > those
> > > > > > > > and
> > > > > > > > > > >> just
> > > > > > > > > > >> > > reuse
> > > > > > > > > > >> > > > > the
> > > > > > > > > > >> > > > > > > java request/response in the client. Since
> > > this
> > > > > KIP
> > > > > > > > proposes
> > > > > > > > > > >> to
> > > > > > > > > > >> > > add a
> > > > > > > > > > >> > > > > > > significant number of new requests,
> perhaps
> > we
> > > > > > should
> > > > > > > > bite
> > > > > > > > > > the
> > > > > > > > > > >> > > bullet
> > > > > > > > > > >> > > > > to
> > > > > > > > > > >> > > > > > > clean up the existing scala requests first
> > > before
> > > > > > > > adding new
> > > > > > > > > > >> ones?
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > Jun
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii
> > > Biletskyi
> > > > > <
> > > > > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > As said above - I list again all
> comments
> > > from
> > > > > > this
> > > > > > > > thread
> > > > > > > > > > >> so we
> > > > > > > > > > >> > > > > > > > can see what's left and finalize all
> > pending
> > > > > > issues.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > > >> > > > > > > > 1. This is much needed functionality,
> but
> > > there
> > > > > > are
> > > > > > > a
> > > > > > > > lot
> > > > > > > > > > >> of the
> > > > > > > > > > >> > > so
> > > > > > > > > > >> > > > > > let's
> > > > > > > > > > >> > > > > > > > really think these protocols through. We
> > > really
> > > > > > want
> > > > > > > > to
> > > > > > > > > > end
> > > > > > > > > > >> up
> > > > > > > > > > >> > > > with a
> > > > > > > > > > >> > > > > > set
> > > > > > > > > > >> > > > > > > > of well thought-out, orthoganol apis.
> For
> > > this
> > > > > > > reason
> > > > > > > > I
> > > > > > > > > > >> think it
> > > > > > > > > > >> > > is
> > > > > > > > > > >> > > > > > > really
> > > > > > > > > > >> > > > > > > > important to think through the end state
> > > even if
> > > > > > > that
> > > > > > > > > > >> includes
> > > > > > > > > > >> > > APIs
> > > > > > > > > > >> > > > > we
> > > > > > > > > > >> > > > > > > > won't implement in the first phase.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: Definitely behind this. Would
> > appreciate
> > > if
> > > > > > there
> > > > > > > > are
> > > > > > > > > > >> concrete
> > > > > > > > > > >> > > > > > > comments
> > > > > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 2. Let's please please please wait until
> > we
> > > have
> > > > > > > > switched
> > > > > > > > > > >> the
> > > > > > > > > > >> > > > server
> > > > > > > > > > >> > > > > > over
> > > > > > > > > > >> > > > > > > > to the new java protocol definitions. If
> > we
> > > add
> > > > > > > upteen
> > > > > > > > > > more
> > > > > > > > > > >> ad
> > > > > > > > > > >> > > hoc
> > > > > > > > > > >> > > > > > scala
> > > > > > > > > > >> > > > > > > > objects that is just generating more
> work
> > > for
> > > > > the
> > > > > > > > > > >> conversion we
> > > > > > > > > > >> > > > know
> > > > > > > > > > >> > > > > we
> > > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed
> > scala
> > > > > > > protocol
> > > > > > > > > > >> classes.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 3. This proposal introduces a new type
> of
> > > > > optional
> > > > > > > > > > >> parameter.
> > > > > > > > > > >> > > This
> > > > > > > > > > >> > > > is
> > > > > > > > > > >> > > > > > > > inconsistent with everything else in the
> > > > > protocol
> > > > > > > > where we
> > > > > > > > > > >> use -1
> > > > > > > > > > >> > > > or
> > > > > > > > > > >> > > > > > some
> > > > > > > > > > >> > > > > > > > other marker value. You could argue
> either
> > > way
> > > > > but
> > > > > > > > let's
> > > > > > > > > > >> stick
> > > > > > > > > > >> > > with
> > > > > > > > > > >> > > > > > that
> > > > > > > > > > >> > > > > > > > for consistency. For clients that
> > > implemented
> > > > > the
> > > > > > > > protocol
> > > > > > > > > > >> in a
> > > > > > > > > > >> > > > > better
> > > > > > > > > > >> > > > > > > way
> > > > > > > > > > >> > > > > > > > than our scala code these basic
> primitives
> > > are
> > > > > > hard
> > > > > > > to
> > > > > > > > > > >> change.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed
> > > MaybeOf
> > > > > > type
> > > > > > > > and
> > > > > > > > > > >> changed
> > > > > > > > > > >> > > > > > protocol
> > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This seems to
> > duplicate
> > > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > > >> > > > > which
> > > > > > > > > > >> > > > > > > has
> > > > > > > > > > >> > > > > > > > brokers, topics, and partitions. I think
> > we
> > > > > should
> > > > > > > > rename
> > > > > > > > > > >> that
> > > > > > > > > > >> > > > > request
> > > > > > > > > > >> > > > > > > > ClusterMetadataRequest (or just
> > > MetadataRequest)
> > > > > > and
> > > > > > > > > > >> include the
> > > > > > > > > > >> > > id
> > > > > > > > > > >> > > > > of
> > > > > > > > > > >> > > > > > > the
> > > > > > > > > > >> > > > > > > > controller. Or are there other things we
> > > could
> > > > > add
> > > > > > > > here?
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: I agree. Updated the KIP. Let's
> extends
> > > > > > > > TopicMetadata
> > > > > > > > > > to
> > > > > > > > > > >> > > > version 2
> > > > > > > > > > >> > > > > > and
> > > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 5. We have a tendency to try to make a
> lot
> > > of
> > > > > > > requests
> > > > > > > > > > that
> > > > > > > > > > >> can
> > > > > > > > > > >> > > > only
> > > > > > > > > > >> > > > > go
> > > > > > > > > > >> > > > > > > to
> > > > > > > > > > >> > > > > > > > particular nodes. This adds a lot of
> > burden
> > > for
> > > > > > > client
> > > > > > > > > > >> > > > > implementations
> > > > > > > > > > >> > > > > > > (it
> > > > > > > > > > >> > > > > > > > sounds easy but each discovery can fail
> in
> > > many
> > > > > > > parts
> > > > > > > > so
> > > > > > > > > > it
> > > > > > > > > > >> ends
> > > > > > > > > > >> > > up
> > > > > > > > > > >> > > > > > > being a
> > > > > > > > > > >> > > > > > > > full state machine to do right). I think
> > we
> > > > > should
> > > > > > > > > > consider
> > > > > > > > > > >> > > making
> > > > > > > > > > >> > > > > > admin
> > > > > > > > > > >> > > > > > > > commands and ideally as many of the
> other
> > > apis
> > > > > as
> > > > > > > > possible
> > > > > > > > > > >> > > > available
> > > > > > > > > > >> > > > > on
> > > > > > > > > > >> > > > > > > all
> > > > > > > > > > >> > > > > > > > brokers and just redirect to the
> > controller
> > > on
> > > > > the
> > > > > > > > broker
> > > > > > > > > > >> side.
> > > > > > > > > > >> > > > > Perhaps
> > > > > > > > > > >> > > > > > > > there would be a general way to
> > encapsulate
> > > this
> > > > > > > > > > re-routing
> > > > > > > > > > >> > > > behavior.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: It's a very interesting idea, but
> seems
> > > there
> > > > > > are
> > > > > > > > some
> > > > > > > > > > >> > > concerns
> > > > > > > > > > >> > > > > > about
> > > > > > > > > > >> > > > > > > > this
> > > > > > > > > > >> > > > > > > > feature (like performance
> considerations,
> > > how
> > > > > this
> > > > > > > > will
> > > > > > > > > > >> > > complicate
> > > > > > > > > > >> > > > > > server
> > > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > > >> > > > > > > > I believe this shouldn't be a blocker.
> If
> > > this
> > > > > > > > feature is
> > > > > > > > > > >> > > > implemented
> > > > > > > > > > >> > > > > > at
> > > > > > > > > > >> > > > > > > > some
> > > > > > > > > > >> > > > > > > > point it won't affect Admin changes - at
> > > least
> > > > > no
> > > > > > > > changes
> > > > > > > > > > to
> > > > > > > > > > >> > > public
> > > > > > > > > > >> > > > > API
> > > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 6. We should probably normalize the key
> > > value
> > > > > > pairs
> > > > > > > > used
> > > > > > > > > > for
> > > > > > > > > > >> > > > configs
> > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > >> > > > > > > > than embedding a new formatting. So two
> > > strings
> > > > > > > rather
> > > > > > > > > > than
> > > > > > > > > > >> one
> > > > > > > > > > >> > > > with
> > > > > > > > > > >> > > > > an
> > > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch -
> normalized
> > > > > configs
> > > > > > > and
> > > > > > > > > > >> changed
> > > > > > > > > > >> > > > > protocol
> > > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 7. Is the postcondition of these APIs
> that
> > > the
> > > > > > > > command has
> > > > > > > > > > >> begun
> > > > > > > > > > >> > > or
> > > > > > > > > > >> > > > > > that
> > > > > > > > > > >> > > > > > > > the command has been completed? It is a
> > lot
> > > more
> > > > > > > > usable if
> > > > > > > > > > >> the
> > > > > > > > > > >> > > > > command
> > > > > > > > > > >> > > > > > > has
> > > > > > > > > > >> > > > > > > > been completed so you know that if you
> > > create a
> > > > > > > topic
> > > > > > > > and
> > > > > > > > > > >> then
> > > > > > > > > > >> > > > > publish
> > > > > > > > > > >> > > > > > to
> > > > > > > > > > >> > > > > > > > it you won't get an exception about
> there
> > > being
> > > > > no
> > > > > > > > such
> > > > > > > > > > >> topic.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: For long running requests (like
> > reassign
> > > > > > > > partitions) -
> > > > > > > > > > >> the
> > > > > > > > > > >> > > post
> > > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > > >> > > > > > > > command has begun - so we don't block
> the
> > > > > client.
> > > > > > In
> > > > > > > > case
> > > > > > > > > > >> of your
> > > > > > > > > > >> > > > > > > example -
> > > > > > > > > > >> > > > > > > > topic commands, this will be refactored
> > and
> > > > > topic
> > > > > > > > commands
> > > > > > > > > > >> will
> > > > > > > > > > >> > > be
> > > > > > > > > > >> > > > > > > executed
> > > > > > > > > > >> > > > > > > > immediately, since the Controller will
> > serve
> > > > > Admin
> > > > > > > > > > requests
> > > > > > > > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 8. Describe topic and list topics
> > duplicate
> > > a
> > > > > lot
> > > > > > of
> > > > > > > > stuff
> > > > > > > > > > >> in the
> > > > > > > > > > >> > > > > > > metadata
> > > > > > > > > > >> > > > > > > > request. Is there a reason to give back
> > > topics
> > > > > > > marked
> > > > > > > > for
> > > > > > > > > > >> > > > deletion? I
> > > > > > > > > > >> > > > > > > feel
> > > > > > > > > > >> > > > > > > > like if we just make the post-condition
> of
> > > the
> > > > > > > delete
> > > > > > > > > > >> command be
> > > > > > > > > > >> > > > that
> > > > > > > > > > >> > > > > > the
> > > > > > > > > > >> > > > > > > > topic is deleted that will get rid of
> the
> > > need
> > > > > for
> > > > > > > > this
> > > > > > > > > > >> right?
> > > > > > > > > > >> > > And
> > > > > > > > > > >> > > > it
> > > > > > > > > > >> > > > > > > will
> > > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed
> > > topics
> > > > > > marked
> > > > > > > > for
> > > > > > > > > > >> deletion
> > > > > > > > > > >> > > > in
> > > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 9. Should we consider batching these
> > > requests?
> > > > > We
> > > > > > > have
> > > > > > > > > > >> generally
> > > > > > > > > > >> > > > > tried
> > > > > > > > > > >> > > > > > to
> > > > > > > > > > >> > > > > > > > allow multiple operations to be batched.
> > My
> > > > > > > suspicion
> > > > > > > > is
> > > > > > > > > > >> that
> > > > > > > > > > >> > > > without
> > > > > > > > > > >> > > > > > > this
> > > > > > > > > > >> > > > > > > > we will get a lot of code that does
> > > something
> > > > > like
> > > > > > > > > > >> > > > > > > >    for(topic: adminClient.listTopics())
> > > > > > > > > > >> > > > > > > >       adminClient.describeTopic(topic)
> > > > > > > > > > >> > > > > > > > this code will work great when you test
> > on 5
> > > > > > topics
> > > > > > > > but
> > > > > > > > > > not
> > > > > > > > > > >> do as
> > > > > > > > > > >> > > > > well
> > > > > > > > > > >> > > > > > if
> > > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check "Topic
> > > Admin
> > > > > > > Schema"
> > > > > > > > > > >> section.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 10. I think we should also discuss how
> we
> > > want
> > > > > to
> > > > > > > > expose a
> > > > > > > > > > >> > > > > programmatic
> > > > > > > > > > >> > > > > > > JVM
> > > > > > > > > > >> > > > > > > > client api for these operations.
> Currently
> > > > > people
> > > > > > > > rely on
> > > > > > > > > > >> > > > AdminUtils
> > > > > > > > > > >> > > > > > > which
> > > > > > > > > > >> > > > > > > > is totally sketchy. I think we probably
> > need
> > > > > > another
> > > > > > > > > > client
> > > > > > > > > > >> under
> > > > > > > > > > >> > > > > > > clients/
> > > > > > > > > > >> > > > > > > > that exposes administrative
> functionality.
> > > We
> > > > > will
> > > > > > > > need
> > > > > > > > > > >> this just
> > > > > > > > > > >> > > > to
> > > > > > > > > > >> > > > > > > > properly test the new apis, I suspect.
> We
> > > should
> > > > > > > > figure
> > > > > > > > > > out
> > > > > > > > > > >> that
> > > > > > > > > > >> > > > API.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check "Admin
> > > Client"
> > > > > > > > section
> > > > > > > > > > >> with an
> > > > > > > > > > >> > > > > > initial
> > > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 11. The other information that would be
> > > really
> > > > > > > useful
> > > > > > > > to
> > > > > > > > > > get
> > > > > > > > > > >> > > would
> > > > > > > > > > >> > > > be
> > > > > > > > > > >> > > > > > > > information about partitions--how much
> > data
> > > is
> > > > > in
> > > > > > > the
> > > > > > > > > > >> partition,
> > > > > > > > > > >> > > > what
> > > > > > > > > > >> > > > > > are
> > > > > > > > > > >> > > > > > > > the segment offsets, what is the log-end
> > > offset
> > > > > > > (i.e.
> > > > > > > > last
> > > > > > > > > > >> > > offset),
> > > > > > > > > > >> > > > > > what
> > > > > > > > > > >> > > > > > > is
> > > > > > > > > > >> > > > > > > > the compaction point, etc. I think that
> > done
> > > > > right
> > > > > > > > this
> > > > > > > > > > >> would be
> > > > > > > > > > >> > > > the
> > > > > > > > > > >> > > > > > > > successor to the very awkward
> > OffsetRequest
> > > we
> > > > > > have
> > > > > > > > today.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest
> > in
> > > the
> > > > > > > latest
> > > > > > > > > > >> patch. I
> > > > > > > > > > >> > > > > believe
> > > > > > > > > > >> > > > > > > > this should
> > > > > > > > > > >> > > > > > > > be resolved in a separate KIP / jira
> > ticket.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 12. Generally we can do good error
> > handling
> > > > > > without
> > > > > > > > > > needing
> > > > > > > > > > >> > > custom
> > > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > > >> > > > > > > > messages. I.e. generally the client has
> > the
> > > > > > context
> > > > > > > to
> > > > > > > > > > know
> > > > > > > > > > >> that
> > > > > > > > > > >> > > if
> > > > > > > > > > >> > > > > it
> > > > > > > > > > >> > > > > > > got
> > > > > > > > > > >> > > > > > > > an error that the topic doesn't exist to
> > say
> > > > > > "Topic
> > > > > > > X
> > > > > > > > > > >> doesn't
> > > > > > > > > > >> > > > exist"
> > > > > > > > > > >> > > > > > > rather
> > > > > > > > > > >> > > > > > > > than "error code 14" (or whatever).
> Maybe
> > > there
> > > > > > are
> > > > > > > > > > specific
> > > > > > > > > > >> > > cases
> > > > > > > > > > >> > > > > > where
> > > > > > > > > > >> > > > > > > > this is hard? If we want to add
> > server-side
> > > > > error
> > > > > > > > messages
> > > > > > > > > > >> we
> > > > > > > > > > >> > > > really
> > > > > > > > > > >> > > > > do
> > > > > > > > > > >> > > > > > > > need to do this in a consistent way
> across
> > > the
> > > > > > > > protocol.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: Updated the KIP - please check
> > "Protocol
> > > > > > Errors"
> > > > > > > > > > >> section. I
> > > > > > > > > > >> > > > added
> > > > > > > > > > >> > > > > > the
> > > > > > > > > > >> > > > > > > > comprehensive, fine-grained list of
> error
> > > codes.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > > >> > > > > > > > 13. Describe topic request: it would be
> > > great to
> > > > > > go
> > > > > > > > beyond
> > > > > > > > > > >> just
> > > > > > > > > > >> > > > > > batching
> > > > > > > > > > >> > > > > > > on
> > > > > > > > > > >> > > > > > > > topic name regex for this request. For
> > > example,
> > > > > a
> > > > > > > very
> > > > > > > > > > >> common use
> > > > > > > > > > >> > > > > case
> > > > > > > > > > >> > > > > > of
> > > > > > > > > > >> > > > > > > > the topic command is to list all topics
> > > whose
> > > > > > config
> > > > > > > > A's
> > > > > > > > > > >> value is
> > > > > > > > > > >> > > > B.
> > > > > > > > > > >> > > > > > With
> > > > > > > > > > >> > > > > > > > topic name regex then we have to first
> > > retrieve
> > > > > > > > __all__
> > > > > > > > > > >> topics's
> > > > > > > > > > >> > > > > > > > description info and then filter at the
> > > client
> > > > > > end,
> > > > > > > > which
> > > > > > > > > > >> will
> > > > > > > > > > >> > > be a
> > > > > > > > > > >> > > > > > huge
> > > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > > >> > > > > > > > AND
> > > > > > > > > > >> > > > > > > > 14. Config K-Vs in create topic: this is
> > > related
> > > > > > to
> > > > > > > > the
> > > > > > > > > > >> previous
> > > > > > > > > > >> > > > > point;
> > > > > > > > > > >> > > > > > > > maybe we can add another metadata K-V or
> > > just a
> > > > > > > > metadata
> > > > > > > > > > >> string
> > > > > > > > > > >> > > > along
> > > > > > > > > > >> > > > > > > side
> > > > > > > > > > >> > > > > > > > with config K-V in create topic like we
> > did
> > > for
> > > > > > > offset
> > > > > > > > > > >> commit
> > > > > > > > > > >> > > > > request.
> > > > > > > > > > >> > > > > > > This
> > > > > > > > > > >> > > > > > > > field can be quite useful in storing
> > > information
> > > > > > > like
> > > > > > > > > > >> "owner" of
> > > > > > > > > > >> > > > the
> > > > > > > > > > >> > > > > > > topic
> > > > > > > > > > >> > > > > > > > who issue the create command, etc, which
> > is
> > > > > quite
> > > > > > > > > > important
> > > > > > > > > > >> for a
> > > > > > > > > > >> > > > > > > > multi-tenant setting. Then in the
> describe
> > > topic
> > > > > > > > request
> > > > > > > > > > we
> > > > > > > > > > >> can
> > > > > > > > > > >> > > > also
> > > > > > > > > > >> > > > > > > batch
> > > > > > > > > > >> > > > > > > > on regex of the metadata field.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: As discussed it is very interesting
> but
> > > can
> > > > > be
> > > > > > > > > > >> implemented
> > > > > > > > > > >> > > later
> > > > > > > > > > >> > > > > > after
> > > > > > > > > > >> > > > > > > > we have some basic functionality there.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > 15. Today all the admin operations are
> > > async in
> > > > > > the
> > > > > > > > sense
> > > > > > > > > > >> that
> > > > > > > > > > >> > > > > command
> > > > > > > > > > >> > > > > > > will
> > > > > > > > > > >> > > > > > > > return once it is written in ZK, and
> that
> > > is why
> > > > > > we
> > > > > > > > need
> > > > > > > > > > >> extra
> > > > > > > > > > >> > > > > > > verification
> > > > > > > > > > >> > > > > > > > like testUtil.waitForTopicCreated() /
> > verify
> > > > > > > partition
> > > > > > > > > > >> > > reassignment
> > > > > > > > > > >> > > > > > > > request, etc. With admin requests we
> could
> > > add a
> > > > > > > flag
> > > > > > > > to
> > > > > > > > > > >> enable /
> > > > > > > > > > >> > > > > > disable
> > > > > > > > > > >> > > > > > > > synchronous requests; when it is turned
> > on,
> > > the
> > > > > > > > response
> > > > > > > > > > >> will not
> > > > > > > > > > >> > > > > > return
> > > > > > > > > > >> > > > > > > > until the request has been completed.
> And
> > > for
> > > > > > async
> > > > > > > > > > >> requests we
> > > > > > > > > > >> > > can
> > > > > > > > > > >> > > > > > add a
> > > > > > > > > > >> > > > > > > > "token" field in the response, and then
> > only
> > > > > need
> > > > > > a
> > > > > > > > > > general
> > > > > > > > > > >> > > "admin
> > > > > > > > > > >> > > > > > > > verification request" with the given
> token
> > > to
> > > > > > check
> > > > > > > > if the
> > > > > > > > > > >> async
> > > > > > > > > > >> > > > > > request
> > > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: I see your point. My idea was to
> > provide
> > > > > > specific
> > > > > > > > > > >> > > > Verify...Request
> > > > > > > > > > >> > > > > > per
> > > > > > > > > > >> > > > > > > > each
> > > > > > > > > > >> > > > > > > > long running request, where needed. We
> can
> > > do it
> > > > > > the
> > > > > > > > way
> > > > > > > > > > you
> > > > > > > > > > >> > > > suggest.
> > > > > > > > > > >> > > > > > The
> > > > > > > > > > >> > > > > > > > only
> > > > > > > > > > >> > > > > > > > concern is that introducing a token we
> > again
> > > > > will
> > > > > > > make
> > > > > > > > > > >> schema
> > > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > > >> > > > > > > We
> > > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > > >> > > > > > > > to do similar thing introducing single
> > > > > > AdminRequest
> > > > > > > > for
> > > > > > > > > > all
> > > > > > > > > > >> topic
> > > > > > > > > > >> > > > > > > commands
> > > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > > >> > > > > > > > this idea because we wanted to have
> schema
> > > > > > defined.
> > > > > > > So
> > > > > > > > > > this
> > > > > > > > > > >> is
> > > > > > > > > > >> > > > more a
> > > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > > >> > > > > > > > a) have fixed schema but introduce each
> > > time new
> > > > > > > > > > >> Verify...Request
> > > > > > > > > > >> > > > for
> > > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > > >> > > > > > > > b) use one request for verification but
> > > > > generalize
> > > > > > > it
> > > > > > > > with
> > > > > > > > > > >> token
> > > > > > > > > > >> > > > > > > > I'm fine with whatever decision
> community
> > > come
> > > > > to.
> > > > > > > > Just
> > > > > > > > > > let
> > > > > > > > > > >> me
> > > > > > > > > > >> > > know
> > > > > > > > > > >> > > > > > your
> > > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > > >> > > > > > > > 16. Specifically for ownership, I think
> > the
> > > plan
> > > > > > is
> > > > > > > > to add
> > > > > > > > > > >> ACL
> > > > > > > > > > >> > > (it
> > > > > > > > > > >> > > > > > sounds
> > > > > > > > > > >> > > > > > > > like you are describing ACL) via an
> > external
> > > > > > system
> > > > > > > > > > (Argus,
> > > > > > > > > > >> > > > Sentry).
> > > > > > > > > > >> > > > > > > > I remember KIP-11 described this, but I
> > > can't
> > > > > find
> > > > > > > > the KIP
> > > > > > > > > > >> any
> > > > > > > > > > >> > > > > longer.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > A: Okay, no problem. Not sure though how
> > we
> > > are
> > > > > > > going
> > > > > > > > to
> > > > > > > > > > >> handle
> > > > > > > > > > >> > > it.
> > > > > > > > > > >> > > > > > Wait
> > > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > > >> > > > > > > > will be committed first and include
> > changes
> > > to
> > > > > > > > > > >> TopicMetadata from
> > > > > > > > > > >> > > > the
> > > > > > > > > > >> > > > > > > later
> > > > > > > > > > >> > > > > > > > one?
> > > > > > > > > > >> > > > > > > > Anyway, I added this note to "Open
> > > Questions"
> > > > > > > section
> > > > > > > > so
> > > > > > > > > > we
> > > > > > > > > > >> don't
> > > > > > > > > > >> > > > > miss
> > > > > > > > > > >> > > > > > > this
> > > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii
> > > > > > Biletskyi <
> > > > > > > > > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > > Today I uploaded the patch that covers
> > > some of
> > > > > > the
> > > > > > > > > > >> discussed
> > > > > > > > > > >> > > and
> > > > > > > > > > >> > > > > > agreed
> > > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > > >> > > > > > > > > - removed MaybeOf optional type
> > > > > > > > > > >> > > > > > > > > - switched to java protocol
> definitions
> > > > > > > > > > >> > > > > > > > > - simplified messages (normalized
> > configs,
> > > > > > removed
> > > > > > > > topic
> > > > > > > > > > >> marked
> > > > > > > > > > >> > > > for
> > > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > > I also updated the KIP-4 with
> respective
> > > > > changes
> > > > > > > and
> > > > > > > > > > >> wrote down
> > > > > > > > > > >> > > > my
> > > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > > >> > > > > > > > > - Batch Admin Operations -> updated
> Wire
> > > > > > Protocol
> > > > > > > > schema
> > > > > > > > > > >> > > proposal
> > > > > > > > > > >> > > > > > > > > - Remove ClusterMetadata -> changed to
> > > extend
> > > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > > >> > > > > > > > > - Admin Client -> updated my initial
> > > proposal
> > > > > to
> > > > > > > > reflect
> > > > > > > > > > >> > > batching
> > > > > > > > > > >> > > > > > > > > - Error codes -> proposed fine-grained
> > > error
> > > > > > code
> > > > > > > > > > instead
> > > > > > > > > > >> of
> > > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > > I will also send a separate email to
> > > cover all
> > > > > > > > comments
> > > > > > > > > > >> from
> > > > > > > > > > >> > > this
> > > > > > > > > > >> > > > > > > thread.
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen
> > > Shapira
> > > > > <
> > > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > > >> > > > > > > > >
> > > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > >
> > > > > > > > > > >>
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > > >> > > > > > > > >> It actually specifies changes to the
> > > Metadata
> > > > > > > > protocol,
> > > > > > > > > > >> so
> > > > > > > > > > >> > > > making
> > > > > > > > > > >> > > > > > sure
> > > > > > > > > > >> > > > > > > > >> both KIPs are consistent in this
> regard
> > > will
> > > > > be
> > > > > > > > good.
> > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM,
> Gwen
> > > > > Shapira
> > > > > > <
> > > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > >> > > > > > > > >> > Specifically for ownership, I think
> > the
> > > > > plan
> > > > > > is
> > > > > > > > to
> > > > > > > > > > add
> > > > > > > > > > >> ACL
> > > > > > > > > > >> > > (it
> > > > > > > > > > >> > > > > > > sounds
> > > > > > > > > > >> > > > > > > > >> > like you are describing ACL) via an
> > > > > external
> > > > > > > > system
> > > > > > > > > > >> (Argus,
> > > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > > >> > > > > > > > >> > I remember KIP-11 described this,
> > but I
> > > > > can't
> > > > > > > > find
> > > > > > > > > > the
> > > > > > > > > > >> KIP
> > > > > > > > > > >> > > any
> > > > > > > > > > >> > > > > > > longer.
> > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > >> > > > > > > > >> > Regardless, I think KIP-4 focuses
> on
> > > > > getting
> > > > > > > > > > >> information
> > > > > > > > > > >> > > that
> > > > > > > > > > >> > > > > > > already
> > > > > > > > > > >> > > > > > > > >> > exists from Kafka brokers, not on
> > > adding
> > > > > > > > information
> > > > > > > > > > >> that
> > > > > > > > > > >> > > > > perhaps
> > > > > > > > > > >> > > > > > > > >> > should exist but doesn't yet?
> > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM,
> > > Guozhang
> > > > > > Wang
> > > > > > > <
> > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > >> > > > > > > > >> >> Just want to elaborate a bit more
> on
> > > the
> > > > > > > > > > create-topic
> > > > > > > > > > >> > > > metadata
> > > > > > > > > > >> > > > > > and
> > > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > > >> > > > > > > > >> >> describe-topic based on config /
> > > metadata
> > > > > in
> > > > > > > my
> > > > > > > > > > >> previous
> > > > > > > > > > >> > > > email
> > > > > > > > > > >> > > > > as
> > > > > > > > > > >> > > > > > > we
> > > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The main motivation
> > is
> > > to
> > > > > > have
> > > > > > > > some
> > > > > > > > > > >> sort of
> > > > > > > > > > >> > > > > topic
> > > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > > >> > > > > > > > >> >> mechanisms, which I think is quite
> > > > > important
> > > > > > > in
> > > > > > > > a
> > > > > > > > > > >> > > > multi-tenant
> > > > > > > > > > >> > > > > /
> > > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > > >> > > > > > > > >> >> architecture: today anyone can
> > create
> > > > > topics
> > > > > > > in
> > > > > > > > a
> > > > > > > > > > >> shared
> > > > > > > > > > >> > > > Kafka
> > > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > > >> > > > > > > > >> >> there is no concept or "ownership"
> > of
> > > > > topics
> > > > > > > > that
> > > > > > > > > > are
> > > > > > > > > > >> > > created
> > > > > > > > > > >> > > > > by
> > > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > > >> > > > > > > > >> >> users. For example, at LinkedIn we
> > > > > basically
> > > > > > > > > > >> distinguish
> > > > > > > > > > >> > > > topic
> > > > > > > > > > >> > > > > > > owners
> > > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > > >> > > > > > > > >> >> some casual topic name prefix,
> which
> > > is a
> > > > > > bit
> > > > > > > > > > awkward
> > > > > > > > > > >> and
> > > > > > > > > > >> > > > does
> > > > > > > > > > >> > > > > > not
> > > > > > > > > > >> > > > > > > > fly
> > > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > > >> > > > > > > > >> >> we scale our customers. It would
> be
> > > great
> > > > > to
> > > > > > > use
> > > > > > > > > > >> > > > > describe-topics
> > > > > > > > > > >> > > > > > > such
> > > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > >> > > > > > > > >> >> Describe all topics that is
> created
> > > by me.
> > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > >> > > > > > > > >> >> Describe all topics whose
> retention
> > > time
> > > > > is
> > > > > > > > > > overriden
> > > > > > > > > > >> to X.
> > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > >> > > > > > > > >> >> Describe all topics whose writable
> > > group
> > > > > > > include
> > > > > > > > > > user
> > > > > > > > > > >> Y
> > > > > > > > > > >> > > (this
> > > > > > > > > > >> > > > > is
> > > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > >> > > > > > > > >> >> One possible way to achieve this
> is
> > to
> > > > > add a
> > > > > > > > > > metadata
> > > > > > > > > > >> file
> > > > > > > > > > >> > > in
> > > > > > > > > > >> > > > > the
> > > > > > > > > > >> > > > > > > > >> >> create-topic request, whose value
> > will
> > > > > also
> > > > > > be
> > > > > > > > > > >> written ZK
> > > > > > > > > > >> > > as
> > > > > > > > > > >> > > > we
> > > > > > > > > > >> > > > > > > > create
> > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > >> > > > > > > > >> >> topic; then describe-topics can
> > > choose to
> > > > > > > batch
> > > > > > > > > > topics
> > > > > > > > > > >> > > based
> > > > > > > > > > >> > > > on
> > > > > > > > > > >> > > > > > 1)
> > > > > > > > > > >> > > > > > > > name
> > > > > > > > > > >> > > > > > > > >> >> regex, 2) config K-V matching, 3)
> > > metadata
> > > > > > > > regex,
> > > > > > > > > > etc.
> > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM,
> > > Guozhang
> > > > > > Wang
> > > > > > > <
> > > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > > >> > > > > > > > >> >>> Thanks for the updated wiki. A
> few
> > > > > comments
> > > > > > > > below:
> > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > >> > > > > > > > >> >>> 1. Error description in
> response: I
> > > think
> > > > > > if
> > > > > > > > some
> > > > > > > > > > >> > > errorCode
> > > > > > > > > > >> > > > > > could
> > > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > > >> > > > > > > > >> >>> several different error cases
> then
> > we
> > > > > > should
> > > > > > > > really
> > > > > > > > > > >> change
> > > > > > > > > > >> > > > it
> > > > > > > > > > >> > > > > to
> > > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > > >> > > > > > > > >> >>> codes. In general the errorCode
> > > itself
> > > > > > would
> > > > > > > be
> > > > > > > > > > >> precise
> > > > > > > > > > >> > > and
> > > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > > >> > > > > > > > >> >>> describing the server side
> errors.
> > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic request: it
> would
> > > be
> > > > > > great
> > > > > > > > to go
> > > > > > > > > > >> beyond
> > > > > > > > > > >> > > > just
> > > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > > >> > > > > > > > >> >>> topic name regex for this
> request.
> > > For
> > > > > > > > example, a
> > > > > > > > > > >> very
> > > > > > > > > > >> > > > common
> > > > > > > > > > >> > > > > > use
> > > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > > >> > > > > > > > >> >>> the topic command is to list all
> > > topics
> > > > > > whose
> > > > > > > > > > config
> > > > > > > > > > >> A's
> > > > > > > > > > >> > > > value
> > > > > > > > > > >> > > > > > is
> > > > > > > > > > >> > > > > > > B.
> > > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > > >> > > > > > > > >> >>> topic name regex then we have to
> > > first
> > > > > > > retrieve
> > > > > > > > > > >> __all__
> > > > > > > > > > >> > > > > topics's
> > > > > > > > > > >> > > > > > > > >> >>> description info and then filter
> at
> > > the
> > > > > > > client
> > > > > > > > end,
> > > > > > > > > > >> which
> > > > > > > > > > >> > > > will
> > > > > > > > > > >> > > > > > be
> > > > > > > > > > >> > > > > > > a
> > > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in create topic:
> > this
> > > is
> > > > > > > > related to
> > > > > > > > > > >> the
> > > > > > > > > > >> > > > > previous
> > > > > > > > > > >> > > > > > > > point;
> > > > > > > > > > >> > > > > > > > >> >>> maybe we can add another metadata
> > > K-V or
> > > > > > > just a
> > > > > > > > > > >> metadata
> > > > > > > > > > >> > > > > string
> > > > > > > > > > >> > > > > > > > along
> > > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > > >> > > > > > > > >> >>> with config K-V in create topic
> > like
> > > we
> > > > > did
> > > > > > > for
> > > > > > > > > > >> offset
> > > > > > > > > > >> > > > commit
> > > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > > >> > > > > > > > >> >>> field can be quite useful in
> > storing
> > > > > > > > information
> > > > > > > > > > like
> > > > > > > > > > >> > > > "owner"
> > > > > > > > > > >> > > > > of
> > > > > > > > > > >> > > > > > > the
> > > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > > >> > > > > > > > >> >>> who issue the create command,
> etc,
> > > which
> > > > > is
> > > > > > > > quite
> > > > > > > > > > >> > > important
> > > > > > > > > > >> > > > > for
> > > > > > > > > > >> > > > > > a
> > > > > > > > > > >> > > > > > > > >> >>> multi-tenant setting. Then in the
> > > > > describe
> > > > > > > > topic
> > > > > > > > > > >> request
> > > > > > > > > > >> > > we
> > > > > > > > > > >> > > > > can
> > > > > > > > > > >> > > > > > > also
> > > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > > >> > > > > > > > >> >>> on regex of the metadata field.
> > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > >> > > > > > > > >> >>> 4. Today all the admin operations
> > are
> > > > > async
> > > > > > > in
> > > > > > > > the
> > > > > > > > > > >> sense
> > > > > > > > > > >> > > > that
> > > > > > > > > > >> > > > > > > > command
> > > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > > >> > > > > > > > >> >>> return once it is written in ZK,
> > and
> > > that
> > > > > > is
> > > > > > > > why we
> > > > > > > > > > >> need
> > > > > > > > > > >> > > > extra
> > > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > > >> > > > > > > > >> >>> like
> > testUtil.waitForTopicCreated() /
> > > > > > verify
> > > > > > > > > > >> partition
> > > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > > >> > > > > > > > >> >>> request, etc. With admin requests
> > we
> > > > > could
> > > > > > > add
> > > > > > > > a
> > > > > > > > > > >> flag to
> > > > > > > > > > >> > > > > enable
> > > > > > > > > > >> > > > > > /
> > > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > > >> > > > > > > > >> >>> synchronous requests; when it is
> > > turned
> > > > > on,
> > > > > > > the
> > > > > > > > > > >> response
> > > > > > > > > > >> > > > will
> > > > > > > > > > >> > > > > > not
> > > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > > >> > > > > > > > >> >>> until the request has been
> > > completed. And
> > > > > > for
> > > > > > > > async
> > > > > > > > > > >> > > requests
> > > > > > > > > > >> > > > > we
> > > > > > > > > > >> > > > > > > can
> > > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > > >> > > > > > > > >> >>> "token" field in the response,
> and
> > > then
> > > > > > only
> > > > > > > > need a
> > > > > > > > > > >> > > general
> > > > > > > > > > >> > > > > > "admin
> > > > > > > > > > >> > > > > > > > >> >>> verification request" with the
> > given
> > > > > token
> > > > > > to
> > > > > > > > check
> > > > > > > > > > >> if the
> > > > > > > > > > >> > > > > async
> > > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > >> > > > > > > > >> >>> 5. +1 for extending Metadata
> > request
> > > to
> > > > > > > include
> > > > > > > > > > >> > > controller /
> > > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > > >> > > > > > > > >> >>> information, and then we can
> remove
> > > the
> > > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM,
> > Joel
> > > > > > Koshy <
> > > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > > >> > > > > > > > >> >>>> Thanks for sending that out Joe
> -
> > I
> > > > > don't
> > > > > > > > think I
> > > > > > > > > > >> will be
> > > > > > > > > > >> > > > > able
> > > > > > > > > > >> > > > > > to
> > > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > > >> > > > > > > > >> >>>> it today, so if notes can be
> sent
> > > out
> > > > > > > > afterward
> > > > > > > > > > that
> > > > > > > > > > >> > > would
> > > > > > > > > > >> > > > be
> > > > > > > > > > >> > > > > > > > great.
> > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at
> 09:16:13AM
> > > > > -0800,
> > > > > > > Gwen
> > > > > > > > > > >> Shapira
> > > > > > > > > > >> > > > wrote:
> > > > > > > > > > >> > > > > > > > >> >>>> > Thanks for sending this out
> Joe.
> > > > > Looking
> > > > > > > > forward
> > > > > > > > > > >> to
> > > > > > > > > > >> > > > > chatting
> > > > > > > > > > >> > > > > > > with
> > > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46
> AM,
> > > Joe
> > > > > > Stein
> > > > > > > <
> > > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just sent out a
> google
> > > > > hangout
> > > > > > > > invite
> > > > > > > > > > to
> > > > > > > > > > >> all
> > > > > > > > > > >> > > > pmc,
> > > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > > >> > > > > > > > >> >>>> > > everyone I found working on
> a
> > > KIP.
> > > > > If
> > > > > > I
> > > > > > > > missed
> > > > > > > > > > >> anyone
> > > > > > > > > > >> > > > in
> > > > > > > > > > >> > > > > > the
> > > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > > >> > > > > > > > >> >>>> > > let me know and can update
> it,
> > > np.
> > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > >> > > > > > > > >> >>>> > > We should do this every
> > Tuesday
> > > @
> > > > > 2pm
> > > > > > > > Eastern
> > > > > > > > > > >> Time.
> > > > > > > > > > >> > > > Maybe
> > > > > > > > > > >> > > > > > we
> > > > > > > > > > >> > > > > > > > can
> > > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > > >> > > > > > > > >> >>>> > > help to make a google
> account
> > > so we
> > > > > > can
> > > > > > > > manage
> > > > > > > > > > >> > > better?
> > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > >
> > > > > > > > > > >>
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > > >> > > > > > > > >> >>>> > > in progress and related JIRA
> > > that
> > > > > are
> > > > > > > > > > >> interdependent
> > > > > > > > > > >> > > > and
> > > > > > > > > > >> > > > > > > common
> > > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59
> > > PM, Jay
> > > > > > > > Kreps <
> > > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on Google
> hangouts
> > > that
> > > > > > will
> > > > > > > > also
> > > > > > > > > > >> record
> > > > > > > > > > >> > > > and
> > > > > > > > > > >> > > > > > make
> > > > > > > > > > >> > > > > > > > the
> > > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at
> 11:49
> > > AM,
> > > > > > Jeff
> > > > > > > > > > Holoman
> > > > > > > > > > >> <
> > > > > > > > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > We're happy to send out a
> > > Webex
> > > > > for
> > > > > > > > this
> > > > > > > > > > >> purpose.
> > > > > > > > > > >> > > We
> > > > > > > > > > >> > > > > > could
> > > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions if there is
> > > interest and
> > > > > > > > publish
> > > > > > > > > > >> them
> > > > > > > > > > >> > > out.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at
> > > 11:28 AM,
> > > > > > Jay
> > > > > > > > > > Kreps <
> > > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try to get the
> > > technical
> > > > > > > > hang-ups
> > > > > > > > > > >> sorted
> > > > > > > > > > >> > > > out,
> > > > > > > > > > >> > > > > > > > though.
> > > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > there is some benefit
> to
> > > live
> > > > > > > > discussion
> > > > > > > > > > vs
> > > > > > > > > > >> > > > > writing. I
> > > > > > > > > > >> > > > > > > am
> > > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post instructions
> and
> > > give
> > > > > > > > ourselves a
> > > > > > > > > > >> few
> > > > > > > > > > >> > > > > attempts
> > > > > > > > > > >> > > > > > > we
> > > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that time
> > would
> > > work
> > > > > > for
> > > > > > > > > > >> me...any
> > > > > > > > > > >> > > > > > objections?
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at
> > > 8:18
> > > > > AM,
> > > > > > > Joe
> > > > > > > > > > Stein
> > > > > > > > > > >> <
> > > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly would be great
> > > maybe
> > > > > > like
> > > > > > > > every
> > > > > > > > > > >> > > Tuesday ~
> > > > > > > > > > >> > > > > 1pm
> > > > > > > > > > >> > > > > > > ET
> > > > > > > > > > >> > > > > > > > /
> > > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't mind google
> > > hangout
> > > > > but
> > > > > > > > there
> > > > > > > > > > is
> > > > > > > > > > >> > > always
> > > > > > > > > > >> > > > > some
> > > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we know the apache
> irc
> > > > > channel
> > > > > > > > works.
> > > > > > > > > > We
> > > > > > > > > > >> can
> > > > > > > > > > >> > > > start
> > > > > > > > > > >> > > > > > > there
> > > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull
> > > transcripts
> > > > > > too
> > > > > > > > and
> > > > > > > > > > >> > > associate
> > > > > > > > > > >> > > > to
> > > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it helpful for
> things.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015
> at
> > > 11:10
> > > > > > AM,
> > > > > > > > Jay
> > > > > > > > > > >> Kreps <
> > > > > > > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked about
> > > doing a
> > > > > > > Google
> > > > > > > > > > >> Hangout to
> > > > > > > > > > >> > > > chat
> > > > > > > > > > >> > > > > > > about
> > > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > generalizing that a
> > > little
> > > > > > > > > > further...I
> > > > > > > > > > >> > > > actually
> > > > > > > > > > >> > > > > > > think
> > > > > > > > > > >> > > > > > > > it
> > > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > everyone spending a
> > > > > > reasonable
> > > > > > > > chunk
> > > > > > > > > > of
> > > > > > > > > > >> > > their
> > > > > > > > > > >> > > > > week
> > > > > > > > > > >> > > > > > > on
> > > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync up once a
> week.
> > I
> > > > > think
> > > > > > we
> > > > > > > > could
> > > > > > > > > > >> use
> > > > > > > > > > >> > > time
> > > > > > > > > > >> > > > > to
> > > > > > > > > > >> > > > > > > talk
> > > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we
> > > are on
> > > > > > top
> > > > > > > of
> > > > > > > > > > code
> > > > > > > > > > >> > > > reviews,
> > > > > > > > > > >> > > > > > talk
> > > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We can make it
> > publicly
> > > > > > > > available so
> > > > > > > > > > >> that
> > > > > > > > > > >> > > any
> > > > > > > > > > >> > > > > one
> > > > > > > > > > >> > > > > > > can
> > > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any interest in
> doing
> > > this?
> > > > > > If
> > > > > > > so
> > > > > > > > > > I'll
> > > > > > > > > > >> try
> > > > > > > > > > >> > > to
> > > > > > > > > > >> > > > > set
> > > > > > > > > > >> > > > > > it
> > > > > > > > > > >> > > > > > > > up
> > > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24,
> 2015
> > at
> > > > > 3:57
> > > > > > > AM,
> > > > > > > > > > Andrii
> > > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > > >> > > > > > <
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > wrote:
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP
> > > page,
> > > > > > fixed
> > > > > > > /
> > > > > > > > > > >> aligned
> > > > > > > > > > >> > > > > document
> > > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > very initial
> > > proposal for
> > > > > > > > > > >> AdminClient so
> > > > > > > > > > >> > > we
> > > > > > > > > > >> > > > > have
> > > > > > > > > > >> > > > > > > > >> something
> > > > > > > > > > >> > > > > > > > >> >>>> to
> > > > > > > > > > >> > > > > > > > >> >>>> > >> start
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > from
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > while
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > discussing the
> KIP.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > > >> > > > > > > > >>
> > > > > > > > > > >> > > > > > > >
> > > > > > > > > > >> > > > > > >
> > > > > > > > > > >> > > > > >
> > > > > > > > > > >> > > > >
> > > > > > > > > > >> > > >
> > > > > > > > > > >> > >
> > > > > > > > > > >>
> > > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18,
> > 2015
> > > at
> > > > > > 9:01
> > > > > > > > PM,
> > > > > > > > > > >> Andrii
> > > > > > > > > > >> > > > > > Biletskyi
> > > > > > > > > > >> > > > > > > <
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > > wrote:
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re error
> > messages:
> > > you
> > > > > > are
> > > > > > > > right,
> > > > > > > > > > >> in
> > > > > > > > > > >> > > most
> > > > > > > > > > >> > > > > > cases
> > > > > > > > > > >> > > > > > > > >> client
> > > > > > > > > > >> > > > > > > > >> >>>> will
> > > > > > > > > > >> > > > > > > > >> >>>> > >> have
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > enough
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > context to show
> > > > > > descriptive
> > > > > > > > error
> > > > > > > > > > >> > > message.
> > > > > > > > > > >> > > > > My
> > > > > > > > > > >> > > > > > > > >> concern is
> > > > > > > > > > >> > > > > > > > >> >>>> that
> > > > > > > > > > >> > > > > > > > >> >>>> > >> we
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > will
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > have
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > to
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > add lots of new
> > > error
> > > > > > codes
> > > > > > > > for
> > > > > > > > > > >> each
> > > > > > > > > > >> > > > > possible
> > > > > > > > > > >> > > > > > > > >> error. Of
> > > > > > > > > > >> > > > > > > > >> >>>> course,
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > we
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > could
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > reuse
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > some of
> existing
> > > like
> > > > > > > > > > >> > > > > > > UknownTopicOrPartitionCode,
> > > > > > > > > > >> > > > > > > > >> but we
> > > > > > > > > > >> > > > > > > > >> >>>> will
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > also
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > need
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > to
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> > > > > > > > > > >> TopicAlreadyExistsCode,
> > > > > > > > > > >> > > > > > > > >> >>>> TopicConfigInvalid (both
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > topic
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > name and
> config,
> > > and
> > > > > > > probably
> > > > > > > > > > user
> > > > > > > > > > >> would
> > > > > > > > > > >> > > > > like
> > > > > > > > > > >> > > > > > to
> > > > > > > > > > >> > > > > > > > >> know
> > > > > > > > > > >> > > > > > > > >> >>>> what
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > exactly
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his
> > > > > config),
> > > > > > > > > > >> > > > > > > InvalidReplicaAssignment,
> > > > > > > > > > >> > > > > > > > >> >>>> > >> InternalError
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > (e.g.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > zookeeper
> > failure)
> > > etc.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > And this is
> only
> > > for
> > > > > > > > > > TopicCommand,
> > > > > > > > > > >> we
> > > > > > > > > > >> > > will
> > > > > > > > > > >> > > > > > also
> > > > > > > > > > >> > > > > > > > >> need to
> > > > > > > > > > >> > > > > > > > >> >>>> add
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > similar
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > for
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > ReassignPartitions,
> > > > > > > > > > >> PreferredReplica. So
> > > > > > > > > > >> > > > > we'll
> > > > > > > > > > >> > > > > > > end
> > > > > > > > > > >> > > > > > > > >> up
> > > > > > > > > > >> > > > > > > > >> >>>> with a
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > large
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > list
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > of
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > error codes,
> used
> > > only
> > > > > in
> > > > > > > > Admin
> > > > > > > > > > >> > > protocol.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Having said
> > that, I
> > > > > agree
> > > > > > > my
> > > > > > > > > > >> proposal is
> > > > > > > > > > >> > > > not
> > > > > > > > > > >> > > > > > > > >> consistent
> > > > > > > > > > >> > > > > > > > >> >>>> with
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > other
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > cases.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can
> find
> > > > > better
> > > > > > > > solution
> > > > > > > > > > >> or
> > > > > > > > > > >> > > > > something
> > > > > > > > > > >> > > > > > > > >> >>>> in-between.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout
> chat:
> > I
> > > > > think
> > > > > > it
> > > > > > > > is a
> > > > > > > > > > >> great
> > > > > > > > > > >> > > > idea.
> > > > > > > > > > >> > > > > > > This
> > > > > > > > > > >> > > > > > > > >> way we
> > > > > > > > > > >> > > > > > > > >> >>>> can
> > > > > > > > > > >> > > > > > > > >> >>>> > >> move
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > on
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > faster.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Let's agree
> > > somehow on
> > > > > > > > date/time
> > > > > > > > > > so
> > > > > > > > > > >> > > people
> > > > > > > > > > >> > > > > can
> > > > > > > > > > >> > > > > > > > join.
> > > > > > > > > > >> > > > > > > > >> >>>> Will work
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > me
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > and
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > next week
> almost
> > > > > anytime
> > > > > > if
> > > > > > > > > > agreed
> > > > > > > > > > >> in
> > > > > > > > > > >> > > > > advance.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18,
> > > 2015 at
> > > > > > > 7:09
> > > > > > > > PM,
> > > > > > > > > > >> Jay
> > > > > > > > > > >> > > > Kreps <
> > > > > > > > > > >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > wrote:
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Generally we
> can
> > > do
> > > > > good
> > > > > > > > error
> > > > > > > > > > >> handling
> > > > > > > > > > >> > > > > > without
> > > > > > > > > > >> > > > > > > > >> needing
> > > > > > > > > > >> > > > > > > > >> >>>> custom
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > server-side
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e.
> > > > > generally
> > > > > > > the
> > > > > > > > > > >> client has
> > > > > > > > > > >> > > > the
> > > > > > > > > > >> > > > > > > > >> context to
> > > > > > > > > > >> > > > > > > > >> >>>> know
> > > > > > > > > > >> > > > > > > > >> >>>> > >> that
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > if
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > got
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> an error that
> > the
> > > > > topic
> > > > > > > > doesn't
> > > > > > > > > > >> exist
> > > > > > > > > > >> > > to
> > > > > > > > > > >> > > > > say
> > > > > > > > > > >> > > > > > > > >> "Topic X
> > > > > > > > > > >> > > > > > > > >> >>>> doesn't
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > exist"
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> rather
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> than "error
> code
> > > 14"
> > > > > (or
> > > > > > > > > > >> whatever).
> > > > > > > > > > >> > > Maybe
> > > > > > > > > > >> > > > > > there
> > > > > > > > > > >> > > > > > > > are
> > > > > > > > > > >> > > > > > > > >> >>>> specific
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > cases
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > where
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> this is hard?
> If
> > > we
> > > > > want
> > > > > > > to
> > > > > > > > add
> > > > > > > > > > >> > > > server-side
> > > > > > > > > > >> > > > > > > error
> > > > > > > > > > >> > > > > > > > >> >>>> messages we
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > really
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > do
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> need to do
> this
> > > in a
> > > > > > > > consistent
> > > > > > > > > > >> way
> > > > > > > > > > >> > > > across
> > > > > > > > > > >> > > > > > the
> > > > > > > > > > >> > > > > > > > >> protocol.
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> I still have a
> > > bunch
> > > > > of
> > > > > > > open
> > > > > > > > > > >> questions
> > > > > > > > > > >> > > > here
> > > > > > > > > > >> > > > > > > from
> > > > > > > > > > >> > > > > > > > my
> > > > > > > > > > >> > > > > > > > >> >>>> previous
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > list. I
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > will
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> be out for the
> > > next
> > > > > few
> > > > > > > > days for
> > > > > > > > > > >> Strata
> > > > > > > > > > >> > > > > > though.
> > > > > > > > > > >> > > > > > > > >> Maybe
> > > > > > > > > > >> > > > > > > > >> >>>> we could
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > do
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > a
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Google
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat
> on
> > > any
> > > > > open
> > > > > > > > issues
> > > > > > > > > > >> some
> > > > > > > > > > >> > > time
> > > > > > > > > > >> > > > > > > towards
> > > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > > >> > > > > > > > >> >>>> end of
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > next
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > for
> > > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> anyone inte
> > ...
> >
> > [Message clipped]
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Guys,
I like this idea too. Let's stick with that. I'll update KIP accordingly.

I was also thinking we can avoid adding dedicated status check
requests for topic commands. - We have everything in DescribeTopic
for that! E.g.:
User issued CreateTopic - to check the status client sends DescribeTopic
and checks whether is something returned for that topic. The same for
alteration, deletion.
Btw, PreferredReplica status can be also checked with DescribeTopicRequest
(head of assigned replicas list == leader).
For ReassignPartitions as discussed we'll need to have a separate Verify...
request.

Thanks,
Andrii Biletskyi


On Thu, Mar 19, 2015 at 6:03 PM, Guozhang Wang <wa...@gmail.com> wrote:

> +1 on broker writing to ZK for async handling. I was thinking that in the
> end state the admin requests would be eventually sent to controller either
> through re-routing or clients discovering them, instead of letting
> controller listen on ZK admin path. But thinking about it a second time, I
> think it is actually simpler to let controller manage
> incoming queued-up admin requests through ZK.
>
> Guozhang
>
>
> On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <jj...@gmail.com> wrote:
>
> > +1 as well. I think it helps to keep the rerouting approach orthogonal
> > to this KIP.
> >
> > On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps wrote:
> > > I'm +1 on Jun's suggestion as long as it can work for all the requests.
> > >
> > > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Andrii,
> > > >
> > > > I think we agreed on the following.
> > > >
> > > > (a) Admin requests can be sent to and handled by any broker.
> > > > (b) Admin requests are processed asynchronously, at least for now.
> > That is,
> > > > when the client gets a response, it just means that the request is
> > > > initiated, but not necessarily completed. Then, it's up to the client
> > to
> > > > issue another request to check the status for completion.
> > > >
> > > > To support (a), we were thinking of doing request forwarding to the
> > > > controller (utilizing KAFKA-1912). I am making an alternative
> proposal.
> > > > Basically, the broker can just write to ZooKeeper to inform the
> > controller
> > > > about the request. For example, to handle partitionReassignment, the
> > broker
> > > > will just write the requested partitions to
> /admin/reassign_partitions
> > > > (like what AdminUtils currently does) and then send a response to the
> > > > client. This shouldn't take long and the implementation will be
> simpler
> > > > than forwarding the requests to the controller through RPC.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Jun,
> > > > >
> > > > > I might be wrong but didn't we agree we will let any broker from
> the
> > > > > cluster handle *long-running* admin requests (at this time
> > > > preferredReplica
> > > > > and
> > > > > reassignPartitions), via zk admin path. Thus CreateTopics etc
> should
> > be
> > > > > sent
> > > > > only to the controller.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > >
> > > > > > Joel, Andril,
> > > > > >
> > > > > > I think we agreed that those admin requests can be issued to any
> > > > broker.
> > > > > > Because of that, there doesn't seem to be a strong need to know
> the
> > > > > > controller. So, perhaps we can proceed by not making any change
> to
> > the
> > > > > > format of TMR right now. When we start using create topic request
> > in
> > > > the
> > > > > > producer, we will need a new version of TMR that doesn't trigger
> > auto
> > > > > topic
> > > > > > creation. But that can be done later.
> > > > > >
> > > > > > As a first cut implementation, I think the broker can just write
> > to ZK
> > > > > > directly for
> > > > > >
> createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > > requests, instead of forwarding them to the controller. This will
> > > > > simplify
> > > > > > the implementation on the broker side.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <
> jjkoshy.w@gmail.com>
> > > > > wrote:
> > > > > >
> > > > > > > (Thanks Andrii for the summary)
> > > > > > >
> > > > > > > For (1) yes we will circle back on that shortly after syncing
> up
> > in
> > > > > > > person. I think it is close to getting committed although
> > development
> > > > > > > for KAFKA-1927 can probably begin without it.
> > > > > > >
> > > > > > > There is one more item we covered at the hangout. i.e., whether
> > we
> > > > > > > want to add the coordinator to the topic metadata response or
> > provide
> > > > > > > a clearer ClusterMetadataRequest.
> > > > > > >
> > > > > > > There are two reasons I think we should try and avoid adding
> the
> > > > > > > field:
> > > > > > > - It is irrelevant to topic metadata
> > > > > > > - If we finally do request rerouting in Kafka then the field
> > would
> > > > add
> > > > > > >   little to no value. (It still helps to have a separate
> > > > > > >   ClusterMetadataRequest to query for cluster-wide information
> > such
> > > > as
> > > > > > >   'which broker is the controller?' as Joe mentioned.)
> > > > > > >
> > > > > > > I think it would be cleaner to have an explicit
> > > > ClusterMetadataRequest
> > > > > > > that you can send to any broker in order to obtain the
> controller
> > > > (and
> > > > > > > in the future possibly other cluster-wide information). I think
> > the
> > > > > > > main argument against doing this and instead adding it to the
> > topic
> > > > > > > metadata response was convenience - i.e., you don't have to
> > discover
> > > > > > > the controller in advance. However, I don't see much actual
> > > > > > > benefit/convenience in this and in fact think it is a
> non-issue.
> > Let
> > > > > > > me know if I'm overlooking something here.
> > > > > > >
> > > > > > > As an example, say we need to initiate partition reassignment
> by
> > > > > > > issuing the new ReassignPartitionsRequest to the controller
> > (assume
> > > > we
> > > > > > > already have the desired manual partition assignment).  If we
> > are to
> > > > > > > augment topic metadata response then the flow be something like
> > this
> > > > :
> > > > > > >
> > > > > > > - Issue topic metadata request to any broker (and discover the
> > > > > > >   controller
> > > > > > > - Connect to controller if required (i.e., if the broker above
> !=
> > > > > > >   controller)
> > > > > > > - Issue the partition reassignment request to the controller.
> > > > > > >
> > > > > > > With an explicit cluster metadata request it would be:
> > > > > > > - Issue cluster metadata request to any broker
> > > > > > > - Connect to controller if required (i.e., if the broker above
> !=
> > > > > > >   controller)
> > > > > > > - Issue the partition reassignment request
> > > > > > >
> > > > > > > So it seems to add little practical value and bloats topic
> > metadata
> > > > > > > response with an irrelevant detail.
> > > > > > >
> > > > > > > The other angle to this is the following - is it a matter of
> > naming?
> > > > > > > Should we just rename topic metadata request/response to just
> > > > > > > MetadataRequest/Response and add cluster metadata to it? By
> that
> > same
> > > > > > > token should we also allow querying for the consumer
> coordinator
> > (and
> > > > > > > in future transaction coordinator) as well? This leads to a
> > bloated
> > > > > > > request which isn't very appealing and altogether confusing.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Joel
> > > > > > >
> > > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > > > > > > > Andri,
> > > > > > > >
> > > > > > > > Thanks for the summary.
> > > > > > > >
> > > > > > > > 1. I just realized that in order to start working on
> > KAFKA-1927, we
> > > > > > will
> > > > > > > > need to merge the changes to OffsetCommitRequest (from 0.8.2)
> > to
> > > > > trunk.
> > > > > > > > This is planned to be done as part of KAFKA-1634. So, we will
> > need
> > > > > > > Guozhang
> > > > > > > > and Joel's help to wrap this up.
> > > > > > > >
> > > > > > > > 2. Thinking about this a bit more, if the semantic of those
> > "write"
> > > > > > > > requests is async (i.e., after the client gets a response, it
> > just
> > > > > > means
> > > > > > > > that the operation is initiated, but not necessarily
> > completed), we
> > > > > > don't
> > > > > > > > really need to forward the requests to the controller.
> > Instead, the
> > > > > > > > receiving broker can just write the operation to ZK as the
> > admin
> > > > > > command
> > > > > > > > line tool previously does. This will simplify the
> > implementation.
> > > > > > > >
> > > > > > > > 8. There is another implementation detail for describe topic.
> > > > > Ideally,
> > > > > > we
> > > > > > > > want to read the topic config from the broker cache, instead
> of
> > > > > > > ZooKeeper.
> > > > > > > > Currently, every broker reads the topic-level config for all
> > > > topics.
> > > > > > > > However, it ignores those for topics not hosted on itself.
> So,
> > we
> > > > may
> > > > > > > need
> > > > > > > > to change TopicConfigManager a bit so that it caches the
> > configs
> > > > for
> > > > > > all
> > > > > > > > topics.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > >
> > > > > > > > Jun
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >
> > > > > > > > > Guys,
> > > > > > > > >
> > > > > > > > > Thanks for a great discussion!
> > > > > > > > > Here are the actions points:
> > > > > > > > >
> > > > > > > > > 1. Q: Get rid of all scala requests objects, use java
> > protocol
> > > > > > > definitions.
> > > > > > > > >     A: Gwen kindly took that (KAFKA-1927). It's important
> to
> > > > speed
> > > > > up
> > > > > > > > > review procedure
> > > > > > > > >          there since this ticket blocks other important
> > changes.
> > > > > > > > >
> > > > > > > > > 2. Q: Generic re-reroute facility vs client maintaining
> > cluster
> > > > > > state.
> > > > > > > > >     A: Jay has added pseudo code to KAFKA-1912 - need to
> > consider
> > > > > > > whether
> > > > > > > > > this will be
> > > > > > > > >         easy to implement as a server-side feature
> (comments
> > are
> > > > > > > > > welcomed!).
> > > > > > > > >
> > > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > > >     A: This might be useful for clients, add this to
> > > > > > > TopicMetadataResponse
> > > > > > > > > (already in KIP).
> > > > > > > > >
> > > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > > >     A: I will add proposed by Jun solution (using clientId
> > for
> > > > > that)
> > > > > > > to the
> > > > > > > > > KIP.
> > > > > > > > >
> > > > > > > > > 5. Q: Bumping new versions of TMR vs grabbing all protocol
> > > > changes
> > > > > in
> > > > > > > one
> > > > > > > > > version.
> > > > > > > > >     A: It was decided to try to gather all changes to
> > protocol
> > > > > > (before
> > > > > > > > > release).
> > > > > > > > >         In case of TMR it worth checking: KAFKA-2020 and
> > KIP-13
> > > > > > > (quotas)
> > > > > > > > >
> > > > > > > > > 6. Q: JSON lib is needed to deserialize user's input in CLI
> > tool.
> > > > > > > > >     A: Use jackson for that, /tools project is a separate
> > jar so
> > > > > > > shouldn't
> > > > > > > > > be a big deal.
> > > > > > > > >
> > > > > > > > > 7.  Q: VerifyReassingPartitions vs generic status check
> > command.
> > > > > > > > >      A: For long-running requests like reassign partitions
> > > > > *progress*
> > > > > > > check
> > > > > > > > > request is useful,
> > > > > > > > >          it makes sense to introduce it.
> > > > > > > > >
> > > > > > > > >  Please add, correct me if I missed something.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Andrii Biletskyi
> > > > > > > > >
> > > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >
> > > > > > > > > > Joel,
> > > > > > > > > >
> > > > > > > > > > You are right, I removed ClusterMetadata because we have
> > > > > partially
> > > > > > > > > > what we need in TopicMetadata. Also, as Jay pointed out
> > > > earlier,
> > > > > we
> > > > > > > > > > would like to have "orthogonal" API, but at the same time
> > we
> > > > need
> > > > > > > > > > to be backward compatible.
> > > > > > > > > >
> > > > > > > > > > But I like your idea and even have some other arguments
> for
> > > > this
> > > > > > > option:
> > > > > > > > > > There is also DescribeTopicRequest which was proposed in
> > this
> > > > > KIP,
> > > > > > > > > > it returns topic configs, partitions, replication factor
> > plus
> > > > > > > partition
> > > > > > > > > > ISR, ASR,
> > > > > > > > > > leader replica. The later part is really already there in
> > > > > > > > > > TopicMetadataRequest.
> > > > > > > > > > So again we'll have to add stuff to TMR, not to duplicate
> > some
> > > > > info
> > > > > > > in
> > > > > > > > > > newly added requests. However, this way we'll end up with
> > > > > "monster"
> > > > > > > > > > request which returns cluster metadata, topic replication
> > and
> > > > > > config
> > > > > > > info
> > > > > > > > > > plus partition replication data. Seems logical to split
> > TMR to
> > > > > > > > > > - ClusterMetadata (brokers + controller, maybe smth else)
> > > > > > > > > > - TopicMetadata (topic info + partition details)
> > > > > > > > > > But since current TMR is involved in lots of places
> > (including
> > > > > > > network
> > > > > > > > > > client,
> > > > > > > > > > as I understand) this might be very serious change and it
> > > > > probably
> > > > > > > makes
> > > > > > > > > > sense to stick with current approach.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Andrii Biletskyi
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <
> > > > jjkoshy.w@gmail.com
> > > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > >> I may be missing some context but hopefully this will
> > also be
> > > > > > > covered
> > > > > > > > > >> today: I thought the earlier proposal where there was an
> > > > > explicit
> > > > > > > > > >> ClusterMetadata request was clearer and explicit. During
> > the
> > > > > > course
> > > > > > > of
> > > > > > > > > >> this thread I think the conclusion was that the main
> need
> > was
> > > > > for
> > > > > > > > > >> controller information and that can be rolled into the
> > topic
> > > > > > > metadata
> > > > > > > > > >> response but that seems a bit irrelevant to topic
> > metadata.
> > > > > FWIW I
> > > > > > > > > >> think the full broker-list is also irrelevant to topic
> > > > metadata,
> > > > > > but
> > > > > > > > > >> it is already there and in use. I think there is still
> > room
> > > > for
> > > > > an
> > > > > > > > > >> explicit ClusterMetadata request since there may be
> other
> > > > > > > > > >> cluster-level information that we may want to add over
> > time
> > > > (and
> > > > > > > that
> > > > > > > > > >> have nothing to do with topic metadata).
> > > > > > > > > >>
> > > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii
> Biletskyi
> > > > > wrote:
> > > > > > > > > >> > Jun,
> > > > > > > > > >> >
> > > > > > > > > >> > 101. Okay, if you say that such use case is
> important. I
> > > > also
> > > > > > > think
> > > > > > > > > >> > using clientId for these purposes is fine - if we
> > already
> > > > have
> > > > > > > this
> > > > > > > > > >> field
> > > > > > > > > >> > as part of all Wire protocol messages, why not use
> that.
> > > > > > > > > >> > I will update KIP-4 page if nobody has other ideas
> > (which
> > > > may
> > > > > > > come up
> > > > > > > > > >> > during the call today).
> > > > > > > > > >> >
> > > > > > > > > >> > 102.1 Agree, I'll update the KIP accordingly. I think
> > we can
> > > > > add
> > > > > > > new,
> > > > > > > > > >> > fine-grained error codes if some error code received
> in
> > > > > specific
> > > > > > > case
> > > > > > > > > >> > won't give enough context to return a descriptive
> error
> > > > > message
> > > > > > > for
> > > > > > > > > >> user.
> > > > > > > > > >> >
> > > > > > > > > >> > Look forward to discussing all outstanding issues in
> > detail
> > > > > > today
> > > > > > > > > during
> > > > > > > > > >> > the call.
> > > > > > > > > >> >
> > > > > > > > > >> > Thanks,
> > > > > > > > > >> > Andrii Biletskyi
> > > > > > > > > >> >
> > > > > > > > > >> >
> > > > > > > > > >> >
> > > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <
> > jun@confluent.io
> > > > >
> > > > > > > wrote:
> > > > > > > > > >> >
> > > > > > > > > >> > > 101. There may be a use case where you only want the
> > > > topics
> > > > > to
> > > > > > > be
> > > > > > > > > >> created
> > > > > > > > > >> > > manually by admins. Currently, you can do that by
> > > > disabling
> > > > > > auto
> > > > > > > > > topic
> > > > > > > > > >> > > creation and issue topic creation from the
> > TopicCommand.
> > > > If
> > > > > we
> > > > > > > > > >> disable auto
> > > > > > > > > >> > > topic creation completely on the broker and don't
> > have a
> > > > way
> > > > > > to
> > > > > > > > > >> distinguish
> > > > > > > > > >> > > between topic creation requests from the regular
> > clients
> > > > and
> > > > > > the
> > > > > > > > > >> admin, we
> > > > > > > > > >> > > can't support manual topic creation any more. I was
> > > > thinking
> > > > > > > that
> > > > > > > > > >> another
> > > > > > > > > >> > > way of distinguishing the clients making the topic
> > > > creation
> > > > > > > requests
> > > > > > > > > >> is
> > > > > > > > > >> > > using clientId. For example, the admin tool can set
> > it to
> > > > > > > something
> > > > > > > > > >> like
> > > > > > > > > >> > > admin and the broker can treat that clientId
> > specially.
> > > > > > > > > >> > >
> > > > > > > > > >> > > Also, there is a related discussion in KAFKA-2020.
> > > > > Currently,
> > > > > > > we do
> > > > > > > > > >> the
> > > > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > > > >> > >
> > > > > > > > > >> > > 1. If leader is not available, we set the partition
> > level
> > > > > > error
> > > > > > > code
> > > > > > > > > >> to
> > > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > > >> > > 2. If a non-leader replica is not available, we take
> > that
> > > > > > > replica
> > > > > > > > > out
> > > > > > > > > >> of
> > > > > > > > > >> > > the assigned replica list and isr in the response.
> As
> > an
> > > > > > > indication
> > > > > > > > > >> for
> > > > > > > > > >> > > doing that, we set the partition level error code to
> > > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > > >> > >
> > > > > > > > > >> > > This has a few problems. First, ReplicaNotAvailable
> > > > probably
> > > > > > > > > >> shouldn't be
> > > > > > > > > >> > > an error, at least for the normal producer/consumer
> > > > clients
> > > > > > that
> > > > > > > > > just
> > > > > > > > > >> want
> > > > > > > > > >> > > to find out the leader. Second, it can happen that
> > both
> > > > the
> > > > > > > leader
> > > > > > > > > and
> > > > > > > > > >> > > another replica are not available at the same time.
> > There
> > > > is
> > > > > > no
> > > > > > > > > error
> > > > > > > > > >> code
> > > > > > > > > >> > > to indicate both. Third, even if a replica is not
> > > > available,
> > > > > > > it's
> > > > > > > > > >> still
> > > > > > > > > >> > > useful to return its replica id since some clients
> > (e.g.
> > > > > admin
> > > > > > > tool)
> > > > > > > > > >> may
> > > > > > > > > >> > > still make use of it.
> > > > > > > > > >> > >
> > > > > > > > > >> > > One way to address this issue is to always return
> the
> > > > > replica
> > > > > > > id for
> > > > > > > > > >> > > leader, assigned replicas, and isr regardless of
> > whether
> > > > the
> > > > > > > > > >> corresponding
> > > > > > > > > >> > > broker is live or not. Since we also return the list
> > of
> > > > live
> > > > > > > > > brokers,
> > > > > > > > > >> the
> > > > > > > > > >> > > client can figure out whether a leader or a replica
> is
> > > > live
> > > > > or
> > > > > > > not
> > > > > > > > > >> and act
> > > > > > > > > >> > > accordingly. This way, we don't need to set the
> > partition
> > > > > > level
> > > > > > > > > error
> > > > > > > > > >> code
> > > > > > > > > >> > > when the leader or a replica is not available. This
> > > > doesn't
> > > > > > > change
> > > > > > > > > >> the wire
> > > > > > > > > >> > > protocol, but does change the semantics. Since we
> are
> > > > > evolving
> > > > > > > the
> > > > > > > > > >> protocol
> > > > > > > > > >> > > of TopicMetadataRequest here, we can potentially
> > piggyback
> > > > > the
> > > > > > > > > change.
> > > > > > > > > >> > >
> > > > > > > > > >> > > 102.1 For those types of errors due to invalid
> input,
> > > > > > shouldn't
> > > > > > > we
> > > > > > > > > >> just
> > > > > > > > > >> > > guard it at parameter validation time and throw
> > > > > > > > > >> InvalidArgumentException
> > > > > > > > > >> > > without even sending the request to the broker?
> > > > > > > > > >> > >
> > > > > > > > > >> > > Thanks,
> > > > > > > > > >> > >
> > > > > > > > > >> > > Jun
> > > > > > > > > >> > >
> > > > > > > > > >> > >
> > > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > > > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > >> > >
> > > > > > > > > >> > > > Jun,
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Answering your questions:
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > 101. If I understand you correctly, you are saying
> > > > future
> > > > > > > producer
> > > > > > > > > >> > > versions
> > > > > > > > > >> > > > (which
> > > > > > > > > >> > > > will be ported to TMR_V1) won't be able to
> > automatically
> > > > > > > create
> > > > > > > > > >> topic (if
> > > > > > > > > >> > > > we
> > > > > > > > > >> > > > unconditionally remove topic creation from there).
> > But
> > > > we
> > > > > > > need to
> > > > > > > > > >> this
> > > > > > > > > >> > > > preserve logic.
> > > > > > > > > >> > > > Ok, about your proposal: I'm not a big fan too,
> > when it
> > > > > > comes
> > > > > > > to
> > > > > > > > > >> > > > differentiating
> > > > > > > > > >> > > > clients directly in protocol schema. And also I'm
> > not
> > > > > sure I
> > > > > > > > > >> understand
> > > > > > > > > >> > > at
> > > > > > > > > >> > > > all why
> > > > > > > > > >> > > > auto.create.topics.enable is a server side
> > > > configuration.
> > > > > > Can
> > > > > > > we
> > > > > > > > > >> > > deprecate
> > > > > > > > > >> > > > this setting
> > > > > > > > > >> > > > in future versions, add this setting to producer
> and
> > > > based
> > > > > > on
> > > > > > > that
> > > > > > > > > >> upon
> > > > > > > > > >> > > > receiving
> > > > > > > > > >> > > > UnknownTopic create topic explicitly by a separate
> > > > > producer
> > > > > > > call
> > > > > > > > > via
> > > > > > > > > >> > > > adminClient?
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > 102.1. Hm, yes. It's because we want to support
> > batching
> > > > > and
> > > > > > > at
> > > > > > > > > the
> > > > > > > > > >> same
> > > > > > > > > >> > > > time we
> > > > > > > > > >> > > > want to give descriptive error messages for
> clients.
> > > > Since
> > > > > > > > > >> AdminClient
> > > > > > > > > >> > > > holds the context
> > > > > > > > > >> > > > to construct such messages (e.g. AdminClient layer
> > can
> > > > > know
> > > > > > > that
> > > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > > >> > > > means two cases: either invalid number - e.g. -1;
> or
> > > > > > > > > >> replication-factor
> > > > > > > > > >> > > was
> > > > > > > > > >> > > > provided while
> > > > > > > > > >> > > > partitions argument wasn't) - I wrapped responses
> in
> > > > > > > Exceptions.
> > > > > > > > > >> But I'm
> > > > > > > > > >> > > > open to any
> > > > > > > > > >> > > > other ideas, this was just initial version.
> > > > > > > > > >> > > > 102.2. Yes, I agree. I'll change that to probably
> > some
> > > > > other
> > > > > > > dto.
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > Thanks,
> > > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <
> > > > > jun@confluent.io>
> > > > > > > > > wrote:
> > > > > > > > > >> > > >
> > > > > > > > > >> > > > > Andrii,
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > 101. That's what I was thinking too, but it may
> > not be
> > > > > > that
> > > > > > > > > >> simple. In
> > > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > > >> > > > > we can let it not trigger auto topic creation.
> > Then,
> > > > in
> > > > > > the
> > > > > > > > > >> producer
> > > > > > > > > >> > > > side,
> > > > > > > > > >> > > > > if it gets an UnknownTopicException, it can
> > explicitly
> > > > > > > issue a
> > > > > > > > > >> > > > > createTopicRequest for auto topic creation. On
> the
> > > > > > consumer
> > > > > > > > > side,
> > > > > > > > > >> it
> > > > > > > > > >> > > will
> > > > > > > > > >> > > > > never issue createTopicRequest. This works when
> > auto
> > > > > topic
> > > > > > > > > >> creation is
> > > > > > > > > >> > > > > enabled on the broker side. However, I am not
> > sure how
> > > > > > > things
> > > > > > > > > >> will work
> > > > > > > > > >> > > > > when auto topic creation is disabled on the
> broker
> > > > side.
> > > > > > In
> > > > > > > this
> > > > > > > > > >> case,
> > > > > > > > > >> > > we
> > > > > > > > > >> > > > > want to have a way to manually create a topic,
> > > > > potentially
> > > > > > > > > through
> > > > > > > > > >> > > admin
> > > > > > > > > >> > > > > commands. However, then we need a way to
> > distinguish
> > > > > > > > > >> createTopicRequest
> > > > > > > > > >> > > > > issued from the producer clients and the admin
> > tools.
> > > > > May
> > > > > > > be we
> > > > > > > > > >> can
> > > > > > > > > >> > > add a
> > > > > > > > > >> > > > > new field in createTopicRequest and set it
> > differently
> > > > > in
> > > > > > > the
> > > > > > > > > >> producer
> > > > > > > > > >> > > > > client and the admin client. However, I am not
> > sure if
> > > > > > > that's
> > > > > > > > > the
> > > > > > > > > >> best
> > > > > > > > > >> > > > > approach.
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > 2. Yes, refactoring existing requests is a
> > non-trivial
> > > > > > > amount of
> > > > > > > > > >> work.
> > > > > > > > > >> > > I
> > > > > > > > > >> > > > > posted some comments in KAFKA-1927. We will
> > probably
> > > > > have
> > > > > > > to fix
> > > > > > > > > >> > > > KAFKA-1927
> > > > > > > > > >> > > > > first, before adding the new logic in
> KAFKA-1694.
> > > > > > > Otherwise, the
> > > > > > > > > >> > > changes
> > > > > > > > > >> > > > > will be too big.
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > > >> > > > > 102.1. It's a bit weird that we return exception
> > in
> > > > the
> > > > > > > api. It
> > > > > > > > > >> seems
> > > > > > > > > >> > > > that
> > > > > > > > > >> > > > > we should either return error code or throw an
> > > > exception
> > > > > > > when
> > > > > > > > > >> getting
> > > > > > > > > >> > > the
> > > > > > > > > >> > > > > response state.
> > > > > > > > > >> > > > > 102.2. We probably shouldn't explicitly use the
> > > > request
> > > > > > > object
> > > > > > > > > in
> > > > > > > > > >> the
> > > > > > > > > >> > > > api.
> > > > > > > > > >> > > > > Not every request evolution requires an api
> > change.
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > Thanks,
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > Jun
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii
> Biletskyi
> > <
> > > > > > > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > > > > Jun,
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > Thanks for you comments. Answers inline:
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > 100. There are a few fields such as
> > > > ReplicaAssignment,
> > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> represented
> > as a
> > > > > > > string,
> > > > > > > > > but
> > > > > > > > > >> > > > contain
> > > > > > > > > >> > > > > > > composite structures in json. Could we
> flatten
> > > > them
> > > > > > out
> > > > > > > > > >> directly in
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > Yes, now with Admin Client this looks a bit
> > weird.
> > > > My
> > > > > > > initial
> > > > > > > > > >> > > > motivation
> > > > > > > > > >> > > > > > was:
> > > > > > > > > >> > > > > > ReassignPartitionCommand accepts input in
> json,
> > we
> > > > > want
> > > > > > to
> > > > > > > > > >> remain
> > > > > > > > > >> > > > tools'
> > > > > > > > > >> > > > > > interfaces unchanged, where possible.
> > > > > > > > > >> > > > > > If we port it to deserialized format, in CLI
> > (/tools
> > > > > > > project)
> > > > > > > > > >> we will
> > > > > > > > > >> > > > > have
> > > > > > > > > >> > > > > > to add some
> > > > > > > > > >> > > > > > json library since /tools is written in java
> and
> > > > we'll
> > > > > > > need to
> > > > > > > > > >> > > > > deserialize
> > > > > > > > > >> > > > > > json file
> > > > > > > > > >> > > > > > provided by a user. Can we quickly agree on
> what
> > > > this
> > > > > > > library
> > > > > > > > > >> should
> > > > > > > > > >> > > be
> > > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > 101. Does TopicMetadataRequest v1 still
> trigger
> > auto
> > > > > > topic
> > > > > > > > > >> creation?
> > > > > > > > > >> > > > This
> > > > > > > > > >> > > > > > > will be a bit weird now that we have a
> > separate
> > > > > topic
> > > > > > > > > >> creation api.
> > > > > > > > > >> > > > > Have
> > > > > > > > > >> > > > > > > you thought about how the new
> > createTopicRequest
> > > > and
> > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > >> > > > > > > v1 will be used in the producer/consumer
> > client,
> > > > in
> > > > > > > addition
> > > > > > > > > >> to
> > > > > > > > > >> > > admin
> > > > > > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > >> > > from
> > > > > > > > > >> > > > > the
> > > > > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > I agree, this strange logic should be fixed.
> > I'm not
> > > > > > > confident
> > > > > > > > > >> in
> > > > > > > > > >> > > this
> > > > > > > > > >> > > > > > Kafka part so
> > > > > > > > > >> > > > > > correct me if I'm wrong, but it doesn't look
> > like a
> > > > > hard
> > > > > > > thing
> > > > > > > > > >> to
> > > > > > > > > >> > > do, I
> > > > > > > > > >> > > > > > think we can
> > > > > > > > > >> > > > > > leverage AdminClient for that in Producer and
> > > > > > > unconditionally
> > > > > > > > > >> remove
> > > > > > > > > >> > > > > topic
> > > > > > > > > >> > > > > > creation from the TopicMetadataRequest_V1.
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > 2. I think Jay meant getting rid of scala
> > classes
> > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > >> > > did
> > > > > > > > > >> > > > > > that
> > > > > > > > > >> > > > > > > as a stop-gap thing when adding the new
> > requests
> > > > for
> > > > > > the
> > > > > > > > > >> consumers.
> > > > > > > > > >> > > > > > > However, the long term plan is to get rid of
> > all
> > > > > those
> > > > > > > and
> > > > > > > > > >> just
> > > > > > > > > >> > > reuse
> > > > > > > > > >> > > > > the
> > > > > > > > > >> > > > > > > java request/response in the client. Since
> > this
> > > > KIP
> > > > > > > proposes
> > > > > > > > > >> to
> > > > > > > > > >> > > add a
> > > > > > > > > >> > > > > > > significant number of new requests, perhaps
> we
> > > > > should
> > > > > > > bite
> > > > > > > > > the
> > > > > > > > > >> > > bullet
> > > > > > > > > >> > > > > to
> > > > > > > > > >> > > > > > > clean up the existing scala requests first
> > before
> > > > > > > adding new
> > > > > > > > > >> ones?
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > Yes, looks like I misunderstood the point of
> > > > > > > > > >> ...RequestAndHeader.
> > > > > > > > > >> > > > Okay, I
> > > > > > > > > >> > > > > > will
> > > > > > > > > >> > > > > > rework that. The only thing is that I don't
> see
> > any
> > > > > > > example
> > > > > > > > > how
> > > > > > > > > >> it
> > > > > > > > > >> > > was
> > > > > > > > > >> > > > > done
> > > > > > > > > >> > > > > > for at
> > > > > > > > > >> > > > > > least one existing protocol message. Thus, as
> I
> > > > > > > understand, I
> > > > > > > > > >> have to
> > > > > > > > > >> > > > > think
> > > > > > > > > >> > > > > > how we
> > > > > > > > > >> > > > > > are going to do it.
> > > > > > > > > >> > > > > > Re porting all existing RQ/RP in this patch.
> > Sounds
> > > > > > > > > reasonable,
> > > > > > > > > >> but
> > > > > > > > > >> > > if
> > > > > > > > > >> > > > > it's
> > > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > > >> > > > > > requirement to have Admin KIP done, I'm afraid
> > this
> > > > > can
> > > > > > > be a
> > > > > > > > > >> serious
> > > > > > > > > >> > > > > > blocker for us.
> > > > > > > > > >> > > > > > There are 13 protocol messages and all that
> > would
> > > > > > require
> > > > > > > not
> > > > > > > > > >> only
> > > > > > > > > >> > > unit
> > > > > > > > > >> > > > > > tests but quite
> > > > > > > > > >> > > > > > intensive manual testing, no? I'm afraid I'm
> > not the
> > > > > > > right guy
> > > > > > > > > >> to
> > > > > > > > > >> > > cover
> > > > > > > > > >> > > > > > pretty much all
> > > > > > > > > >> > > > > > Kafka core internals :). Let me know your
> > thoughts
> > > > on
> > > > > > this
> > > > > > > > > >> item. Btw
> > > > > > > > > >> > > > > there
> > > > > > > > > >> > > > > > is a ticket to
> > > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > > >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > > >> > > > ).
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > Thanks,
> > > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <
> > > > > > > jun@confluent.io>
> > > > > > > > > >> wrote:
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > > > > Andrii,
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > 100. There are a few fields such as
> > > > > ReplicaAssignment,
> > > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > > >> > > > > > > and PartitionsSerialized that are
> represented
> > as a
> > > > > > > string,
> > > > > > > > > but
> > > > > > > > > >> > > > contain
> > > > > > > > > >> > > > > > > composite structures in json. Could we
> flatten
> > > > them
> > > > > > out
> > > > > > > > > >> directly in
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > 101. Does TopicMetadataRequest v1 still
> > trigger
> > > > auto
> > > > > > > topic
> > > > > > > > > >> > > creation?
> > > > > > > > > >> > > > > This
> > > > > > > > > >> > > > > > > will be a bit weird now that we have a
> > separate
> > > > > topic
> > > > > > > > > >> creation api.
> > > > > > > > > >> > > > > Have
> > > > > > > > > >> > > > > > > you thought about how the new
> > createTopicRequest
> > > > and
> > > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > > >> > > > > > > v1 will be used in the producer/consumer
> > client,
> > > > in
> > > > > > > addition
> > > > > > > > > >> to
> > > > > > > > > >> > > admin
> > > > > > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > >> > > from
> > > > > > > > > >> > > > > the
> > > > > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > 2. I think Jay meant getting rid of scala
> > classes
> > > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > > >> > > did
> > > > > > > > > >> > > > > > that
> > > > > > > > > >> > > > > > > as a stop-gap thing when adding the new
> > requests
> > > > for
> > > > > > the
> > > > > > > > > >> consumers.
> > > > > > > > > >> > > > > > > However, the long term plan is to get rid of
> > all
> > > > > those
> > > > > > > and
> > > > > > > > > >> just
> > > > > > > > > >> > > reuse
> > > > > > > > > >> > > > > the
> > > > > > > > > >> > > > > > > java request/response in the client. Since
> > this
> > > > KIP
> > > > > > > proposes
> > > > > > > > > >> to
> > > > > > > > > >> > > add a
> > > > > > > > > >> > > > > > > significant number of new requests, perhaps
> we
> > > > > should
> > > > > > > bite
> > > > > > > > > the
> > > > > > > > > >> > > bullet
> > > > > > > > > >> > > > > to
> > > > > > > > > >> > > > > > > clean up the existing scala requests first
> > before
> > > > > > > adding new
> > > > > > > > > >> ones?
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > Thanks,
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > Jun
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii
> > Biletskyi
> > > > <
> > > > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > > Hi,
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > As said above - I list again all comments
> > from
> > > > > this
> > > > > > > thread
> > > > > > > > > >> so we
> > > > > > > > > >> > > > > > > > can see what's left and finalize all
> pending
> > > > > issues.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > > >> > > > > > > > 1. This is much needed functionality, but
> > there
> > > > > are
> > > > > > a
> > > > > > > lot
> > > > > > > > > >> of the
> > > > > > > > > >> > > so
> > > > > > > > > >> > > > > > let's
> > > > > > > > > >> > > > > > > > really think these protocols through. We
> > really
> > > > > want
> > > > > > > to
> > > > > > > > > end
> > > > > > > > > >> up
> > > > > > > > > >> > > > with a
> > > > > > > > > >> > > > > > set
> > > > > > > > > >> > > > > > > > of well thought-out, orthoganol apis. For
> > this
> > > > > > reason
> > > > > > > I
> > > > > > > > > >> think it
> > > > > > > > > >> > > is
> > > > > > > > > >> > > > > > > really
> > > > > > > > > >> > > > > > > > important to think through the end state
> > even if
> > > > > > that
> > > > > > > > > >> includes
> > > > > > > > > >> > > APIs
> > > > > > > > > >> > > > > we
> > > > > > > > > >> > > > > > > > won't implement in the first phase.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: Definitely behind this. Would
> appreciate
> > if
> > > > > there
> > > > > > > are
> > > > > > > > > >> concrete
> > > > > > > > > >> > > > > > > comments
> > > > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 2. Let's please please please wait until
> we
> > have
> > > > > > > switched
> > > > > > > > > >> the
> > > > > > > > > >> > > > server
> > > > > > > > > >> > > > > > over
> > > > > > > > > >> > > > > > > > to the new java protocol definitions. If
> we
> > add
> > > > > > upteen
> > > > > > > > > more
> > > > > > > > > >> ad
> > > > > > > > > >> > > hoc
> > > > > > > > > >> > > > > > scala
> > > > > > > > > >> > > > > > > > objects that is just generating more work
> > for
> > > > the
> > > > > > > > > >> conversion we
> > > > > > > > > >> > > > know
> > > > > > > > > >> > > > > we
> > > > > > > > > >> > > > > > > > have to do.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed
> scala
> > > > > > protocol
> > > > > > > > > >> classes.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 3. This proposal introduces a new type of
> > > > optional
> > > > > > > > > >> parameter.
> > > > > > > > > >> > > This
> > > > > > > > > >> > > > is
> > > > > > > > > >> > > > > > > > inconsistent with everything else in the
> > > > protocol
> > > > > > > where we
> > > > > > > > > >> use -1
> > > > > > > > > >> > > > or
> > > > > > > > > >> > > > > > some
> > > > > > > > > >> > > > > > > > other marker value. You could argue either
> > way
> > > > but
> > > > > > > let's
> > > > > > > > > >> stick
> > > > > > > > > >> > > with
> > > > > > > > > >> > > > > > that
> > > > > > > > > >> > > > > > > > for consistency. For clients that
> > implemented
> > > > the
> > > > > > > protocol
> > > > > > > > > >> in a
> > > > > > > > > >> > > > > better
> > > > > > > > > >> > > > > > > way
> > > > > > > > > >> > > > > > > > than our scala code these basic primitives
> > are
> > > > > hard
> > > > > > to
> > > > > > > > > >> change.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed
> > MaybeOf
> > > > > type
> > > > > > > and
> > > > > > > > > >> changed
> > > > > > > > > >> > > > > > protocol
> > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 4. ClusterMetadata: This seems to
> duplicate
> > > > > > > > > >> TopicMetadataRequest
> > > > > > > > > >> > > > > which
> > > > > > > > > >> > > > > > > has
> > > > > > > > > >> > > > > > > > brokers, topics, and partitions. I think
> we
> > > > should
> > > > > > > rename
> > > > > > > > > >> that
> > > > > > > > > >> > > > > request
> > > > > > > > > >> > > > > > > > ClusterMetadataRequest (or just
> > MetadataRequest)
> > > > > and
> > > > > > > > > >> include the
> > > > > > > > > >> > > id
> > > > > > > > > >> > > > > of
> > > > > > > > > >> > > > > > > the
> > > > > > > > > >> > > > > > > > controller. Or are there other things we
> > could
> > > > add
> > > > > > > here?
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: I agree. Updated the KIP. Let's extends
> > > > > > > TopicMetadata
> > > > > > > > > to
> > > > > > > > > >> > > > version 2
> > > > > > > > > >> > > > > > and
> > > > > > > > > >> > > > > > > > include controller.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 5. We have a tendency to try to make a lot
> > of
> > > > > > requests
> > > > > > > > > that
> > > > > > > > > >> can
> > > > > > > > > >> > > > only
> > > > > > > > > >> > > > > go
> > > > > > > > > >> > > > > > > to
> > > > > > > > > >> > > > > > > > particular nodes. This adds a lot of
> burden
> > for
> > > > > > client
> > > > > > > > > >> > > > > implementations
> > > > > > > > > >> > > > > > > (it
> > > > > > > > > >> > > > > > > > sounds easy but each discovery can fail in
> > many
> > > > > > parts
> > > > > > > so
> > > > > > > > > it
> > > > > > > > > >> ends
> > > > > > > > > >> > > up
> > > > > > > > > >> > > > > > > being a
> > > > > > > > > >> > > > > > > > full state machine to do right). I think
> we
> > > > should
> > > > > > > > > consider
> > > > > > > > > >> > > making
> > > > > > > > > >> > > > > > admin
> > > > > > > > > >> > > > > > > > commands and ideally as many of the other
> > apis
> > > > as
> > > > > > > possible
> > > > > > > > > >> > > > available
> > > > > > > > > >> > > > > on
> > > > > > > > > >> > > > > > > all
> > > > > > > > > >> > > > > > > > brokers and just redirect to the
> controller
> > on
> > > > the
> > > > > > > broker
> > > > > > > > > >> side.
> > > > > > > > > >> > > > > Perhaps
> > > > > > > > > >> > > > > > > > there would be a general way to
> encapsulate
> > this
> > > > > > > > > re-routing
> > > > > > > > > >> > > > behavior.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: It's a very interesting idea, but seems
> > there
> > > > > are
> > > > > > > some
> > > > > > > > > >> > > concerns
> > > > > > > > > >> > > > > > about
> > > > > > > > > >> > > > > > > > this
> > > > > > > > > >> > > > > > > > feature (like performance considerations,
> > how
> > > > this
> > > > > > > will
> > > > > > > > > >> > > complicate
> > > > > > > > > >> > > > > > server
> > > > > > > > > >> > > > > > > > etc).
> > > > > > > > > >> > > > > > > > I believe this shouldn't be a blocker. If
> > this
> > > > > > > feature is
> > > > > > > > > >> > > > implemented
> > > > > > > > > >> > > > > > at
> > > > > > > > > >> > > > > > > > some
> > > > > > > > > >> > > > > > > > point it won't affect Admin changes - at
> > least
> > > > no
> > > > > > > changes
> > > > > > > > > to
> > > > > > > > > >> > > public
> > > > > > > > > >> > > > > API
> > > > > > > > > >> > > > > > > > will be required.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 6. We should probably normalize the key
> > value
> > > > > pairs
> > > > > > > used
> > > > > > > > > for
> > > > > > > > > >> > > > configs
> > > > > > > > > >> > > > > > > rather
> > > > > > > > > >> > > > > > > > than embedding a new formatting. So two
> > strings
> > > > > > rather
> > > > > > > > > than
> > > > > > > > > >> one
> > > > > > > > > >> > > > with
> > > > > > > > > >> > > > > an
> > > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: Fixed in the latest patch - normalized
> > > > configs
> > > > > > and
> > > > > > > > > >> changed
> > > > > > > > > >> > > > > protocol
> > > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 7. Is the postcondition of these APIs that
> > the
> > > > > > > command has
> > > > > > > > > >> begun
> > > > > > > > > >> > > or
> > > > > > > > > >> > > > > > that
> > > > > > > > > >> > > > > > > > the command has been completed? It is a
> lot
> > more
> > > > > > > usable if
> > > > > > > > > >> the
> > > > > > > > > >> > > > > command
> > > > > > > > > >> > > > > > > has
> > > > > > > > > >> > > > > > > > been completed so you know that if you
> > create a
> > > > > > topic
> > > > > > > and
> > > > > > > > > >> then
> > > > > > > > > >> > > > > publish
> > > > > > > > > >> > > > > > to
> > > > > > > > > >> > > > > > > > it you won't get an exception about there
> > being
> > > > no
> > > > > > > such
> > > > > > > > > >> topic.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: For long running requests (like
> reassign
> > > > > > > partitions) -
> > > > > > > > > >> the
> > > > > > > > > >> > > post
> > > > > > > > > >> > > > > > > > condition is
> > > > > > > > > >> > > > > > > > command has begun - so we don't block the
> > > > client.
> > > > > In
> > > > > > > case
> > > > > > > > > >> of your
> > > > > > > > > >> > > > > > > example -
> > > > > > > > > >> > > > > > > > topic commands, this will be refactored
> and
> > > > topic
> > > > > > > commands
> > > > > > > > > >> will
> > > > > > > > > >> > > be
> > > > > > > > > >> > > > > > > executed
> > > > > > > > > >> > > > > > > > immediately, since the Controller will
> serve
> > > > Admin
> > > > > > > > > requests
> > > > > > > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 8. Describe topic and list topics
> duplicate
> > a
> > > > lot
> > > > > of
> > > > > > > stuff
> > > > > > > > > >> in the
> > > > > > > > > >> > > > > > > metadata
> > > > > > > > > >> > > > > > > > request. Is there a reason to give back
> > topics
> > > > > > marked
> > > > > > > for
> > > > > > > > > >> > > > deletion? I
> > > > > > > > > >> > > > > > > feel
> > > > > > > > > >> > > > > > > > like if we just make the post-condition of
> > the
> > > > > > delete
> > > > > > > > > >> command be
> > > > > > > > > >> > > > that
> > > > > > > > > >> > > > > > the
> > > > > > > > > >> > > > > > > > topic is deleted that will get rid of the
> > need
> > > > for
> > > > > > > this
> > > > > > > > > >> right?
> > > > > > > > > >> > > And
> > > > > > > > > >> > > > it
> > > > > > > > > >> > > > > > > will
> > > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed
> > topics
> > > > > marked
> > > > > > > for
> > > > > > > > > >> deletion
> > > > > > > > > >> > > > in
> > > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 9. Should we consider batching these
> > requests?
> > > > We
> > > > > > have
> > > > > > > > > >> generally
> > > > > > > > > >> > > > > tried
> > > > > > > > > >> > > > > > to
> > > > > > > > > >> > > > > > > > allow multiple operations to be batched.
> My
> > > > > > suspicion
> > > > > > > is
> > > > > > > > > >> that
> > > > > > > > > >> > > > without
> > > > > > > > > >> > > > > > > this
> > > > > > > > > >> > > > > > > > we will get a lot of code that does
> > something
> > > > like
> > > > > > > > > >> > > > > > > >    for(topic: adminClient.listTopics())
> > > > > > > > > >> > > > > > > >       adminClient.describeTopic(topic)
> > > > > > > > > >> > > > > > > > this code will work great when you test
> on 5
> > > > > topics
> > > > > > > but
> > > > > > > > > not
> > > > > > > > > >> do as
> > > > > > > > > >> > > > > well
> > > > > > > > > >> > > > > > if
> > > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: Updated the KIP - please check "Topic
> > Admin
> > > > > > Schema"
> > > > > > > > > >> section.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 10. I think we should also discuss how we
> > want
> > > > to
> > > > > > > expose a
> > > > > > > > > >> > > > > programmatic
> > > > > > > > > >> > > > > > > JVM
> > > > > > > > > >> > > > > > > > client api for these operations. Currently
> > > > people
> > > > > > > rely on
> > > > > > > > > >> > > > AdminUtils
> > > > > > > > > >> > > > > > > which
> > > > > > > > > >> > > > > > > > is totally sketchy. I think we probably
> need
> > > > > another
> > > > > > > > > client
> > > > > > > > > >> under
> > > > > > > > > >> > > > > > > clients/
> > > > > > > > > >> > > > > > > > that exposes administrative functionality.
> > We
> > > > will
> > > > > > > need
> > > > > > > > > >> this just
> > > > > > > > > >> > > > to
> > > > > > > > > >> > > > > > > > properly test the new apis, I suspect. We
> > should
> > > > > > > figure
> > > > > > > > > out
> > > > > > > > > >> that
> > > > > > > > > >> > > > API.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: Updated the KIP - please check "Admin
> > Client"
> > > > > > > section
> > > > > > > > > >> with an
> > > > > > > > > >> > > > > > initial
> > > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 11. The other information that would be
> > really
> > > > > > useful
> > > > > > > to
> > > > > > > > > get
> > > > > > > > > >> > > would
> > > > > > > > > >> > > > be
> > > > > > > > > >> > > > > > > > information about partitions--how much
> data
> > is
> > > > in
> > > > > > the
> > > > > > > > > >> partition,
> > > > > > > > > >> > > > what
> > > > > > > > > >> > > > > > are
> > > > > > > > > >> > > > > > > > the segment offsets, what is the log-end
> > offset
> > > > > > (i.e.
> > > > > > > last
> > > > > > > > > >> > > offset),
> > > > > > > > > >> > > > > > what
> > > > > > > > > >> > > > > > > is
> > > > > > > > > >> > > > > > > > the compaction point, etc. I think that
> done
> > > > right
> > > > > > > this
> > > > > > > > > >> would be
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > > > > > successor to the very awkward
> OffsetRequest
> > we
> > > > > have
> > > > > > > today.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest
> in
> > the
> > > > > > latest
> > > > > > > > > >> patch. I
> > > > > > > > > >> > > > > believe
> > > > > > > > > >> > > > > > > > this should
> > > > > > > > > >> > > > > > > > be resolved in a separate KIP / jira
> ticket.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 12. Generally we can do good error
> handling
> > > > > without
> > > > > > > > > needing
> > > > > > > > > >> > > custom
> > > > > > > > > >> > > > > > > > server-side
> > > > > > > > > >> > > > > > > > messages. I.e. generally the client has
> the
> > > > > context
> > > > > > to
> > > > > > > > > know
> > > > > > > > > >> that
> > > > > > > > > >> > > if
> > > > > > > > > >> > > > > it
> > > > > > > > > >> > > > > > > got
> > > > > > > > > >> > > > > > > > an error that the topic doesn't exist to
> say
> > > > > "Topic
> > > > > > X
> > > > > > > > > >> doesn't
> > > > > > > > > >> > > > exist"
> > > > > > > > > >> > > > > > > rather
> > > > > > > > > >> > > > > > > > than "error code 14" (or whatever). Maybe
> > there
> > > > > are
> > > > > > > > > specific
> > > > > > > > > >> > > cases
> > > > > > > > > >> > > > > > where
> > > > > > > > > >> > > > > > > > this is hard? If we want to add
> server-side
> > > > error
> > > > > > > messages
> > > > > > > > > >> we
> > > > > > > > > >> > > > really
> > > > > > > > > >> > > > > do
> > > > > > > > > >> > > > > > > > need to do this in a consistent way across
> > the
> > > > > > > protocol.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: Updated the KIP - please check
> "Protocol
> > > > > Errors"
> > > > > > > > > >> section. I
> > > > > > > > > >> > > > added
> > > > > > > > > >> > > > > > the
> > > > > > > > > >> > > > > > > > comprehensive, fine-grained list of error
> > codes.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > > >> > > > > > > > 13. Describe topic request: it would be
> > great to
> > > > > go
> > > > > > > beyond
> > > > > > > > > >> just
> > > > > > > > > >> > > > > > batching
> > > > > > > > > >> > > > > > > on
> > > > > > > > > >> > > > > > > > topic name regex for this request. For
> > example,
> > > > a
> > > > > > very
> > > > > > > > > >> common use
> > > > > > > > > >> > > > > case
> > > > > > > > > >> > > > > > of
> > > > > > > > > >> > > > > > > > the topic command is to list all topics
> > whose
> > > > > config
> > > > > > > A's
> > > > > > > > > >> value is
> > > > > > > > > >> > > > B.
> > > > > > > > > >> > > > > > With
> > > > > > > > > >> > > > > > > > topic name regex then we have to first
> > retrieve
> > > > > > > __all__
> > > > > > > > > >> topics's
> > > > > > > > > >> > > > > > > > description info and then filter at the
> > client
> > > > > end,
> > > > > > > which
> > > > > > > > > >> will
> > > > > > > > > >> > > be a
> > > > > > > > > >> > > > > > huge
> > > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > > >> > > > > > > > AND
> > > > > > > > > >> > > > > > > > 14. Config K-Vs in create topic: this is
> > related
> > > > > to
> > > > > > > the
> > > > > > > > > >> previous
> > > > > > > > > >> > > > > point;
> > > > > > > > > >> > > > > > > > maybe we can add another metadata K-V or
> > just a
> > > > > > > metadata
> > > > > > > > > >> string
> > > > > > > > > >> > > > along
> > > > > > > > > >> > > > > > > side
> > > > > > > > > >> > > > > > > > with config K-V in create topic like we
> did
> > for
> > > > > > offset
> > > > > > > > > >> commit
> > > > > > > > > >> > > > > request.
> > > > > > > > > >> > > > > > > This
> > > > > > > > > >> > > > > > > > field can be quite useful in storing
> > information
> > > > > > like
> > > > > > > > > >> "owner" of
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > > > > topic
> > > > > > > > > >> > > > > > > > who issue the create command, etc, which
> is
> > > > quite
> > > > > > > > > important
> > > > > > > > > >> for a
> > > > > > > > > >> > > > > > > > multi-tenant setting. Then in the describe
> > topic
> > > > > > > request
> > > > > > > > > we
> > > > > > > > > >> can
> > > > > > > > > >> > > > also
> > > > > > > > > >> > > > > > > batch
> > > > > > > > > >> > > > > > > > on regex of the metadata field.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: As discussed it is very interesting but
> > can
> > > > be
> > > > > > > > > >> implemented
> > > > > > > > > >> > > later
> > > > > > > > > >> > > > > > after
> > > > > > > > > >> > > > > > > > we have some basic functionality there.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > 15. Today all the admin operations are
> > async in
> > > > > the
> > > > > > > sense
> > > > > > > > > >> that
> > > > > > > > > >> > > > > command
> > > > > > > > > >> > > > > > > will
> > > > > > > > > >> > > > > > > > return once it is written in ZK, and that
> > is why
> > > > > we
> > > > > > > need
> > > > > > > > > >> extra
> > > > > > > > > >> > > > > > > verification
> > > > > > > > > >> > > > > > > > like testUtil.waitForTopicCreated() /
> verify
> > > > > > partition
> > > > > > > > > >> > > reassignment
> > > > > > > > > >> > > > > > > > request, etc. With admin requests we could
> > add a
> > > > > > flag
> > > > > > > to
> > > > > > > > > >> enable /
> > > > > > > > > >> > > > > > disable
> > > > > > > > > >> > > > > > > > synchronous requests; when it is turned
> on,
> > the
> > > > > > > response
> > > > > > > > > >> will not
> > > > > > > > > >> > > > > > return
> > > > > > > > > >> > > > > > > > until the request has been completed. And
> > for
> > > > > async
> > > > > > > > > >> requests we
> > > > > > > > > >> > > can
> > > > > > > > > >> > > > > > add a
> > > > > > > > > >> > > > > > > > "token" field in the response, and then
> only
> > > > need
> > > > > a
> > > > > > > > > general
> > > > > > > > > >> > > "admin
> > > > > > > > > >> > > > > > > > verification request" with the given token
> > to
> > > > > check
> > > > > > > if the
> > > > > > > > > >> async
> > > > > > > > > >> > > > > > request
> > > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: I see your point. My idea was to
> provide
> > > > > specific
> > > > > > > > > >> > > > Verify...Request
> > > > > > > > > >> > > > > > per
> > > > > > > > > >> > > > > > > > each
> > > > > > > > > >> > > > > > > > long running request, where needed. We can
> > do it
> > > > > the
> > > > > > > way
> > > > > > > > > you
> > > > > > > > > >> > > > suggest.
> > > > > > > > > >> > > > > > The
> > > > > > > > > >> > > > > > > > only
> > > > > > > > > >> > > > > > > > concern is that introducing a token we
> again
> > > > will
> > > > > > make
> > > > > > > > > >> schema
> > > > > > > > > >> > > > > > "dynamic".
> > > > > > > > > >> > > > > > > We
> > > > > > > > > >> > > > > > > > wanted
> > > > > > > > > >> > > > > > > > to do similar thing introducing single
> > > > > AdminRequest
> > > > > > > for
> > > > > > > > > all
> > > > > > > > > >> topic
> > > > > > > > > >> > > > > > > commands
> > > > > > > > > >> > > > > > > > but rejected
> > > > > > > > > >> > > > > > > > this idea because we wanted to have schema
> > > > > defined.
> > > > > > So
> > > > > > > > > this
> > > > > > > > > >> is
> > > > > > > > > >> > > > more a
> > > > > > > > > >> > > > > > > > choice between:
> > > > > > > > > >> > > > > > > > a) have fixed schema but introduce each
> > time new
> > > > > > > > > >> Verify...Request
> > > > > > > > > >> > > > for
> > > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > > >> > > > > > > > b) use one request for verification but
> > > > generalize
> > > > > > it
> > > > > > > with
> > > > > > > > > >> token
> > > > > > > > > >> > > > > > > > I'm fine with whatever decision community
> > come
> > > > to.
> > > > > > > Just
> > > > > > > > > let
> > > > > > > > > >> me
> > > > > > > > > >> > > know
> > > > > > > > > >> > > > > > your
> > > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > > >> > > > > > > > 16. Specifically for ownership, I think
> the
> > plan
> > > > > is
> > > > > > > to add
> > > > > > > > > >> ACL
> > > > > > > > > >> > > (it
> > > > > > > > > >> > > > > > sounds
> > > > > > > > > >> > > > > > > > like you are describing ACL) via an
> external
> > > > > system
> > > > > > > > > (Argus,
> > > > > > > > > >> > > > Sentry).
> > > > > > > > > >> > > > > > > > I remember KIP-11 described this, but I
> > can't
> > > > find
> > > > > > > the KIP
> > > > > > > > > >> any
> > > > > > > > > >> > > > > longer.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > A: Okay, no problem. Not sure though how
> we
> > are
> > > > > > going
> > > > > > > to
> > > > > > > > > >> handle
> > > > > > > > > >> > > it.
> > > > > > > > > >> > > > > > Wait
> > > > > > > > > >> > > > > > > > which KIP
> > > > > > > > > >> > > > > > > > will be committed first and include
> changes
> > to
> > > > > > > > > >> TopicMetadata from
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > > > > later
> > > > > > > > > >> > > > > > > > one?
> > > > > > > > > >> > > > > > > > Anyway, I added this note to "Open
> > Questions"
> > > > > > section
> > > > > > > so
> > > > > > > > > we
> > > > > > > > > >> don't
> > > > > > > > > >> > > > > miss
> > > > > > > > > >> > > > > > > this
> > > > > > > > > >> > > > > > > > piece.
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii
> > > > > Biletskyi <
> > > > > > > > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > > Today I uploaded the patch that covers
> > some of
> > > > > the
> > > > > > > > > >> discussed
> > > > > > > > > >> > > and
> > > > > > > > > >> > > > > > agreed
> > > > > > > > > >> > > > > > > > > items:
> > > > > > > > > >> > > > > > > > > - removed MaybeOf optional type
> > > > > > > > > >> > > > > > > > > - switched to java protocol definitions
> > > > > > > > > >> > > > > > > > > - simplified messages (normalized
> configs,
> > > > > removed
> > > > > > > topic
> > > > > > > > > >> marked
> > > > > > > > > >> > > > for
> > > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > > I also updated the KIP-4 with respective
> > > > changes
> > > > > > and
> > > > > > > > > >> wrote down
> > > > > > > > > >> > > > my
> > > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > > >> > > > > > > > > - Batch Admin Operations -> updated Wire
> > > > > Protocol
> > > > > > > schema
> > > > > > > > > >> > > proposal
> > > > > > > > > >> > > > > > > > > - Remove ClusterMetadata -> changed to
> > extend
> > > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > > >> > > > > > > > > - Admin Client -> updated my initial
> > proposal
> > > > to
> > > > > > > reflect
> > > > > > > > > >> > > batching
> > > > > > > > > >> > > > > > > > > - Error codes -> proposed fine-grained
> > error
> > > > > code
> > > > > > > > > instead
> > > > > > > > > >> of
> > > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > > I will also send a separate email to
> > cover all
> > > > > > > comments
> > > > > > > > > >> from
> > > > > > > > > >> > > this
> > > > > > > > > >> > > > > > > thread.
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen
> > Shapira
> > > > <
> > > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > > >> > > > > > > > >
> > > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > > >> > > > > > > > >>
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > >
> > > > > > > > > >>
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > > >> > > > > > > > >> )
> > > > > > > > > >> > > > > > > > >> It actually specifies changes to the
> > Metadata
> > > > > > > protocol,
> > > > > > > > > >> so
> > > > > > > > > >> > > > making
> > > > > > > > > >> > > > > > sure
> > > > > > > > > >> > > > > > > > >> both KIPs are consistent in this regard
> > will
> > > > be
> > > > > > > good.
> > > > > > > > > >> > > > > > > > >>
> > > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen
> > > > Shapira
> > > > > <
> > > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > >> > > > > > > > >> > Specifically for ownership, I think
> the
> > > > plan
> > > > > is
> > > > > > > to
> > > > > > > > > add
> > > > > > > > > >> ACL
> > > > > > > > > >> > > (it
> > > > > > > > > >> > > > > > > sounds
> > > > > > > > > >> > > > > > > > >> > like you are describing ACL) via an
> > > > external
> > > > > > > system
> > > > > > > > > >> (Argus,
> > > > > > > > > >> > > > > > Sentry).
> > > > > > > > > >> > > > > > > > >> > I remember KIP-11 described this,
> but I
> > > > can't
> > > > > > > find
> > > > > > > > > the
> > > > > > > > > >> KIP
> > > > > > > > > >> > > any
> > > > > > > > > >> > > > > > > longer.
> > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > >> > > > > > > > >> > Regardless, I think KIP-4 focuses on
> > > > getting
> > > > > > > > > >> information
> > > > > > > > > >> > > that
> > > > > > > > > >> > > > > > > already
> > > > > > > > > >> > > > > > > > >> > exists from Kafka brokers, not on
> > adding
> > > > > > > information
> > > > > > > > > >> that
> > > > > > > > > >> > > > > perhaps
> > > > > > > > > >> > > > > > > > >> > should exist but doesn't yet?
> > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > >> > > > > > > > >> >
> > > > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM,
> > Guozhang
> > > > > Wang
> > > > > > <
> > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > >> > > > > > > > >> >> Just want to elaborate a bit more on
> > the
> > > > > > > > > create-topic
> > > > > > > > > >> > > > metadata
> > > > > > > > > >> > > > > > and
> > > > > > > > > >> > > > > > > > >> batching
> > > > > > > > > >> > > > > > > > >> >> describe-topic based on config /
> > metadata
> > > > in
> > > > > > my
> > > > > > > > > >> previous
> > > > > > > > > >> > > > email
> > > > > > > > > >> > > > > as
> > > > > > > > > >> > > > > > > we
> > > > > > > > > >> > > > > > > > >> work
> > > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The main motivation
> is
> > to
> > > > > have
> > > > > > > some
> > > > > > > > > >> sort of
> > > > > > > > > >> > > > > topic
> > > > > > > > > >> > > > > > > > >> management
> > > > > > > > > >> > > > > > > > >> >> mechanisms, which I think is quite
> > > > important
> > > > > > in
> > > > > > > a
> > > > > > > > > >> > > > multi-tenant
> > > > > > > > > >> > > > > /
> > > > > > > > > >> > > > > > > > cloud
> > > > > > > > > >> > > > > > > > >> >> architecture: today anyone can
> create
> > > > topics
> > > > > > in
> > > > > > > a
> > > > > > > > > >> shared
> > > > > > > > > >> > > > Kafka
> > > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > > >> > > > > > > > >> >> there is no concept or "ownership"
> of
> > > > topics
> > > > > > > that
> > > > > > > > > are
> > > > > > > > > >> > > created
> > > > > > > > > >> > > > > by
> > > > > > > > > >> > > > > > > > >> different
> > > > > > > > > >> > > > > > > > >> >> users. For example, at LinkedIn we
> > > > basically
> > > > > > > > > >> distinguish
> > > > > > > > > >> > > > topic
> > > > > > > > > >> > > > > > > owners
> > > > > > > > > >> > > > > > > > >> via
> > > > > > > > > >> > > > > > > > >> >> some casual topic name prefix, which
> > is a
> > > > > bit
> > > > > > > > > awkward
> > > > > > > > > >> and
> > > > > > > > > >> > > > does
> > > > > > > > > >> > > > > > not
> > > > > > > > > >> > > > > > > > fly
> > > > > > > > > >> > > > > > > > >> as
> > > > > > > > > >> > > > > > > > >> >> we scale our customers. It would be
> > great
> > > > to
> > > > > > use
> > > > > > > > > >> > > > > describe-topics
> > > > > > > > > >> > > > > > > such
> > > > > > > > > >> > > > > > > > >> as:
> > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > >> > > > > > > > >> >> Describe all topics that is created
> > by me.
> > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > >> > > > > > > > >> >> Describe all topics whose retention
> > time
> > > > is
> > > > > > > > > overriden
> > > > > > > > > >> to X.
> > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > >> > > > > > > > >> >> Describe all topics whose writable
> > group
> > > > > > include
> > > > > > > > > user
> > > > > > > > > >> Y
> > > > > > > > > >> > > (this
> > > > > > > > > >> > > > > is
> > > > > > > > > >> > > > > > > > >> related to
> > > > > > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > >> > > > > > > > >> >> One possible way to achieve this is
> to
> > > > add a
> > > > > > > > > metadata
> > > > > > > > > >> file
> > > > > > > > > >> > > in
> > > > > > > > > >> > > > > the
> > > > > > > > > >> > > > > > > > >> >> create-topic request, whose value
> will
> > > > also
> > > > > be
> > > > > > > > > >> written ZK
> > > > > > > > > >> > > as
> > > > > > > > > >> > > > we
> > > > > > > > > >> > > > > > > > create
> > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > >> > > > > > > > >> >> topic; then describe-topics can
> > choose to
> > > > > > batch
> > > > > > > > > topics
> > > > > > > > > >> > > based
> > > > > > > > > >> > > > on
> > > > > > > > > >> > > > > > 1)
> > > > > > > > > >> > > > > > > > name
> > > > > > > > > >> > > > > > > > >> >> regex, 2) config K-V matching, 3)
> > metadata
> > > > > > > regex,
> > > > > > > > > etc.
> > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM,
> > Guozhang
> > > > > Wang
> > > > > > <
> > > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > >> > > > > > > > >> >>
> > > > > > > > > >> > > > > > > > >> >>> Thanks for the updated wiki. A few
> > > > comments
> > > > > > > below:
> > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > >> > > > > > > > >> >>> 1. Error description in response: I
> > think
> > > > > if
> > > > > > > some
> > > > > > > > > >> > > errorCode
> > > > > > > > > >> > > > > > could
> > > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > > >> > > > > > > > >> >>> several different error cases then
> we
> > > > > should
> > > > > > > really
> > > > > > > > > >> change
> > > > > > > > > >> > > > it
> > > > > > > > > >> > > > > to
> > > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > > >> > > > > > > > >> >>> codes. In general the errorCode
> > itself
> > > > > would
> > > > > > be
> > > > > > > > > >> precise
> > > > > > > > > >> > > and
> > > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > > >> > > > > > > > >> >>> describing the server side errors.
> > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > >> > > > > > > > >> >>> 2. Describe topic request: it would
> > be
> > > > > great
> > > > > > > to go
> > > > > > > > > >> beyond
> > > > > > > > > >> > > > just
> > > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > > >> > > > > > > > >> >>> topic name regex for this request.
> > For
> > > > > > > example, a
> > > > > > > > > >> very
> > > > > > > > > >> > > > common
> > > > > > > > > >> > > > > > use
> > > > > > > > > >> > > > > > > > >> case of
> > > > > > > > > >> > > > > > > > >> >>> the topic command is to list all
> > topics
> > > > > whose
> > > > > > > > > config
> > > > > > > > > >> A's
> > > > > > > > > >> > > > value
> > > > > > > > > >> > > > > > is
> > > > > > > > > >> > > > > > > B.
> > > > > > > > > >> > > > > > > > >> With
> > > > > > > > > >> > > > > > > > >> >>> topic name regex then we have to
> > first
> > > > > > retrieve
> > > > > > > > > >> __all__
> > > > > > > > > >> > > > > topics's
> > > > > > > > > >> > > > > > > > >> >>> description info and then filter at
> > the
> > > > > > client
> > > > > > > end,
> > > > > > > > > >> which
> > > > > > > > > >> > > > will
> > > > > > > > > >> > > > > > be
> > > > > > > > > >> > > > > > > a
> > > > > > > > > >> > > > > > > > >> huge
> > > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in create topic:
> this
> > is
> > > > > > > related to
> > > > > > > > > >> the
> > > > > > > > > >> > > > > previous
> > > > > > > > > >> > > > > > > > point;
> > > > > > > > > >> > > > > > > > >> >>> maybe we can add another metadata
> > K-V or
> > > > > > just a
> > > > > > > > > >> metadata
> > > > > > > > > >> > > > > string
> > > > > > > > > >> > > > > > > > along
> > > > > > > > > >> > > > > > > > >> side
> > > > > > > > > >> > > > > > > > >> >>> with config K-V in create topic
> like
> > we
> > > > did
> > > > > > for
> > > > > > > > > >> offset
> > > > > > > > > >> > > > commit
> > > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > > >> > > > > > > > >> >>> field can be quite useful in
> storing
> > > > > > > information
> > > > > > > > > like
> > > > > > > > > >> > > > "owner"
> > > > > > > > > >> > > > > of
> > > > > > > > > >> > > > > > > the
> > > > > > > > > >> > > > > > > > >> topic
> > > > > > > > > >> > > > > > > > >> >>> who issue the create command, etc,
> > which
> > > > is
> > > > > > > quite
> > > > > > > > > >> > > important
> > > > > > > > > >> > > > > for
> > > > > > > > > >> > > > > > a
> > > > > > > > > >> > > > > > > > >> >>> multi-tenant setting. Then in the
> > > > describe
> > > > > > > topic
> > > > > > > > > >> request
> > > > > > > > > >> > > we
> > > > > > > > > >> > > > > can
> > > > > > > > > >> > > > > > > also
> > > > > > > > > >> > > > > > > > >> batch
> > > > > > > > > >> > > > > > > > >> >>> on regex of the metadata field.
> > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > >> > > > > > > > >> >>> 4. Today all the admin operations
> are
> > > > async
> > > > > > in
> > > > > > > the
> > > > > > > > > >> sense
> > > > > > > > > >> > > > that
> > > > > > > > > >> > > > > > > > command
> > > > > > > > > >> > > > > > > > >> will
> > > > > > > > > >> > > > > > > > >> >>> return once it is written in ZK,
> and
> > that
> > > > > is
> > > > > > > why we
> > > > > > > > > >> need
> > > > > > > > > >> > > > extra
> > > > > > > > > >> > > > > > > > >> verification
> > > > > > > > > >> > > > > > > > >> >>> like
> testUtil.waitForTopicCreated() /
> > > > > verify
> > > > > > > > > >> partition
> > > > > > > > > >> > > > > > > reassignment
> > > > > > > > > >> > > > > > > > >> >>> request, etc. With admin requests
> we
> > > > could
> > > > > > add
> > > > > > > a
> > > > > > > > > >> flag to
> > > > > > > > > >> > > > > enable
> > > > > > > > > >> > > > > > /
> > > > > > > > > >> > > > > > > > >> disable
> > > > > > > > > >> > > > > > > > >> >>> synchronous requests; when it is
> > turned
> > > > on,
> > > > > > the
> > > > > > > > > >> response
> > > > > > > > > >> > > > will
> > > > > > > > > >> > > > > > not
> > > > > > > > > >> > > > > > > > >> return
> > > > > > > > > >> > > > > > > > >> >>> until the request has been
> > completed. And
> > > > > for
> > > > > > > async
> > > > > > > > > >> > > requests
> > > > > > > > > >> > > > > we
> > > > > > > > > >> > > > > > > can
> > > > > > > > > >> > > > > > > > >> add a
> > > > > > > > > >> > > > > > > > >> >>> "token" field in the response, and
> > then
> > > > > only
> > > > > > > need a
> > > > > > > > > >> > > general
> > > > > > > > > >> > > > > > "admin
> > > > > > > > > >> > > > > > > > >> >>> verification request" with the
> given
> > > > token
> > > > > to
> > > > > > > check
> > > > > > > > > >> if the
> > > > > > > > > >> > > > > async
> > > > > > > > > >> > > > > > > > >> request
> > > > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > >> > > > > > > > >> >>> 5. +1 for extending Metadata
> request
> > to
> > > > > > include
> > > > > > > > > >> > > controller /
> > > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > > >> > > > > > > > >> >>> information, and then we can remove
> > the
> > > > > > > > > >> ConsumerMetadata /
> > > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM,
> Joel
> > > > > Koshy <
> > > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > > >> > > > > > > > >> >>>> Thanks for sending that out Joe -
> I
> > > > don't
> > > > > > > think I
> > > > > > > > > >> will be
> > > > > > > > > >> > > > > able
> > > > > > > > > >> > > > > > to
> > > > > > > > > >> > > > > > > > >> make
> > > > > > > > > >> > > > > > > > >> >>>> it today, so if notes can be sent
> > out
> > > > > > > afterward
> > > > > > > > > that
> > > > > > > > > >> > > would
> > > > > > > > > >> > > > be
> > > > > > > > > >> > > > > > > > great.
> > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM
> > > > -0800,
> > > > > > Gwen
> > > > > > > > > >> Shapira
> > > > > > > > > >> > > > wrote:
> > > > > > > > > >> > > > > > > > >> >>>> > Thanks for sending this out Joe.
> > > > Looking
> > > > > > > forward
> > > > > > > > > >> to
> > > > > > > > > >> > > > > chatting
> > > > > > > > > >> > > > > > > with
> > > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM,
> > Joe
> > > > > Stein
> > > > > > <
> > > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just sent out a google
> > > > hangout
> > > > > > > invite
> > > > > > > > > to
> > > > > > > > > >> all
> > > > > > > > > >> > > > pmc,
> > > > > > > > > >> > > > > > > > >> committers
> > > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > > >> > > > > > > > >> >>>> > > everyone I found working on a
> > KIP.
> > > > If
> > > > > I
> > > > > > > missed
> > > > > > > > > >> anyone
> > > > > > > > > >> > > > in
> > > > > > > > > >> > > > > > the
> > > > > > > > > >> > > > > > > > >> invite
> > > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > > >> > > > > > > > >> >>>> > > let me know and can update it,
> > np.
> > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > >> > > > > > > > >> >>>> > > We should do this every
> Tuesday
> > @
> > > > 2pm
> > > > > > > Eastern
> > > > > > > > > >> Time.
> > > > > > > > > >> > > > Maybe
> > > > > > > > > >> > > > > > we
> > > > > > > > > >> > > > > > > > can
> > > > > > > > > >> > > > > > > > >> get
> > > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > > >> > > > > > > > >> >>>> > > help to make a google account
> > so we
> > > > > can
> > > > > > > manage
> > > > > > > > > >> > > better?
> > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > >> > > > > > > > >>
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > >
> > > > > > > > > >>
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > > >> > > > > > > > >> >>>> > > in progress and related JIRA
> > that
> > > > are
> > > > > > > > > >> interdependent
> > > > > > > > > >> > > > and
> > > > > > > > > >> > > > > > > common
> > > > > > > > > >> > > > > > > > >> work.
> > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59
> > PM, Jay
> > > > > > > Kreps <
> > > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts
> > that
> > > > > will
> > > > > > > also
> > > > > > > > > >> record
> > > > > > > > > >> > > > and
> > > > > > > > > >> > > > > > make
> > > > > > > > > >> > > > > > > > the
> > > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49
> > AM,
> > > > > Jeff
> > > > > > > > > Holoman
> > > > > > > > > >> <
> > > > > > > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > We're happy to send out a
> > Webex
> > > > for
> > > > > > > this
> > > > > > > > > >> purpose.
> > > > > > > > > >> > > We
> > > > > > > > > >> > > > > > could
> > > > > > > > > >> > > > > > > > >> record
> > > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > > >> > > > > > > > >> >>>> > >> > sessions if there is
> > interest and
> > > > > > > publish
> > > > > > > > > >> them
> > > > > > > > > >> > > out.
> > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at
> > 11:28 AM,
> > > > > Jay
> > > > > > > > > Kreps <
> > > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try to get the
> > technical
> > > > > > > hang-ups
> > > > > > > > > >> sorted
> > > > > > > > > >> > > > out,
> > > > > > > > > >> > > > > > > > though.
> > > > > > > > > >> > > > > > > > >> I
> > > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > there is some benefit to
> > live
> > > > > > > discussion
> > > > > > > > > vs
> > > > > > > > > >> > > > > writing. I
> > > > > > > > > >> > > > > > > am
> > > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > we post instructions and
> > give
> > > > > > > ourselves a
> > > > > > > > > >> few
> > > > > > > > > >> > > > > attempts
> > > > > > > > > >> > > > > > > we
> > > > > > > > > >> > > > > > > > >> can
> > > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that time
> would
> > work
> > > > > for
> > > > > > > > > >> me...any
> > > > > > > > > >> > > > > > objections?
> > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at
> > 8:18
> > > > AM,
> > > > > > Joe
> > > > > > > > > Stein
> > > > > > > > > >> <
> > > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly would be great
> > maybe
> > > > > like
> > > > > > > every
> > > > > > > > > >> > > Tuesday ~
> > > > > > > > > >> > > > > 1pm
> > > > > > > > > >> > > > > > > ET
> > > > > > > > > >> > > > > > > > /
> > > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't mind google
> > hangout
> > > > but
> > > > > > > there
> > > > > > > > > is
> > > > > > > > > >> > > always
> > > > > > > > > >> > > > > some
> > > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > we know the apache irc
> > > > channel
> > > > > > > works.
> > > > > > > > > We
> > > > > > > > > >> can
> > > > > > > > > >> > > > start
> > > > > > > > > >> > > > > > > there
> > > > > > > > > >> > > > > > > > >> and
> > > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull
> > transcripts
> > > > > too
> > > > > > > and
> > > > > > > > > >> > > associate
> > > > > > > > > >> > > > to
> > > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at
> > 11:10
> > > > > AM,
> > > > > > > Jay
> > > > > > > > > >> Kreps <
> > > > > > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked about
> > doing a
> > > > > > Google
> > > > > > > > > >> Hangout to
> > > > > > > > > >> > > > chat
> > > > > > > > > >> > > > > > > about
> > > > > > > > > >> > > > > > > > >> this.
> > > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > generalizing that a
> > little
> > > > > > > > > further...I
> > > > > > > > > >> > > > actually
> > > > > > > > > >> > > > > > > think
> > > > > > > > > >> > > > > > > > it
> > > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > everyone spending a
> > > > > reasonable
> > > > > > > chunk
> > > > > > > > > of
> > > > > > > > > >> > > their
> > > > > > > > > >> > > > > week
> > > > > > > > > >> > > > > > > on
> > > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync up once a week.
> I
> > > > think
> > > > > we
> > > > > > > could
> > > > > > > > > >> use
> > > > > > > > > >> > > time
> > > > > > > > > >> > > > > to
> > > > > > > > > >> > > > > > > talk
> > > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we
> > are on
> > > > > top
> > > > > > of
> > > > > > > > > code
> > > > > > > > > >> > > > reviews,
> > > > > > > > > >> > > > > > talk
> > > > > > > > > >> > > > > > > > >> through
> > > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We can make it
> publicly
> > > > > > > available so
> > > > > > > > > >> that
> > > > > > > > > >> > > any
> > > > > > > > > >> > > > > one
> > > > > > > > > >> > > > > > > can
> > > > > > > > > >> > > > > > > > >> follow
> > > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any interest in doing
> > this?
> > > > > If
> > > > > > so
> > > > > > > > > I'll
> > > > > > > > > >> try
> > > > > > > > > >> > > to
> > > > > > > > > >> > > > > set
> > > > > > > > > >> > > > > > it
> > > > > > > > > >> > > > > > > > up
> > > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015
> at
> > > > 3:57
> > > > > > AM,
> > > > > > > > > Andrii
> > > > > > > > > >> > > > > Biletskyi
> > > > > > > > > >> > > > > > <
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > andrii.biletskyi@stealth.ly>
> > > > > > > wrote:
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP
> > page,
> > > > > fixed
> > > > > > /
> > > > > > > > > >> aligned
> > > > > > > > > >> > > > > document
> > > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > very initial
> > proposal for
> > > > > > > > > >> AdminClient so
> > > > > > > > > >> > > we
> > > > > > > > > >> > > > > have
> > > > > > > > > >> > > > > > > > >> something
> > > > > > > > > >> > > > > > > > >> >>>> to
> > > > > > > > > >> > > > > > > > >> >>>> > >> start
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > from
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > while
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > > >> > > > > > > > >>
> > > > > > > > > >> > > > > > > >
> > > > > > > > > >> > > > > > >
> > > > > > > > > >> > > > > >
> > > > > > > > > >> > > > >
> > > > > > > > > >> > > >
> > > > > > > > > >> > >
> > > > > > > > > >>
> > > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18,
> 2015
> > at
> > > > > 9:01
> > > > > > > PM,
> > > > > > > > > >> Andrii
> > > > > > > > > >> > > > > > Biletskyi
> > > > > > > > > >> > > > > > > <
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > andrii.biletskyi@stealth.ly>
> > > > > > > > > wrote:
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re error
> messages:
> > you
> > > > > are
> > > > > > > right,
> > > > > > > > > >> in
> > > > > > > > > >> > > most
> > > > > > > > > >> > > > > > cases
> > > > > > > > > >> > > > > > > > >> client
> > > > > > > > > >> > > > > > > > >> >>>> will
> > > > > > > > > >> > > > > > > > >> >>>> > >> have
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > enough
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > context to show
> > > > > descriptive
> > > > > > > error
> > > > > > > > > >> > > message.
> > > > > > > > > >> > > > > My
> > > > > > > > > >> > > > > > > > >> concern is
> > > > > > > > > >> > > > > > > > >> >>>> that
> > > > > > > > > >> > > > > > > > >> >>>> > >> we
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > will
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > have
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > to
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > add lots of new
> > error
> > > > > codes
> > > > > > > for
> > > > > > > > > >> each
> > > > > > > > > >> > > > > possible
> > > > > > > > > >> > > > > > > > >> error. Of
> > > > > > > > > >> > > > > > > > >> >>>> course,
> > > > > > > > > >> > > > > > > > >> >>>> > >> > we
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > could
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > reuse
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > some of existing
> > like
> > > > > > > > > >> > > > > > > UknownTopicOrPartitionCode,
> > > > > > > > > >> > > > > > > > >> but we
> > > > > > > > > >> > > > > > > > >> >>>> will
> > > > > > > > > >> > > > > > > > >> >>>> > >> > also
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > need
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > to
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> > > > > > > > > >> TopicAlreadyExistsCode,
> > > > > > > > > >> > > > > > > > >> >>>> TopicConfigInvalid (both
> > > > > > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > topic
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > name and config,
> > and
> > > > > > probably
> > > > > > > > > user
> > > > > > > > > >> would
> > > > > > > > > >> > > > > like
> > > > > > > > > >> > > > > > to
> > > > > > > > > >> > > > > > > > >> know
> > > > > > > > > >> > > > > > > > >> >>>> what
> > > > > > > > > >> > > > > > > > >> >>>> > >> > exactly
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his
> > > > config),
> > > > > > > > > >> > > > > > > InvalidReplicaAssignment,
> > > > > > > > > >> > > > > > > > >> >>>> > >> InternalError
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > (e.g.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > zookeeper
> failure)
> > etc.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > And this is only
> > for
> > > > > > > > > TopicCommand,
> > > > > > > > > >> we
> > > > > > > > > >> > > will
> > > > > > > > > >> > > > > > also
> > > > > > > > > >> > > > > > > > >> need to
> > > > > > > > > >> > > > > > > > >> >>>> add
> > > > > > > > > >> > > > > > > > >> >>>> > >> > similar
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > for
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> ReassignPartitions,
> > > > > > > > > >> PreferredReplica. So
> > > > > > > > > >> > > > > we'll
> > > > > > > > > >> > > > > > > end
> > > > > > > > > >> > > > > > > > >> up
> > > > > > > > > >> > > > > > > > >> >>>> with a
> > > > > > > > > >> > > > > > > > >> >>>> > >> > large
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > list
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > of
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > error codes, used
> > only
> > > > in
> > > > > > > Admin
> > > > > > > > > >> > > protocol.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Having said
> that, I
> > > > agree
> > > > > > my
> > > > > > > > > >> proposal is
> > > > > > > > > >> > > > not
> > > > > > > > > >> > > > > > > > >> consistent
> > > > > > > > > >> > > > > > > > >> >>>> with
> > > > > > > > > >> > > > > > > > >> >>>> > >> > other
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > cases.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find
> > > > better
> > > > > > > solution
> > > > > > > > > >> or
> > > > > > > > > >> > > > > something
> > > > > > > > > >> > > > > > > > >> >>>> in-between.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat:
> I
> > > > think
> > > > > it
> > > > > > > is a
> > > > > > > > > >> great
> > > > > > > > > >> > > > idea.
> > > > > > > > > >> > > > > > > This
> > > > > > > > > >> > > > > > > > >> way we
> > > > > > > > > >> > > > > > > > >> >>>> can
> > > > > > > > > >> > > > > > > > >> >>>> > >> move
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > on
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > faster.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Let's agree
> > somehow on
> > > > > > > date/time
> > > > > > > > > so
> > > > > > > > > >> > > people
> > > > > > > > > >> > > > > can
> > > > > > > > > >> > > > > > > > join.
> > > > > > > > > >> > > > > > > > >> >>>> Will work
> > > > > > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > me
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > and
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > next week almost
> > > > anytime
> > > > > if
> > > > > > > > > agreed
> > > > > > > > > >> in
> > > > > > > > > >> > > > > advance.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18,
> > 2015 at
> > > > > > 7:09
> > > > > > > PM,
> > > > > > > > > >> Jay
> > > > > > > > > >> > > > Kreps <
> > > > > > > > > >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > wrote:
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can
> > do
> > > > good
> > > > > > > error
> > > > > > > > > >> handling
> > > > > > > > > >> > > > > > without
> > > > > > > > > >> > > > > > > > >> needing
> > > > > > > > > >> > > > > > > > >> >>>> custom
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > server-side
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e.
> > > > generally
> > > > > > the
> > > > > > > > > >> client has
> > > > > > > > > >> > > > the
> > > > > > > > > >> > > > > > > > >> context to
> > > > > > > > > >> > > > > > > > >> >>>> know
> > > > > > > > > >> > > > > > > > >> >>>> > >> that
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > if
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > it
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > got
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> an error that
> the
> > > > topic
> > > > > > > doesn't
> > > > > > > > > >> exist
> > > > > > > > > >> > > to
> > > > > > > > > >> > > > > say
> > > > > > > > > >> > > > > > > > >> "Topic X
> > > > > > > > > >> > > > > > > > >> >>>> doesn't
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > exist"
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> rather
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> than "error code
> > 14"
> > > > (or
> > > > > > > > > >> whatever).
> > > > > > > > > >> > > Maybe
> > > > > > > > > >> > > > > > there
> > > > > > > > > >> > > > > > > > are
> > > > > > > > > >> > > > > > > > >> >>>> specific
> > > > > > > > > >> > > > > > > > >> >>>> > >> > cases
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > where
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If
> > we
> > > > want
> > > > > > to
> > > > > > > add
> > > > > > > > > >> > > > server-side
> > > > > > > > > >> > > > > > > error
> > > > > > > > > >> > > > > > > > >> >>>> messages we
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > really
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > do
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> need to do this
> > in a
> > > > > > > consistent
> > > > > > > > > >> way
> > > > > > > > > >> > > > across
> > > > > > > > > >> > > > > > the
> > > > > > > > > >> > > > > > > > >> protocol.
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> I still have a
> > bunch
> > > > of
> > > > > > open
> > > > > > > > > >> questions
> > > > > > > > > >> > > > here
> > > > > > > > > >> > > > > > > from
> > > > > > > > > >> > > > > > > > my
> > > > > > > > > >> > > > > > > > >> >>>> previous
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > list. I
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > will
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> be out for the
> > next
> > > > few
> > > > > > > days for
> > > > > > > > > >> Strata
> > > > > > > > > >> > > > > > though.
> > > > > > > > > >> > > > > > > > >> Maybe
> > > > > > > > > >> > > > > > > > >> >>>> we could
> > > > > > > > > >> > > > > > > > >> >>>> > >> > do
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > a
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Google
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on
> > any
> > > > open
> > > > > > > issues
> > > > > > > > > >> some
> > > > > > > > > >> > > time
> > > > > > > > > >> > > > > > > towards
> > > > > > > > > >> > > > > > > > >> the
> > > > > > > > > >> > > > > > > > >> >>>> end of
> > > > > > > > > >> > > > > > > > >> >>>> > >> > next
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > week
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > for
> > > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> anyone inte
> ...
>
> [Message clipped]

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Guozhang Wang <wa...@gmail.com>.
+1 on broker writing to ZK for async handling. I was thinking that in the
end state the admin requests would be eventually sent to controller either
through re-routing or clients discovering them, instead of letting
controller listen on ZK admin path. But thinking about it a second time, I
think it is actually simpler to let controller manage
incoming queued-up admin requests through ZK.

Guozhang


On Wed, Mar 18, 2015 at 4:16 PM, Joel Koshy <jj...@gmail.com> wrote:

> +1 as well. I think it helps to keep the rerouting approach orthogonal
> to this KIP.
>
> On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps wrote:
> > I'm +1 on Jun's suggestion as long as it can work for all the requests.
> >
> > On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Andrii,
> > >
> > > I think we agreed on the following.
> > >
> > > (a) Admin requests can be sent to and handled by any broker.
> > > (b) Admin requests are processed asynchronously, at least for now.
> That is,
> > > when the client gets a response, it just means that the request is
> > > initiated, but not necessarily completed. Then, it's up to the client
> to
> > > issue another request to check the status for completion.
> > >
> > > To support (a), we were thinking of doing request forwarding to the
> > > controller (utilizing KAFKA-1912). I am making an alternative proposal.
> > > Basically, the broker can just write to ZooKeeper to inform the
> controller
> > > about the request. For example, to handle partitionReassignment, the
> broker
> > > will just write the requested partitions to /admin/reassign_partitions
> > > (like what AdminUtils currently does) and then send a response to the
> > > client. This shouldn't take long and the implementation will be simpler
> > > than forwarding the requests to the controller through RPC.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Jun,
> > > >
> > > > I might be wrong but didn't we agree we will let any broker from the
> > > > cluster handle *long-running* admin requests (at this time
> > > preferredReplica
> > > > and
> > > > reassignPartitions), via zk admin path. Thus CreateTopics etc should
> be
> > > > sent
> > > > only to the controller.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Joel, Andril,
> > > > >
> > > > > I think we agreed that those admin requests can be issued to any
> > > broker.
> > > > > Because of that, there doesn't seem to be a strong need to know the
> > > > > controller. So, perhaps we can proceed by not making any change to
> the
> > > > > format of TMR right now. When we start using create topic request
> in
> > > the
> > > > > producer, we will need a new version of TMR that doesn't trigger
> auto
> > > > topic
> > > > > creation. But that can be done later.
> > > > >
> > > > > As a first cut implementation, I think the broker can just write
> to ZK
> > > > > directly for
> > > > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > > requests, instead of forwarding them to the controller. This will
> > > > simplify
> > > > > the implementation on the broker side.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <jj...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > (Thanks Andrii for the summary)
> > > > > >
> > > > > > For (1) yes we will circle back on that shortly after syncing up
> in
> > > > > > person. I think it is close to getting committed although
> development
> > > > > > for KAFKA-1927 can probably begin without it.
> > > > > >
> > > > > > There is one more item we covered at the hangout. i.e., whether
> we
> > > > > > want to add the coordinator to the topic metadata response or
> provide
> > > > > > a clearer ClusterMetadataRequest.
> > > > > >
> > > > > > There are two reasons I think we should try and avoid adding the
> > > > > > field:
> > > > > > - It is irrelevant to topic metadata
> > > > > > - If we finally do request rerouting in Kafka then the field
> would
> > > add
> > > > > >   little to no value. (It still helps to have a separate
> > > > > >   ClusterMetadataRequest to query for cluster-wide information
> such
> > > as
> > > > > >   'which broker is the controller?' as Joe mentioned.)
> > > > > >
> > > > > > I think it would be cleaner to have an explicit
> > > ClusterMetadataRequest
> > > > > > that you can send to any broker in order to obtain the controller
> > > (and
> > > > > > in the future possibly other cluster-wide information). I think
> the
> > > > > > main argument against doing this and instead adding it to the
> topic
> > > > > > metadata response was convenience - i.e., you don't have to
> discover
> > > > > > the controller in advance. However, I don't see much actual
> > > > > > benefit/convenience in this and in fact think it is a non-issue.
> Let
> > > > > > me know if I'm overlooking something here.
> > > > > >
> > > > > > As an example, say we need to initiate partition reassignment by
> > > > > > issuing the new ReassignPartitionsRequest to the controller
> (assume
> > > we
> > > > > > already have the desired manual partition assignment).  If we
> are to
> > > > > > augment topic metadata response then the flow be something like
> this
> > > :
> > > > > >
> > > > > > - Issue topic metadata request to any broker (and discover the
> > > > > >   controller
> > > > > > - Connect to controller if required (i.e., if the broker above !=
> > > > > >   controller)
> > > > > > - Issue the partition reassignment request to the controller.
> > > > > >
> > > > > > With an explicit cluster metadata request it would be:
> > > > > > - Issue cluster metadata request to any broker
> > > > > > - Connect to controller if required (i.e., if the broker above !=
> > > > > >   controller)
> > > > > > - Issue the partition reassignment request
> > > > > >
> > > > > > So it seems to add little practical value and bloats topic
> metadata
> > > > > > response with an irrelevant detail.
> > > > > >
> > > > > > The other angle to this is the following - is it a matter of
> naming?
> > > > > > Should we just rename topic metadata request/response to just
> > > > > > MetadataRequest/Response and add cluster metadata to it? By that
> same
> > > > > > token should we also allow querying for the consumer coordinator
> (and
> > > > > > in future transaction coordinator) as well? This leads to a
> bloated
> > > > > > request which isn't very appealing and altogether confusing.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Joel
> > > > > >
> > > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > > > > > > Andri,
> > > > > > >
> > > > > > > Thanks for the summary.
> > > > > > >
> > > > > > > 1. I just realized that in order to start working on
> KAFKA-1927, we
> > > > > will
> > > > > > > need to merge the changes to OffsetCommitRequest (from 0.8.2)
> to
> > > > trunk.
> > > > > > > This is planned to be done as part of KAFKA-1634. So, we will
> need
> > > > > > Guozhang
> > > > > > > and Joel's help to wrap this up.
> > > > > > >
> > > > > > > 2. Thinking about this a bit more, if the semantic of those
> "write"
> > > > > > > requests is async (i.e., after the client gets a response, it
> just
> > > > > means
> > > > > > > that the operation is initiated, but not necessarily
> completed), we
> > > > > don't
> > > > > > > really need to forward the requests to the controller.
> Instead, the
> > > > > > > receiving broker can just write the operation to ZK as the
> admin
> > > > > command
> > > > > > > line tool previously does. This will simplify the
> implementation.
> > > > > > >
> > > > > > > 8. There is another implementation detail for describe topic.
> > > > Ideally,
> > > > > we
> > > > > > > want to read the topic config from the broker cache, instead of
> > > > > > ZooKeeper.
> > > > > > > Currently, every broker reads the topic-level config for all
> > > topics.
> > > > > > > However, it ignores those for topics not hosted on itself. So,
> we
> > > may
> > > > > > need
> > > > > > > to change TopicConfigManager a bit so that it caches the
> configs
> > > for
> > > > > all
> > > > > > > topics.
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >
> > > > > > > > Guys,
> > > > > > > >
> > > > > > > > Thanks for a great discussion!
> > > > > > > > Here are the actions points:
> > > > > > > >
> > > > > > > > 1. Q: Get rid of all scala requests objects, use java
> protocol
> > > > > > definitions.
> > > > > > > >     A: Gwen kindly took that (KAFKA-1927). It's important to
> > > speed
> > > > up
> > > > > > > > review procedure
> > > > > > > >          there since this ticket blocks other important
> changes.
> > > > > > > >
> > > > > > > > 2. Q: Generic re-reroute facility vs client maintaining
> cluster
> > > > > state.
> > > > > > > >     A: Jay has added pseudo code to KAFKA-1912 - need to
> consider
> > > > > > whether
> > > > > > > > this will be
> > > > > > > >         easy to implement as a server-side feature (comments
> are
> > > > > > > > welcomed!).
> > > > > > > >
> > > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > > >     A: This might be useful for clients, add this to
> > > > > > TopicMetadataResponse
> > > > > > > > (already in KIP).
> > > > > > > >
> > > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > > >     A: I will add proposed by Jun solution (using clientId
> for
> > > > that)
> > > > > > to the
> > > > > > > > KIP.
> > > > > > > >
> > > > > > > > 5. Q: Bumping new versions of TMR vs grabbing all protocol
> > > changes
> > > > in
> > > > > > one
> > > > > > > > version.
> > > > > > > >     A: It was decided to try to gather all changes to
> protocol
> > > > > (before
> > > > > > > > release).
> > > > > > > >         In case of TMR it worth checking: KAFKA-2020 and
> KIP-13
> > > > > > (quotas)
> > > > > > > >
> > > > > > > > 6. Q: JSON lib is needed to deserialize user's input in CLI
> tool.
> > > > > > > >     A: Use jackson for that, /tools project is a separate
> jar so
> > > > > > shouldn't
> > > > > > > > be a big deal.
> > > > > > > >
> > > > > > > > 7.  Q: VerifyReassingPartitions vs generic status check
> command.
> > > > > > > >      A: For long-running requests like reassign partitions
> > > > *progress*
> > > > > > check
> > > > > > > > request is useful,
> > > > > > > >          it makes sense to introduce it.
> > > > > > > >
> > > > > > > >  Please add, correct me if I missed something.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Andrii Biletskyi
> > > > > > > >
> > > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >
> > > > > > > > > Joel,
> > > > > > > > >
> > > > > > > > > You are right, I removed ClusterMetadata because we have
> > > > partially
> > > > > > > > > what we need in TopicMetadata. Also, as Jay pointed out
> > > earlier,
> > > > we
> > > > > > > > > would like to have "orthogonal" API, but at the same time
> we
> > > need
> > > > > > > > > to be backward compatible.
> > > > > > > > >
> > > > > > > > > But I like your idea and even have some other arguments for
> > > this
> > > > > > option:
> > > > > > > > > There is also DescribeTopicRequest which was proposed in
> this
> > > > KIP,
> > > > > > > > > it returns topic configs, partitions, replication factor
> plus
> > > > > > partition
> > > > > > > > > ISR, ASR,
> > > > > > > > > leader replica. The later part is really already there in
> > > > > > > > > TopicMetadataRequest.
> > > > > > > > > So again we'll have to add stuff to TMR, not to duplicate
> some
> > > > info
> > > > > > in
> > > > > > > > > newly added requests. However, this way we'll end up with
> > > > "monster"
> > > > > > > > > request which returns cluster metadata, topic replication
> and
> > > > > config
> > > > > > info
> > > > > > > > > plus partition replication data. Seems logical to split
> TMR to
> > > > > > > > > - ClusterMetadata (brokers + controller, maybe smth else)
> > > > > > > > > - TopicMetadata (topic info + partition details)
> > > > > > > > > But since current TMR is involved in lots of places
> (including
> > > > > > network
> > > > > > > > > client,
> > > > > > > > > as I understand) this might be very serious change and it
> > > > probably
> > > > > > makes
> > > > > > > > > sense to stick with current approach.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Andrii Biletskyi
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <
> > > jjkoshy.w@gmail.com
> > > > >
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > >> I may be missing some context but hopefully this will
> also be
> > > > > > covered
> > > > > > > > >> today: I thought the earlier proposal where there was an
> > > > explicit
> > > > > > > > >> ClusterMetadata request was clearer and explicit. During
> the
> > > > > course
> > > > > > of
> > > > > > > > >> this thread I think the conclusion was that the main need
> was
> > > > for
> > > > > > > > >> controller information and that can be rolled into the
> topic
> > > > > > metadata
> > > > > > > > >> response but that seems a bit irrelevant to topic
> metadata.
> > > > FWIW I
> > > > > > > > >> think the full broker-list is also irrelevant to topic
> > > metadata,
> > > > > but
> > > > > > > > >> it is already there and in use. I think there is still
> room
> > > for
> > > > an
> > > > > > > > >> explicit ClusterMetadata request since there may be other
> > > > > > > > >> cluster-level information that we may want to add over
> time
> > > (and
> > > > > > that
> > > > > > > > >> have nothing to do with topic metadata).
> > > > > > > > >>
> > > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi
> > > > wrote:
> > > > > > > > >> > Jun,
> > > > > > > > >> >
> > > > > > > > >> > 101. Okay, if you say that such use case is important. I
> > > also
> > > > > > think
> > > > > > > > >> > using clientId for these purposes is fine - if we
> already
> > > have
> > > > > > this
> > > > > > > > >> field
> > > > > > > > >> > as part of all Wire protocol messages, why not use that.
> > > > > > > > >> > I will update KIP-4 page if nobody has other ideas
> (which
> > > may
> > > > > > come up
> > > > > > > > >> > during the call today).
> > > > > > > > >> >
> > > > > > > > >> > 102.1 Agree, I'll update the KIP accordingly. I think
> we can
> > > > add
> > > > > > new,
> > > > > > > > >> > fine-grained error codes if some error code received in
> > > > specific
> > > > > > case
> > > > > > > > >> > won't give enough context to return a descriptive error
> > > > message
> > > > > > for
> > > > > > > > >> user.
> > > > > > > > >> >
> > > > > > > > >> > Look forward to discussing all outstanding issues in
> detail
> > > > > today
> > > > > > > > during
> > > > > > > > >> > the call.
> > > > > > > > >> >
> > > > > > > > >> > Thanks,
> > > > > > > > >> > Andrii Biletskyi
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <
> jun@confluent.io
> > > >
> > > > > > wrote:
> > > > > > > > >> >
> > > > > > > > >> > > 101. There may be a use case where you only want the
> > > topics
> > > > to
> > > > > > be
> > > > > > > > >> created
> > > > > > > > >> > > manually by admins. Currently, you can do that by
> > > disabling
> > > > > auto
> > > > > > > > topic
> > > > > > > > >> > > creation and issue topic creation from the
> TopicCommand.
> > > If
> > > > we
> > > > > > > > >> disable auto
> > > > > > > > >> > > topic creation completely on the broker and don't
> have a
> > > way
> > > > > to
> > > > > > > > >> distinguish
> > > > > > > > >> > > between topic creation requests from the regular
> clients
> > > and
> > > > > the
> > > > > > > > >> admin, we
> > > > > > > > >> > > can't support manual topic creation any more. I was
> > > thinking
> > > > > > that
> > > > > > > > >> another
> > > > > > > > >> > > way of distinguishing the clients making the topic
> > > creation
> > > > > > requests
> > > > > > > > >> is
> > > > > > > > >> > > using clientId. For example, the admin tool can set
> it to
> > > > > > something
> > > > > > > > >> like
> > > > > > > > >> > > admin and the broker can treat that clientId
> specially.
> > > > > > > > >> > >
> > > > > > > > >> > > Also, there is a related discussion in KAFKA-2020.
> > > > Currently,
> > > > > > we do
> > > > > > > > >> the
> > > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > > >> > >
> > > > > > > > >> > > 1. If leader is not available, we set the partition
> level
> > > > > error
> > > > > > code
> > > > > > > > >> to
> > > > > > > > >> > > LeaderNotAvailable.
> > > > > > > > >> > > 2. If a non-leader replica is not available, we take
> that
> > > > > > replica
> > > > > > > > out
> > > > > > > > >> of
> > > > > > > > >> > > the assigned replica list and isr in the response. As
> an
> > > > > > indication
> > > > > > > > >> for
> > > > > > > > >> > > doing that, we set the partition level error code to
> > > > > > > > >> ReplicaNotAvailable.
> > > > > > > > >> > >
> > > > > > > > >> > > This has a few problems. First, ReplicaNotAvailable
> > > probably
> > > > > > > > >> shouldn't be
> > > > > > > > >> > > an error, at least for the normal producer/consumer
> > > clients
> > > > > that
> > > > > > > > just
> > > > > > > > >> want
> > > > > > > > >> > > to find out the leader. Second, it can happen that
> both
> > > the
> > > > > > leader
> > > > > > > > and
> > > > > > > > >> > > another replica are not available at the same time.
> There
> > > is
> > > > > no
> > > > > > > > error
> > > > > > > > >> code
> > > > > > > > >> > > to indicate both. Third, even if a replica is not
> > > available,
> > > > > > it's
> > > > > > > > >> still
> > > > > > > > >> > > useful to return its replica id since some clients
> (e.g.
> > > > admin
> > > > > > tool)
> > > > > > > > >> may
> > > > > > > > >> > > still make use of it.
> > > > > > > > >> > >
> > > > > > > > >> > > One way to address this issue is to always return the
> > > > replica
> > > > > > id for
> > > > > > > > >> > > leader, assigned replicas, and isr regardless of
> whether
> > > the
> > > > > > > > >> corresponding
> > > > > > > > >> > > broker is live or not. Since we also return the list
> of
> > > live
> > > > > > > > brokers,
> > > > > > > > >> the
> > > > > > > > >> > > client can figure out whether a leader or a replica is
> > > live
> > > > or
> > > > > > not
> > > > > > > > >> and act
> > > > > > > > >> > > accordingly. This way, we don't need to set the
> partition
> > > > > level
> > > > > > > > error
> > > > > > > > >> code
> > > > > > > > >> > > when the leader or a replica is not available. This
> > > doesn't
> > > > > > change
> > > > > > > > >> the wire
> > > > > > > > >> > > protocol, but does change the semantics. Since we are
> > > > evolving
> > > > > > the
> > > > > > > > >> protocol
> > > > > > > > >> > > of TopicMetadataRequest here, we can potentially
> piggyback
> > > > the
> > > > > > > > change.
> > > > > > > > >> > >
> > > > > > > > >> > > 102.1 For those types of errors due to invalid input,
> > > > > shouldn't
> > > > > > we
> > > > > > > > >> just
> > > > > > > > >> > > guard it at parameter validation time and throw
> > > > > > > > >> InvalidArgumentException
> > > > > > > > >> > > without even sending the request to the broker?
> > > > > > > > >> > >
> > > > > > > > >> > > Thanks,
> > > > > > > > >> > >
> > > > > > > > >> > > Jun
> > > > > > > > >> > >
> > > > > > > > >> > >
> > > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >> > >
> > > > > > > > >> > > > Jun,
> > > > > > > > >> > > >
> > > > > > > > >> > > > Answering your questions:
> > > > > > > > >> > > >
> > > > > > > > >> > > > 101. If I understand you correctly, you are saying
> > > future
> > > > > > producer
> > > > > > > > >> > > versions
> > > > > > > > >> > > > (which
> > > > > > > > >> > > > will be ported to TMR_V1) won't be able to
> automatically
> > > > > > create
> > > > > > > > >> topic (if
> > > > > > > > >> > > > we
> > > > > > > > >> > > > unconditionally remove topic creation from there).
> But
> > > we
> > > > > > need to
> > > > > > > > >> this
> > > > > > > > >> > > > preserve logic.
> > > > > > > > >> > > > Ok, about your proposal: I'm not a big fan too,
> when it
> > > > > comes
> > > > > > to
> > > > > > > > >> > > > differentiating
> > > > > > > > >> > > > clients directly in protocol schema. And also I'm
> not
> > > > sure I
> > > > > > > > >> understand
> > > > > > > > >> > > at
> > > > > > > > >> > > > all why
> > > > > > > > >> > > > auto.create.topics.enable is a server side
> > > configuration.
> > > > > Can
> > > > > > we
> > > > > > > > >> > > deprecate
> > > > > > > > >> > > > this setting
> > > > > > > > >> > > > in future versions, add this setting to producer and
> > > based
> > > > > on
> > > > > > that
> > > > > > > > >> upon
> > > > > > > > >> > > > receiving
> > > > > > > > >> > > > UnknownTopic create topic explicitly by a separate
> > > > producer
> > > > > > call
> > > > > > > > via
> > > > > > > > >> > > > adminClient?
> > > > > > > > >> > > >
> > > > > > > > >> > > > 102.1. Hm, yes. It's because we want to support
> batching
> > > > and
> > > > > > at
> > > > > > > > the
> > > > > > > > >> same
> > > > > > > > >> > > > time we
> > > > > > > > >> > > > want to give descriptive error messages for clients.
> > > Since
> > > > > > > > >> AdminClient
> > > > > > > > >> > > > holds the context
> > > > > > > > >> > > > to construct such messages (e.g. AdminClient layer
> can
> > > > know
> > > > > > that
> > > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > > >> > > > means two cases: either invalid number - e.g. -1; or
> > > > > > > > >> replication-factor
> > > > > > > > >> > > was
> > > > > > > > >> > > > provided while
> > > > > > > > >> > > > partitions argument wasn't) - I wrapped responses in
> > > > > > Exceptions.
> > > > > > > > >> But I'm
> > > > > > > > >> > > > open to any
> > > > > > > > >> > > > other ideas, this was just initial version.
> > > > > > > > >> > > > 102.2. Yes, I agree. I'll change that to probably
> some
> > > > other
> > > > > > dto.
> > > > > > > > >> > > >
> > > > > > > > >> > > > Thanks,
> > > > > > > > >> > > > Andrii Biletskyi
> > > > > > > > >> > > >
> > > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <
> > > > jun@confluent.io>
> > > > > > > > wrote:
> > > > > > > > >> > > >
> > > > > > > > >> > > > > Andrii,
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > 101. That's what I was thinking too, but it may
> not be
> > > > > that
> > > > > > > > >> simple. In
> > > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > > >> > > > > we can let it not trigger auto topic creation.
> Then,
> > > in
> > > > > the
> > > > > > > > >> producer
> > > > > > > > >> > > > side,
> > > > > > > > >> > > > > if it gets an UnknownTopicException, it can
> explicitly
> > > > > > issue a
> > > > > > > > >> > > > > createTopicRequest for auto topic creation. On the
> > > > > consumer
> > > > > > > > side,
> > > > > > > > >> it
> > > > > > > > >> > > will
> > > > > > > > >> > > > > never issue createTopicRequest. This works when
> auto
> > > > topic
> > > > > > > > >> creation is
> > > > > > > > >> > > > > enabled on the broker side. However, I am not
> sure how
> > > > > > things
> > > > > > > > >> will work
> > > > > > > > >> > > > > when auto topic creation is disabled on the broker
> > > side.
> > > > > In
> > > > > > this
> > > > > > > > >> case,
> > > > > > > > >> > > we
> > > > > > > > >> > > > > want to have a way to manually create a topic,
> > > > potentially
> > > > > > > > through
> > > > > > > > >> > > admin
> > > > > > > > >> > > > > commands. However, then we need a way to
> distinguish
> > > > > > > > >> createTopicRequest
> > > > > > > > >> > > > > issued from the producer clients and the admin
> tools.
> > > > May
> > > > > > be we
> > > > > > > > >> can
> > > > > > > > >> > > add a
> > > > > > > > >> > > > > new field in createTopicRequest and set it
> differently
> > > > in
> > > > > > the
> > > > > > > > >> producer
> > > > > > > > >> > > > > client and the admin client. However, I am not
> sure if
> > > > > > that's
> > > > > > > > the
> > > > > > > > >> best
> > > > > > > > >> > > > > approach.
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > 2. Yes, refactoring existing requests is a
> non-trivial
> > > > > > amount of
> > > > > > > > >> work.
> > > > > > > > >> > > I
> > > > > > > > >> > > > > posted some comments in KAFKA-1927. We will
> probably
> > > > have
> > > > > > to fix
> > > > > > > > >> > > > KAFKA-1927
> > > > > > > > >> > > > > first, before adding the new logic in KAFKA-1694.
> > > > > > Otherwise, the
> > > > > > > > >> > > changes
> > > > > > > > >> > > > > will be too big.
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > > >> > > > > 102.1. It's a bit weird that we return exception
> in
> > > the
> > > > > > api. It
> > > > > > > > >> seems
> > > > > > > > >> > > > that
> > > > > > > > >> > > > > we should either return error code or throw an
> > > exception
> > > > > > when
> > > > > > > > >> getting
> > > > > > > > >> > > the
> > > > > > > > >> > > > > response state.
> > > > > > > > >> > > > > 102.2. We probably shouldn't explicitly use the
> > > request
> > > > > > object
> > > > > > > > in
> > > > > > > > >> the
> > > > > > > > >> > > > api.
> > > > > > > > >> > > > > Not every request evolution requires an api
> change.
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > Thanks,
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > Jun
> > > > > > > > >> > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi
> <
> > > > > > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >> > > > >
> > > > > > > > >> > > > > > Jun,
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > Thanks for you comments. Answers inline:
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > 100. There are a few fields such as
> > > ReplicaAssignment,
> > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > >> > > > > > > and PartitionsSerialized that are represented
> as a
> > > > > > string,
> > > > > > > > but
> > > > > > > > >> > > > contain
> > > > > > > > >> > > > > > > composite structures in json. Could we flatten
> > > them
> > > > > out
> > > > > > > > >> directly in
> > > > > > > > >> > > > the
> > > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > Yes, now with Admin Client this looks a bit
> weird.
> > > My
> > > > > > initial
> > > > > > > > >> > > > motivation
> > > > > > > > >> > > > > > was:
> > > > > > > > >> > > > > > ReassignPartitionCommand accepts input in json,
> we
> > > > want
> > > > > to
> > > > > > > > >> remain
> > > > > > > > >> > > > tools'
> > > > > > > > >> > > > > > interfaces unchanged, where possible.
> > > > > > > > >> > > > > > If we port it to deserialized format, in CLI
> (/tools
> > > > > > project)
> > > > > > > > >> we will
> > > > > > > > >> > > > > have
> > > > > > > > >> > > > > > to add some
> > > > > > > > >> > > > > > json library since /tools is written in java and
> > > we'll
> > > > > > need to
> > > > > > > > >> > > > > deserialize
> > > > > > > > >> > > > > > json file
> > > > > > > > >> > > > > > provided by a user. Can we quickly agree on what
> > > this
> > > > > > library
> > > > > > > > >> should
> > > > > > > > >> > > be
> > > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > 101. Does TopicMetadataRequest v1 still trigger
> auto
> > > > > topic
> > > > > > > > >> creation?
> > > > > > > > >> > > > This
> > > > > > > > >> > > > > > > will be a bit weird now that we have a
> separate
> > > > topic
> > > > > > > > >> creation api.
> > > > > > > > >> > > > > Have
> > > > > > > > >> > > > > > > you thought about how the new
> createTopicRequest
> > > and
> > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > >> > > > > > > v1 will be used in the producer/consumer
> client,
> > > in
> > > > > > addition
> > > > > > > > >> to
> > > > > > > > >> > > admin
> > > > > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > > > > >> TopicMetadataRequest
> > > > > > > > >> > > from
> > > > > > > > >> > > > > the
> > > > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > I agree, this strange logic should be fixed.
> I'm not
> > > > > > confident
> > > > > > > > >> in
> > > > > > > > >> > > this
> > > > > > > > >> > > > > > Kafka part so
> > > > > > > > >> > > > > > correct me if I'm wrong, but it doesn't look
> like a
> > > > hard
> > > > > > thing
> > > > > > > > >> to
> > > > > > > > >> > > do, I
> > > > > > > > >> > > > > > think we can
> > > > > > > > >> > > > > > leverage AdminClient for that in Producer and
> > > > > > unconditionally
> > > > > > > > >> remove
> > > > > > > > >> > > > > topic
> > > > > > > > >> > > > > > creation from the TopicMetadataRequest_V1.
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > 2. I think Jay meant getting rid of scala
> classes
> > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > >> > > did
> > > > > > > > >> > > > > > that
> > > > > > > > >> > > > > > > as a stop-gap thing when adding the new
> requests
> > > for
> > > > > the
> > > > > > > > >> consumers.
> > > > > > > > >> > > > > > > However, the long term plan is to get rid of
> all
> > > > those
> > > > > > and
> > > > > > > > >> just
> > > > > > > > >> > > reuse
> > > > > > > > >> > > > > the
> > > > > > > > >> > > > > > > java request/response in the client. Since
> this
> > > KIP
> > > > > > proposes
> > > > > > > > >> to
> > > > > > > > >> > > add a
> > > > > > > > >> > > > > > > significant number of new requests, perhaps we
> > > > should
> > > > > > bite
> > > > > > > > the
> > > > > > > > >> > > bullet
> > > > > > > > >> > > > > to
> > > > > > > > >> > > > > > > clean up the existing scala requests first
> before
> > > > > > adding new
> > > > > > > > >> ones?
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > Yes, looks like I misunderstood the point of
> > > > > > > > >> ...RequestAndHeader.
> > > > > > > > >> > > > Okay, I
> > > > > > > > >> > > > > > will
> > > > > > > > >> > > > > > rework that. The only thing is that I don't see
> any
> > > > > > example
> > > > > > > > how
> > > > > > > > >> it
> > > > > > > > >> > > was
> > > > > > > > >> > > > > done
> > > > > > > > >> > > > > > for at
> > > > > > > > >> > > > > > least one existing protocol message. Thus, as I
> > > > > > understand, I
> > > > > > > > >> have to
> > > > > > > > >> > > > > think
> > > > > > > > >> > > > > > how we
> > > > > > > > >> > > > > > are going to do it.
> > > > > > > > >> > > > > > Re porting all existing RQ/RP in this patch.
> Sounds
> > > > > > > > reasonable,
> > > > > > > > >> but
> > > > > > > > >> > > if
> > > > > > > > >> > > > > it's
> > > > > > > > >> > > > > > an *obligatory*
> > > > > > > > >> > > > > > requirement to have Admin KIP done, I'm afraid
> this
> > > > can
> > > > > > be a
> > > > > > > > >> serious
> > > > > > > > >> > > > > > blocker for us.
> > > > > > > > >> > > > > > There are 13 protocol messages and all that
> would
> > > > > require
> > > > > > not
> > > > > > > > >> only
> > > > > > > > >> > > unit
> > > > > > > > >> > > > > > tests but quite
> > > > > > > > >> > > > > > intensive manual testing, no? I'm afraid I'm
> not the
> > > > > > right guy
> > > > > > > > >> to
> > > > > > > > >> > > cover
> > > > > > > > >> > > > > > pretty much all
> > > > > > > > >> > > > > > Kafka core internals :). Let me know your
> thoughts
> > > on
> > > > > this
> > > > > > > > >> item. Btw
> > > > > > > > >> > > > > there
> > > > > > > > >> > > > > > is a ticket to
> > > > > > > > >> > > > > > follow-up this issue (
> > > > > > > > >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > > >> > > > ).
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > Thanks,
> > > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <
> > > > > > jun@confluent.io>
> > > > > > > > >> wrote:
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > > > > Andrii,
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > A few more comments.
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > 100. There are a few fields such as
> > > > ReplicaAssignment,
> > > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > > >> > > > > > > and PartitionsSerialized that are represented
> as a
> > > > > > string,
> > > > > > > > but
> > > > > > > > >> > > > contain
> > > > > > > > >> > > > > > > composite structures in json. Could we flatten
> > > them
> > > > > out
> > > > > > > > >> directly in
> > > > > > > > >> > > > the
> > > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > 101. Does TopicMetadataRequest v1 still
> trigger
> > > auto
> > > > > > topic
> > > > > > > > >> > > creation?
> > > > > > > > >> > > > > This
> > > > > > > > >> > > > > > > will be a bit weird now that we have a
> separate
> > > > topic
> > > > > > > > >> creation api.
> > > > > > > > >> > > > > Have
> > > > > > > > >> > > > > > > you thought about how the new
> createTopicRequest
> > > and
> > > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > > >> > > > > > > v1 will be used in the producer/consumer
> client,
> > > in
> > > > > > addition
> > > > > > > > >> to
> > > > > > > > >> > > admin
> > > > > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > > > > >> TopicMetadataRequest
> > > > > > > > >> > > from
> > > > > > > > >> > > > > the
> > > > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > 2. I think Jay meant getting rid of scala
> classes
> > > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > > >> > > did
> > > > > > > > >> > > > > > that
> > > > > > > > >> > > > > > > as a stop-gap thing when adding the new
> requests
> > > for
> > > > > the
> > > > > > > > >> consumers.
> > > > > > > > >> > > > > > > However, the long term plan is to get rid of
> all
> > > > those
> > > > > > and
> > > > > > > > >> just
> > > > > > > > >> > > reuse
> > > > > > > > >> > > > > the
> > > > > > > > >> > > > > > > java request/response in the client. Since
> this
> > > KIP
> > > > > > proposes
> > > > > > > > >> to
> > > > > > > > >> > > add a
> > > > > > > > >> > > > > > > significant number of new requests, perhaps we
> > > > should
> > > > > > bite
> > > > > > > > the
> > > > > > > > >> > > bullet
> > > > > > > > >> > > > > to
> > > > > > > > >> > > > > > > clean up the existing scala requests first
> before
> > > > > > adding new
> > > > > > > > >> ones?
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > Thanks,
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > Jun
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii
> Biletskyi
> > > <
> > > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > > Hi,
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > As said above - I list again all comments
> from
> > > > this
> > > > > > thread
> > > > > > > > >> so we
> > > > > > > > >> > > > > > > > can see what's left and finalize all pending
> > > > issues.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > > >> > > > > > > > 1. This is much needed functionality, but
> there
> > > > are
> > > > > a
> > > > > > lot
> > > > > > > > >> of the
> > > > > > > > >> > > so
> > > > > > > > >> > > > > > let's
> > > > > > > > >> > > > > > > > really think these protocols through. We
> really
> > > > want
> > > > > > to
> > > > > > > > end
> > > > > > > > >> up
> > > > > > > > >> > > > with a
> > > > > > > > >> > > > > > set
> > > > > > > > >> > > > > > > > of well thought-out, orthoganol apis. For
> this
> > > > > reason
> > > > > > I
> > > > > > > > >> think it
> > > > > > > > >> > > is
> > > > > > > > >> > > > > > > really
> > > > > > > > >> > > > > > > > important to think through the end state
> even if
> > > > > that
> > > > > > > > >> includes
> > > > > > > > >> > > APIs
> > > > > > > > >> > > > > we
> > > > > > > > >> > > > > > > > won't implement in the first phase.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: Definitely behind this. Would appreciate
> if
> > > > there
> > > > > > are
> > > > > > > > >> concrete
> > > > > > > > >> > > > > > > comments
> > > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 2. Let's please please please wait until we
> have
> > > > > > switched
> > > > > > > > >> the
> > > > > > > > >> > > > server
> > > > > > > > >> > > > > > over
> > > > > > > > >> > > > > > > > to the new java protocol definitions. If we
> add
> > > > > upteen
> > > > > > > > more
> > > > > > > > >> ad
> > > > > > > > >> > > hoc
> > > > > > > > >> > > > > > scala
> > > > > > > > >> > > > > > > > objects that is just generating more work
> for
> > > the
> > > > > > > > >> conversion we
> > > > > > > > >> > > > know
> > > > > > > > >> > > > > we
> > > > > > > > >> > > > > > > > have to do.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed scala
> > > > > protocol
> > > > > > > > >> classes.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 3. This proposal introduces a new type of
> > > optional
> > > > > > > > >> parameter.
> > > > > > > > >> > > This
> > > > > > > > >> > > > is
> > > > > > > > >> > > > > > > > inconsistent with everything else in the
> > > protocol
> > > > > > where we
> > > > > > > > >> use -1
> > > > > > > > >> > > > or
> > > > > > > > >> > > > > > some
> > > > > > > > >> > > > > > > > other marker value. You could argue either
> way
> > > but
> > > > > > let's
> > > > > > > > >> stick
> > > > > > > > >> > > with
> > > > > > > > >> > > > > > that
> > > > > > > > >> > > > > > > > for consistency. For clients that
> implemented
> > > the
> > > > > > protocol
> > > > > > > > >> in a
> > > > > > > > >> > > > > better
> > > > > > > > >> > > > > > > way
> > > > > > > > >> > > > > > > > than our scala code these basic primitives
> are
> > > > hard
> > > > > to
> > > > > > > > >> change.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed
> MaybeOf
> > > > type
> > > > > > and
> > > > > > > > >> changed
> > > > > > > > >> > > > > > protocol
> > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> > > > > > > > >> TopicMetadataRequest
> > > > > > > > >> > > > > which
> > > > > > > > >> > > > > > > has
> > > > > > > > >> > > > > > > > brokers, topics, and partitions. I think we
> > > should
> > > > > > rename
> > > > > > > > >> that
> > > > > > > > >> > > > > request
> > > > > > > > >> > > > > > > > ClusterMetadataRequest (or just
> MetadataRequest)
> > > > and
> > > > > > > > >> include the
> > > > > > > > >> > > id
> > > > > > > > >> > > > > of
> > > > > > > > >> > > > > > > the
> > > > > > > > >> > > > > > > > controller. Or are there other things we
> could
> > > add
> > > > > > here?
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: I agree. Updated the KIP. Let's extends
> > > > > > TopicMetadata
> > > > > > > > to
> > > > > > > > >> > > > version 2
> > > > > > > > >> > > > > > and
> > > > > > > > >> > > > > > > > include controller.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 5. We have a tendency to try to make a lot
> of
> > > > > requests
> > > > > > > > that
> > > > > > > > >> can
> > > > > > > > >> > > > only
> > > > > > > > >> > > > > go
> > > > > > > > >> > > > > > > to
> > > > > > > > >> > > > > > > > particular nodes. This adds a lot of burden
> for
> > > > > client
> > > > > > > > >> > > > > implementations
> > > > > > > > >> > > > > > > (it
> > > > > > > > >> > > > > > > > sounds easy but each discovery can fail in
> many
> > > > > parts
> > > > > > so
> > > > > > > > it
> > > > > > > > >> ends
> > > > > > > > >> > > up
> > > > > > > > >> > > > > > > being a
> > > > > > > > >> > > > > > > > full state machine to do right). I think we
> > > should
> > > > > > > > consider
> > > > > > > > >> > > making
> > > > > > > > >> > > > > > admin
> > > > > > > > >> > > > > > > > commands and ideally as many of the other
> apis
> > > as
> > > > > > possible
> > > > > > > > >> > > > available
> > > > > > > > >> > > > > on
> > > > > > > > >> > > > > > > all
> > > > > > > > >> > > > > > > > brokers and just redirect to the controller
> on
> > > the
> > > > > > broker
> > > > > > > > >> side.
> > > > > > > > >> > > > > Perhaps
> > > > > > > > >> > > > > > > > there would be a general way to encapsulate
> this
> > > > > > > > re-routing
> > > > > > > > >> > > > behavior.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: It's a very interesting idea, but seems
> there
> > > > are
> > > > > > some
> > > > > > > > >> > > concerns
> > > > > > > > >> > > > > > about
> > > > > > > > >> > > > > > > > this
> > > > > > > > >> > > > > > > > feature (like performance considerations,
> how
> > > this
> > > > > > will
> > > > > > > > >> > > complicate
> > > > > > > > >> > > > > > server
> > > > > > > > >> > > > > > > > etc).
> > > > > > > > >> > > > > > > > I believe this shouldn't be a blocker. If
> this
> > > > > > feature is
> > > > > > > > >> > > > implemented
> > > > > > > > >> > > > > > at
> > > > > > > > >> > > > > > > > some
> > > > > > > > >> > > > > > > > point it won't affect Admin changes - at
> least
> > > no
> > > > > > changes
> > > > > > > > to
> > > > > > > > >> > > public
> > > > > > > > >> > > > > API
> > > > > > > > >> > > > > > > > will be required.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 6. We should probably normalize the key
> value
> > > > pairs
> > > > > > used
> > > > > > > > for
> > > > > > > > >> > > > configs
> > > > > > > > >> > > > > > > rather
> > > > > > > > >> > > > > > > > than embedding a new formatting. So two
> strings
> > > > > rather
> > > > > > > > than
> > > > > > > > >> one
> > > > > > > > >> > > > with
> > > > > > > > >> > > > > an
> > > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: Fixed in the latest patch - normalized
> > > configs
> > > > > and
> > > > > > > > >> changed
> > > > > > > > >> > > > > protocol
> > > > > > > > >> > > > > > > > accordingly.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 7. Is the postcondition of these APIs that
> the
> > > > > > command has
> > > > > > > > >> begun
> > > > > > > > >> > > or
> > > > > > > > >> > > > > > that
> > > > > > > > >> > > > > > > > the command has been completed? It is a lot
> more
> > > > > > usable if
> > > > > > > > >> the
> > > > > > > > >> > > > > command
> > > > > > > > >> > > > > > > has
> > > > > > > > >> > > > > > > > been completed so you know that if you
> create a
> > > > > topic
> > > > > > and
> > > > > > > > >> then
> > > > > > > > >> > > > > publish
> > > > > > > > >> > > > > > to
> > > > > > > > >> > > > > > > > it you won't get an exception about there
> being
> > > no
> > > > > > such
> > > > > > > > >> topic.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: For long running requests (like reassign
> > > > > > partitions) -
> > > > > > > > >> the
> > > > > > > > >> > > post
> > > > > > > > >> > > > > > > > condition is
> > > > > > > > >> > > > > > > > command has begun - so we don't block the
> > > client.
> > > > In
> > > > > > case
> > > > > > > > >> of your
> > > > > > > > >> > > > > > > example -
> > > > > > > > >> > > > > > > > topic commands, this will be refactored and
> > > topic
> > > > > > commands
> > > > > > > > >> will
> > > > > > > > >> > > be
> > > > > > > > >> > > > > > > executed
> > > > > > > > >> > > > > > > > immediately, since the Controller will serve
> > > Admin
> > > > > > > > requests
> > > > > > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 8. Describe topic and list topics duplicate
> a
> > > lot
> > > > of
> > > > > > stuff
> > > > > > > > >> in the
> > > > > > > > >> > > > > > > metadata
> > > > > > > > >> > > > > > > > request. Is there a reason to give back
> topics
> > > > > marked
> > > > > > for
> > > > > > > > >> > > > deletion? I
> > > > > > > > >> > > > > > > feel
> > > > > > > > >> > > > > > > > like if we just make the post-condition of
> the
> > > > > delete
> > > > > > > > >> command be
> > > > > > > > >> > > > that
> > > > > > > > >> > > > > > the
> > > > > > > > >> > > > > > > > topic is deleted that will get rid of the
> need
> > > for
> > > > > > this
> > > > > > > > >> right?
> > > > > > > > >> > > And
> > > > > > > > >> > > > it
> > > > > > > > >> > > > > > > will
> > > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed
> topics
> > > > marked
> > > > > > for
> > > > > > > > >> deletion
> > > > > > > > >> > > > in
> > > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 9. Should we consider batching these
> requests?
> > > We
> > > > > have
> > > > > > > > >> generally
> > > > > > > > >> > > > > tried
> > > > > > > > >> > > > > > to
> > > > > > > > >> > > > > > > > allow multiple operations to be batched. My
> > > > > suspicion
> > > > > > is
> > > > > > > > >> that
> > > > > > > > >> > > > without
> > > > > > > > >> > > > > > > this
> > > > > > > > >> > > > > > > > we will get a lot of code that does
> something
> > > like
> > > > > > > > >> > > > > > > >    for(topic: adminClient.listTopics())
> > > > > > > > >> > > > > > > >       adminClient.describeTopic(topic)
> > > > > > > > >> > > > > > > > this code will work great when you test on 5
> > > > topics
> > > > > > but
> > > > > > > > not
> > > > > > > > >> do as
> > > > > > > > >> > > > > well
> > > > > > > > >> > > > > > if
> > > > > > > > >> > > > > > > > you have 50k.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: Updated the KIP - please check "Topic
> Admin
> > > > > Schema"
> > > > > > > > >> section.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 10. I think we should also discuss how we
> want
> > > to
> > > > > > expose a
> > > > > > > > >> > > > > programmatic
> > > > > > > > >> > > > > > > JVM
> > > > > > > > >> > > > > > > > client api for these operations. Currently
> > > people
> > > > > > rely on
> > > > > > > > >> > > > AdminUtils
> > > > > > > > >> > > > > > > which
> > > > > > > > >> > > > > > > > is totally sketchy. I think we probably need
> > > > another
> > > > > > > > client
> > > > > > > > >> under
> > > > > > > > >> > > > > > > clients/
> > > > > > > > >> > > > > > > > that exposes administrative functionality.
> We
> > > will
> > > > > > need
> > > > > > > > >> this just
> > > > > > > > >> > > > to
> > > > > > > > >> > > > > > > > properly test the new apis, I suspect. We
> should
> > > > > > figure
> > > > > > > > out
> > > > > > > > >> that
> > > > > > > > >> > > > API.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: Updated the KIP - please check "Admin
> Client"
> > > > > > section
> > > > > > > > >> with an
> > > > > > > > >> > > > > > initial
> > > > > > > > >> > > > > > > > API proposal.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 11. The other information that would be
> really
> > > > > useful
> > > > > > to
> > > > > > > > get
> > > > > > > > >> > > would
> > > > > > > > >> > > > be
> > > > > > > > >> > > > > > > > information about partitions--how much data
> is
> > > in
> > > > > the
> > > > > > > > >> partition,
> > > > > > > > >> > > > what
> > > > > > > > >> > > > > > are
> > > > > > > > >> > > > > > > > the segment offsets, what is the log-end
> offset
> > > > > (i.e.
> > > > > > last
> > > > > > > > >> > > offset),
> > > > > > > > >> > > > > > what
> > > > > > > > >> > > > > > > is
> > > > > > > > >> > > > > > > > the compaction point, etc. I think that done
> > > right
> > > > > > this
> > > > > > > > >> would be
> > > > > > > > >> > > > the
> > > > > > > > >> > > > > > > > successor to the very awkward OffsetRequest
> we
> > > > have
> > > > > > today.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in
> the
> > > > > latest
> > > > > > > > >> patch. I
> > > > > > > > >> > > > > believe
> > > > > > > > >> > > > > > > > this should
> > > > > > > > >> > > > > > > > be resolved in a separate KIP / jira ticket.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 12. Generally we can do good error handling
> > > > without
> > > > > > > > needing
> > > > > > > > >> > > custom
> > > > > > > > >> > > > > > > > server-side
> > > > > > > > >> > > > > > > > messages. I.e. generally the client has the
> > > > context
> > > > > to
> > > > > > > > know
> > > > > > > > >> that
> > > > > > > > >> > > if
> > > > > > > > >> > > > > it
> > > > > > > > >> > > > > > > got
> > > > > > > > >> > > > > > > > an error that the topic doesn't exist to say
> > > > "Topic
> > > > > X
> > > > > > > > >> doesn't
> > > > > > > > >> > > > exist"
> > > > > > > > >> > > > > > > rather
> > > > > > > > >> > > > > > > > than "error code 14" (or whatever). Maybe
> there
> > > > are
> > > > > > > > specific
> > > > > > > > >> > > cases
> > > > > > > > >> > > > > > where
> > > > > > > > >> > > > > > > > this is hard? If we want to add server-side
> > > error
> > > > > > messages
> > > > > > > > >> we
> > > > > > > > >> > > > really
> > > > > > > > >> > > > > do
> > > > > > > > >> > > > > > > > need to do this in a consistent way across
> the
> > > > > > protocol.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: Updated the KIP - please check "Protocol
> > > > Errors"
> > > > > > > > >> section. I
> > > > > > > > >> > > > added
> > > > > > > > >> > > > > > the
> > > > > > > > >> > > > > > > > comprehensive, fine-grained list of error
> codes.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > > >> > > > > > > > 13. Describe topic request: it would be
> great to
> > > > go
> > > > > > beyond
> > > > > > > > >> just
> > > > > > > > >> > > > > > batching
> > > > > > > > >> > > > > > > on
> > > > > > > > >> > > > > > > > topic name regex for this request. For
> example,
> > > a
> > > > > very
> > > > > > > > >> common use
> > > > > > > > >> > > > > case
> > > > > > > > >> > > > > > of
> > > > > > > > >> > > > > > > > the topic command is to list all topics
> whose
> > > > config
> > > > > > A's
> > > > > > > > >> value is
> > > > > > > > >> > > > B.
> > > > > > > > >> > > > > > With
> > > > > > > > >> > > > > > > > topic name regex then we have to first
> retrieve
> > > > > > __all__
> > > > > > > > >> topics's
> > > > > > > > >> > > > > > > > description info and then filter at the
> client
> > > > end,
> > > > > > which
> > > > > > > > >> will
> > > > > > > > >> > > be a
> > > > > > > > >> > > > > > huge
> > > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > > >> > > > > > > > AND
> > > > > > > > >> > > > > > > > 14. Config K-Vs in create topic: this is
> related
> > > > to
> > > > > > the
> > > > > > > > >> previous
> > > > > > > > >> > > > > point;
> > > > > > > > >> > > > > > > > maybe we can add another metadata K-V or
> just a
> > > > > > metadata
> > > > > > > > >> string
> > > > > > > > >> > > > along
> > > > > > > > >> > > > > > > side
> > > > > > > > >> > > > > > > > with config K-V in create topic like we did
> for
> > > > > offset
> > > > > > > > >> commit
> > > > > > > > >> > > > > request.
> > > > > > > > >> > > > > > > This
> > > > > > > > >> > > > > > > > field can be quite useful in storing
> information
> > > > > like
> > > > > > > > >> "owner" of
> > > > > > > > >> > > > the
> > > > > > > > >> > > > > > > topic
> > > > > > > > >> > > > > > > > who issue the create command, etc, which is
> > > quite
> > > > > > > > important
> > > > > > > > >> for a
> > > > > > > > >> > > > > > > > multi-tenant setting. Then in the describe
> topic
> > > > > > request
> > > > > > > > we
> > > > > > > > >> can
> > > > > > > > >> > > > also
> > > > > > > > >> > > > > > > batch
> > > > > > > > >> > > > > > > > on regex of the metadata field.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: As discussed it is very interesting but
> can
> > > be
> > > > > > > > >> implemented
> > > > > > > > >> > > later
> > > > > > > > >> > > > > > after
> > > > > > > > >> > > > > > > > we have some basic functionality there.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > 15. Today all the admin operations are
> async in
> > > > the
> > > > > > sense
> > > > > > > > >> that
> > > > > > > > >> > > > > command
> > > > > > > > >> > > > > > > will
> > > > > > > > >> > > > > > > > return once it is written in ZK, and that
> is why
> > > > we
> > > > > > need
> > > > > > > > >> extra
> > > > > > > > >> > > > > > > verification
> > > > > > > > >> > > > > > > > like testUtil.waitForTopicCreated() / verify
> > > > > partition
> > > > > > > > >> > > reassignment
> > > > > > > > >> > > > > > > > request, etc. With admin requests we could
> add a
> > > > > flag
> > > > > > to
> > > > > > > > >> enable /
> > > > > > > > >> > > > > > disable
> > > > > > > > >> > > > > > > > synchronous requests; when it is turned on,
> the
> > > > > > response
> > > > > > > > >> will not
> > > > > > > > >> > > > > > return
> > > > > > > > >> > > > > > > > until the request has been completed. And
> for
> > > > async
> > > > > > > > >> requests we
> > > > > > > > >> > > can
> > > > > > > > >> > > > > > add a
> > > > > > > > >> > > > > > > > "token" field in the response, and then only
> > > need
> > > > a
> > > > > > > > general
> > > > > > > > >> > > "admin
> > > > > > > > >> > > > > > > > verification request" with the given token
> to
> > > > check
> > > > > > if the
> > > > > > > > >> async
> > > > > > > > >> > > > > > request
> > > > > > > > >> > > > > > > > has been completed.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: I see your point. My idea was to provide
> > > > specific
> > > > > > > > >> > > > Verify...Request
> > > > > > > > >> > > > > > per
> > > > > > > > >> > > > > > > > each
> > > > > > > > >> > > > > > > > long running request, where needed. We can
> do it
> > > > the
> > > > > > way
> > > > > > > > you
> > > > > > > > >> > > > suggest.
> > > > > > > > >> > > > > > The
> > > > > > > > >> > > > > > > > only
> > > > > > > > >> > > > > > > > concern is that introducing a token we again
> > > will
> > > > > make
> > > > > > > > >> schema
> > > > > > > > >> > > > > > "dynamic".
> > > > > > > > >> > > > > > > We
> > > > > > > > >> > > > > > > > wanted
> > > > > > > > >> > > > > > > > to do similar thing introducing single
> > > > AdminRequest
> > > > > > for
> > > > > > > > all
> > > > > > > > >> topic
> > > > > > > > >> > > > > > > commands
> > > > > > > > >> > > > > > > > but rejected
> > > > > > > > >> > > > > > > > this idea because we wanted to have schema
> > > > defined.
> > > > > So
> > > > > > > > this
> > > > > > > > >> is
> > > > > > > > >> > > > more a
> > > > > > > > >> > > > > > > > choice between:
> > > > > > > > >> > > > > > > > a) have fixed schema but introduce each
> time new
> > > > > > > > >> Verify...Request
> > > > > > > > >> > > > for
> > > > > > > > >> > > > > > > > long-running requests
> > > > > > > > >> > > > > > > > b) use one request for verification but
> > > generalize
> > > > > it
> > > > > > with
> > > > > > > > >> token
> > > > > > > > >> > > > > > > > I'm fine with whatever decision community
> come
> > > to.
> > > > > > Just
> > > > > > > > let
> > > > > > > > >> me
> > > > > > > > >> > > know
> > > > > > > > >> > > > > > your
> > > > > > > > >> > > > > > > > thoughts.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > > >> > > > > > > > 16. Specifically for ownership, I think the
> plan
> > > > is
> > > > > > to add
> > > > > > > > >> ACL
> > > > > > > > >> > > (it
> > > > > > > > >> > > > > > sounds
> > > > > > > > >> > > > > > > > like you are describing ACL) via an external
> > > > system
> > > > > > > > (Argus,
> > > > > > > > >> > > > Sentry).
> > > > > > > > >> > > > > > > > I remember KIP-11 described this, but I
> can't
> > > find
> > > > > > the KIP
> > > > > > > > >> any
> > > > > > > > >> > > > > longer.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > A: Okay, no problem. Not sure though how we
> are
> > > > > going
> > > > > > to
> > > > > > > > >> handle
> > > > > > > > >> > > it.
> > > > > > > > >> > > > > > Wait
> > > > > > > > >> > > > > > > > which KIP
> > > > > > > > >> > > > > > > > will be committed first and include changes
> to
> > > > > > > > >> TopicMetadata from
> > > > > > > > >> > > > the
> > > > > > > > >> > > > > > > later
> > > > > > > > >> > > > > > > > one?
> > > > > > > > >> > > > > > > > Anyway, I added this note to "Open
> Questions"
> > > > > section
> > > > > > so
> > > > > > > > we
> > > > > > > > >> don't
> > > > > > > > >> > > > > miss
> > > > > > > > >> > > > > > > this
> > > > > > > > >> > > > > > > > piece.
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > Thanks,
> > > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii
> > > > Biletskyi <
> > > > > > > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > > Hi all,
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > > Today I uploaded the patch that covers
> some of
> > > > the
> > > > > > > > >> discussed
> > > > > > > > >> > > and
> > > > > > > > >> > > > > > agreed
> > > > > > > > >> > > > > > > > > items:
> > > > > > > > >> > > > > > > > > - removed MaybeOf optional type
> > > > > > > > >> > > > > > > > > - switched to java protocol definitions
> > > > > > > > >> > > > > > > > > - simplified messages (normalized configs,
> > > > removed
> > > > > > topic
> > > > > > > > >> marked
> > > > > > > > >> > > > for
> > > > > > > > >> > > > > > > > > deletion)
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > > I also updated the KIP-4 with respective
> > > changes
> > > > > and
> > > > > > > > >> wrote down
> > > > > > > > >> > > > my
> > > > > > > > >> > > > > > > > > proposal for
> > > > > > > > >> > > > > > > > > pending items:
> > > > > > > > >> > > > > > > > > - Batch Admin Operations -> updated Wire
> > > > Protocol
> > > > > > schema
> > > > > > > > >> > > proposal
> > > > > > > > >> > > > > > > > > - Remove ClusterMetadata -> changed to
> extend
> > > > > > > > >> > > > TopicMetadataRequest
> > > > > > > > >> > > > > > > > > - Admin Client -> updated my initial
> proposal
> > > to
> > > > > > reflect
> > > > > > > > >> > > batching
> > > > > > > > >> > > > > > > > > - Error codes -> proposed fine-grained
> error
> > > > code
> > > > > > > > instead
> > > > > > > > >> of
> > > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > > I will also send a separate email to
> cover all
> > > > > > comments
> > > > > > > > >> from
> > > > > > > > >> > > this
> > > > > > > > >> > > > > > > thread.
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > > Thanks,
> > > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen
> Shapira
> > > <
> > > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > > > > > wrote:
> > > > > > > > >> > > > > > > > >
> > > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > > >> > > > > > > > >>
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >>
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > >> > > > > > > > >> )
> > > > > > > > >> > > > > > > > >> It actually specifies changes to the
> Metadata
> > > > > > protocol,
> > > > > > > > >> so
> > > > > > > > >> > > > making
> > > > > > > > >> > > > > > sure
> > > > > > > > >> > > > > > > > >> both KIPs are consistent in this regard
> will
> > > be
> > > > > > good.
> > > > > > > > >> > > > > > > > >>
> > > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen
> > > Shapira
> > > > <
> > > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > >> > > > > > > > >> > Specifically for ownership, I think the
> > > plan
> > > > is
> > > > > > to
> > > > > > > > add
> > > > > > > > >> ACL
> > > > > > > > >> > > (it
> > > > > > > > >> > > > > > > sounds
> > > > > > > > >> > > > > > > > >> > like you are describing ACL) via an
> > > external
> > > > > > system
> > > > > > > > >> (Argus,
> > > > > > > > >> > > > > > Sentry).
> > > > > > > > >> > > > > > > > >> > I remember KIP-11 described this, but I
> > > can't
> > > > > > find
> > > > > > > > the
> > > > > > > > >> KIP
> > > > > > > > >> > > any
> > > > > > > > >> > > > > > > longer.
> > > > > > > > >> > > > > > > > >> >
> > > > > > > > >> > > > > > > > >> > Regardless, I think KIP-4 focuses on
> > > getting
> > > > > > > > >> information
> > > > > > > > >> > > that
> > > > > > > > >> > > > > > > already
> > > > > > > > >> > > > > > > > >> > exists from Kafka brokers, not on
> adding
> > > > > > information
> > > > > > > > >> that
> > > > > > > > >> > > > > perhaps
> > > > > > > > >> > > > > > > > >> > should exist but doesn't yet?
> > > > > > > > >> > > > > > > > >> >
> > > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > > >> > > > > > > > >> >
> > > > > > > > >> > > > > > > > >> >
> > > > > > > > >> > > > > > > > >> >
> > > > > > > > >> > > > > > > > >> >
> > > > > > > > >> > > > > > > > >> >
> > > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM,
> Guozhang
> > > > Wang
> > > > > <
> > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > > >> > > > > > > > >> >>
> > > > > > > > >> > > > > > > > >> >> Just want to elaborate a bit more on
> the
> > > > > > > > create-topic
> > > > > > > > >> > > > metadata
> > > > > > > > >> > > > > > and
> > > > > > > > >> > > > > > > > >> batching
> > > > > > > > >> > > > > > > > >> >> describe-topic based on config /
> metadata
> > > in
> > > > > my
> > > > > > > > >> previous
> > > > > > > > >> > > > email
> > > > > > > > >> > > > > as
> > > > > > > > >> > > > > > > we
> > > > > > > > >> > > > > > > > >> work
> > > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The main motivation is
> to
> > > > have
> > > > > > some
> > > > > > > > >> sort of
> > > > > > > > >> > > > > topic
> > > > > > > > >> > > > > > > > >> management
> > > > > > > > >> > > > > > > > >> >> mechanisms, which I think is quite
> > > important
> > > > > in
> > > > > > a
> > > > > > > > >> > > > multi-tenant
> > > > > > > > >> > > > > /
> > > > > > > > >> > > > > > > > cloud
> > > > > > > > >> > > > > > > > >> >> architecture: today anyone can create
> > > topics
> > > > > in
> > > > > > a
> > > > > > > > >> shared
> > > > > > > > >> > > > Kafka
> > > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > > >> > > > > > > > >> >> there is no concept or "ownership" of
> > > topics
> > > > > > that
> > > > > > > > are
> > > > > > > > >> > > created
> > > > > > > > >> > > > > by
> > > > > > > > >> > > > > > > > >> different
> > > > > > > > >> > > > > > > > >> >> users. For example, at LinkedIn we
> > > basically
> > > > > > > > >> distinguish
> > > > > > > > >> > > > topic
> > > > > > > > >> > > > > > > owners
> > > > > > > > >> > > > > > > > >> via
> > > > > > > > >> > > > > > > > >> >> some casual topic name prefix, which
> is a
> > > > bit
> > > > > > > > awkward
> > > > > > > > >> and
> > > > > > > > >> > > > does
> > > > > > > > >> > > > > > not
> > > > > > > > >> > > > > > > > fly
> > > > > > > > >> > > > > > > > >> as
> > > > > > > > >> > > > > > > > >> >> we scale our customers. It would be
> great
> > > to
> > > > > use
> > > > > > > > >> > > > > describe-topics
> > > > > > > > >> > > > > > > such
> > > > > > > > >> > > > > > > > >> as:
> > > > > > > > >> > > > > > > > >> >>
> > > > > > > > >> > > > > > > > >> >> Describe all topics that is created
> by me.
> > > > > > > > >> > > > > > > > >> >>
> > > > > > > > >> > > > > > > > >> >> Describe all topics whose retention
> time
> > > is
> > > > > > > > overriden
> > > > > > > > >> to X.
> > > > > > > > >> > > > > > > > >> >>
> > > > > > > > >> > > > > > > > >> >> Describe all topics whose writable
> group
> > > > > include
> > > > > > > > user
> > > > > > > > >> Y
> > > > > > > > >> > > (this
> > > > > > > > >> > > > > is
> > > > > > > > >> > > > > > > > >> related to
> > > > > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > > > > >> > > > > > > > >> >>
> > > > > > > > >> > > > > > > > >> >> One possible way to achieve this is to
> > > add a
> > > > > > > > metadata
> > > > > > > > >> file
> > > > > > > > >> > > in
> > > > > > > > >> > > > > the
> > > > > > > > >> > > > > > > > >> >> create-topic request, whose value will
> > > also
> > > > be
> > > > > > > > >> written ZK
> > > > > > > > >> > > as
> > > > > > > > >> > > > we
> > > > > > > > >> > > > > > > > create
> > > > > > > > >> > > > > > > > >> the
> > > > > > > > >> > > > > > > > >> >> topic; then describe-topics can
> choose to
> > > > > batch
> > > > > > > > topics
> > > > > > > > >> > > based
> > > > > > > > >> > > > on
> > > > > > > > >> > > > > > 1)
> > > > > > > > >> > > > > > > > name
> > > > > > > > >> > > > > > > > >> >> regex, 2) config K-V matching, 3)
> metadata
> > > > > > regex,
> > > > > > > > etc.
> > > > > > > > >> > > > > > > > >> >>
> > > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > > >> > > > > > > > >> >>
> > > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > > >> > > > > > > > >> >>
> > > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM,
> Guozhang
> > > > Wang
> > > > > <
> > > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > >> > > > > > > > >> >>
> > > > > > > > >> > > > > > > > >> >>> Thanks for the updated wiki. A few
> > > comments
> > > > > > below:
> > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > >> > > > > > > > >> >>> 1. Error description in response: I
> think
> > > > if
> > > > > > some
> > > > > > > > >> > > errorCode
> > > > > > > > >> > > > > > could
> > > > > > > > >> > > > > > > > >> indicate
> > > > > > > > >> > > > > > > > >> >>> several different error cases then we
> > > > should
> > > > > > really
> > > > > > > > >> change
> > > > > > > > >> > > > it
> > > > > > > > >> > > > > to
> > > > > > > > >> > > > > > > > >> multiple
> > > > > > > > >> > > > > > > > >> >>> codes. In general the errorCode
> itself
> > > > would
> > > > > be
> > > > > > > > >> precise
> > > > > > > > >> > > and
> > > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > > >> > > > > > > > >> >>> describing the server side errors.
> > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > >> > > > > > > > >> >>> 2. Describe topic request: it would
> be
> > > > great
> > > > > > to go
> > > > > > > > >> beyond
> > > > > > > > >> > > > just
> > > > > > > > >> > > > > > > > >> batching on
> > > > > > > > >> > > > > > > > >> >>> topic name regex for this request.
> For
> > > > > > example, a
> > > > > > > > >> very
> > > > > > > > >> > > > common
> > > > > > > > >> > > > > > use
> > > > > > > > >> > > > > > > > >> case of
> > > > > > > > >> > > > > > > > >> >>> the topic command is to list all
> topics
> > > > whose
> > > > > > > > config
> > > > > > > > >> A's
> > > > > > > > >> > > > value
> > > > > > > > >> > > > > > is
> > > > > > > > >> > > > > > > B.
> > > > > > > > >> > > > > > > > >> With
> > > > > > > > >> > > > > > > > >> >>> topic name regex then we have to
> first
> > > > > retrieve
> > > > > > > > >> __all__
> > > > > > > > >> > > > > topics's
> > > > > > > > >> > > > > > > > >> >>> description info and then filter at
> the
> > > > > client
> > > > > > end,
> > > > > > > > >> which
> > > > > > > > >> > > > will
> > > > > > > > >> > > > > > be
> > > > > > > > >> > > > > > > a
> > > > > > > > >> > > > > > > > >> huge
> > > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this
> is
> > > > > > related to
> > > > > > > > >> the
> > > > > > > > >> > > > > previous
> > > > > > > > >> > > > > > > > point;
> > > > > > > > >> > > > > > > > >> >>> maybe we can add another metadata
> K-V or
> > > > > just a
> > > > > > > > >> metadata
> > > > > > > > >> > > > > string
> > > > > > > > >> > > > > > > > along
> > > > > > > > >> > > > > > > > >> side
> > > > > > > > >> > > > > > > > >> >>> with config K-V in create topic like
> we
> > > did
> > > > > for
> > > > > > > > >> offset
> > > > > > > > >> > > > commit
> > > > > > > > >> > > > > > > > >> request. This
> > > > > > > > >> > > > > > > > >> >>> field can be quite useful in storing
> > > > > > information
> > > > > > > > like
> > > > > > > > >> > > > "owner"
> > > > > > > > >> > > > > of
> > > > > > > > >> > > > > > > the
> > > > > > > > >> > > > > > > > >> topic
> > > > > > > > >> > > > > > > > >> >>> who issue the create command, etc,
> which
> > > is
> > > > > > quite
> > > > > > > > >> > > important
> > > > > > > > >> > > > > for
> > > > > > > > >> > > > > > a
> > > > > > > > >> > > > > > > > >> >>> multi-tenant setting. Then in the
> > > describe
> > > > > > topic
> > > > > > > > >> request
> > > > > > > > >> > > we
> > > > > > > > >> > > > > can
> > > > > > > > >> > > > > > > also
> > > > > > > > >> > > > > > > > >> batch
> > > > > > > > >> > > > > > > > >> >>> on regex of the metadata field.
> > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > >> > > > > > > > >> >>> 4. Today all the admin operations are
> > > async
> > > > > in
> > > > > > the
> > > > > > > > >> sense
> > > > > > > > >> > > > that
> > > > > > > > >> > > > > > > > command
> > > > > > > > >> > > > > > > > >> will
> > > > > > > > >> > > > > > > > >> >>> return once it is written in ZK, and
> that
> > > > is
> > > > > > why we
> > > > > > > > >> need
> > > > > > > > >> > > > extra
> > > > > > > > >> > > > > > > > >> verification
> > > > > > > > >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() /
> > > > verify
> > > > > > > > >> partition
> > > > > > > > >> > > > > > > reassignment
> > > > > > > > >> > > > > > > > >> >>> request, etc. With admin requests we
> > > could
> > > > > add
> > > > > > a
> > > > > > > > >> flag to
> > > > > > > > >> > > > > enable
> > > > > > > > >> > > > > > /
> > > > > > > > >> > > > > > > > >> disable
> > > > > > > > >> > > > > > > > >> >>> synchronous requests; when it is
> turned
> > > on,
> > > > > the
> > > > > > > > >> response
> > > > > > > > >> > > > will
> > > > > > > > >> > > > > > not
> > > > > > > > >> > > > > > > > >> return
> > > > > > > > >> > > > > > > > >> >>> until the request has been
> completed. And
> > > > for
> > > > > > async
> > > > > > > > >> > > requests
> > > > > > > > >> > > > > we
> > > > > > > > >> > > > > > > can
> > > > > > > > >> > > > > > > > >> add a
> > > > > > > > >> > > > > > > > >> >>> "token" field in the response, and
> then
> > > > only
> > > > > > need a
> > > > > > > > >> > > general
> > > > > > > > >> > > > > > "admin
> > > > > > > > >> > > > > > > > >> >>> verification request" with the given
> > > token
> > > > to
> > > > > > check
> > > > > > > > >> if the
> > > > > > > > >> > > > > async
> > > > > > > > >> > > > > > > > >> request
> > > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > >> > > > > > > > >> >>> 5. +1 for extending Metadata request
> to
> > > > > include
> > > > > > > > >> > > controller /
> > > > > > > > >> > > > > > > > >> coordinator
> > > > > > > > >> > > > > > > > >> >>> information, and then we can remove
> the
> > > > > > > > >> ConsumerMetadata /
> > > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel
> > > > Koshy <
> > > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > >> > > > > > > > >> >>>
> > > > > > > > >> > > > > > > > >> >>>> Thanks for sending that out Joe - I
> > > don't
> > > > > > think I
> > > > > > > > >> will be
> > > > > > > > >> > > > > able
> > > > > > > > >> > > > > > to
> > > > > > > > >> > > > > > > > >> make
> > > > > > > > >> > > > > > > > >> >>>> it today, so if notes can be sent
> out
> > > > > > afterward
> > > > > > > > that
> > > > > > > > >> > > would
> > > > > > > > >> > > > be
> > > > > > > > >> > > > > > > > great.
> > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM
> > > -0800,
> > > > > Gwen
> > > > > > > > >> Shapira
> > > > > > > > >> > > > wrote:
> > > > > > > > >> > > > > > > > >> >>>> > Thanks for sending this out Joe.
> > > Looking
> > > > > > forward
> > > > > > > > >> to
> > > > > > > > >> > > > > chatting
> > > > > > > > >> > > > > > > with
> > > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM,
> Joe
> > > > Stein
> > > > > <
> > > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > > >> > > > > > > > >> wrote:
> > > > > > > > >> > > > > > > > >> >>>> > > Hey, I just sent out a google
> > > hangout
> > > > > > invite
> > > > > > > > to
> > > > > > > > >> all
> > > > > > > > >> > > > pmc,
> > > > > > > > >> > > > > > > > >> committers
> > > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > > >> > > > > > > > >> >>>> > > everyone I found working on a
> KIP.
> > > If
> > > > I
> > > > > > missed
> > > > > > > > >> anyone
> > > > > > > > >> > > > in
> > > > > > > > >> > > > > > the
> > > > > > > > >> > > > > > > > >> invite
> > > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > > >> > > > > > > > >> >>>> > > let me know and can update it,
> np.
> > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > >> > > > > > > > >> >>>> > > We should do this every Tuesday
> @
> > > 2pm
> > > > > > Eastern
> > > > > > > > >> Time.
> > > > > > > > >> > > > Maybe
> > > > > > > > >> > > > > > we
> > > > > > > > >> > > > > > > > can
> > > > > > > > >> > > > > > > > >> get
> > > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > > >> > > > > > > > >> >>>> > > help to make a google account
> so we
> > > > can
> > > > > > manage
> > > > > > > > >> > > better?
> > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > >> > > > > > > > >>
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >>
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > >> > > > > > > > >> >>>> > > in progress and related JIRA
> that
> > > are
> > > > > > > > >> interdependent
> > > > > > > > >> > > > and
> > > > > > > > >> > > > > > > common
> > > > > > > > >> > > > > > > > >> work.
> > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59
> PM, Jay
> > > > > > Kreps <
> > > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts
> that
> > > > will
> > > > > > also
> > > > > > > > >> record
> > > > > > > > >> > > > and
> > > > > > > > >> > > > > > make
> > > > > > > > >> > > > > > > > the
> > > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49
> AM,
> > > > Jeff
> > > > > > > > Holoman
> > > > > > > > >> <
> > > > > > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > >> > > > > > > > >> >>>> > >> > We're happy to send out a
> Webex
> > > for
> > > > > > this
> > > > > > > > >> purpose.
> > > > > > > > >> > > We
> > > > > > > > >> > > > > > could
> > > > > > > > >> > > > > > > > >> record
> > > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > > >> > > > > > > > >> >>>> > >> > sessions if there is
> interest and
> > > > > > publish
> > > > > > > > >> them
> > > > > > > > >> > > out.
> > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at
> 11:28 AM,
> > > > Jay
> > > > > > > > Kreps <
> > > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try to get the
> technical
> > > > > > hang-ups
> > > > > > > > >> sorted
> > > > > > > > >> > > > out,
> > > > > > > > >> > > > > > > > though.
> > > > > > > > >> > > > > > > > >> I
> > > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > > >> > > > > > > > >> >>>> > >> > > there is some benefit to
> live
> > > > > > discussion
> > > > > > > > vs
> > > > > > > > >> > > > > writing. I
> > > > > > > > >> > > > > > > am
> > > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > > >> > > > > > > > >> >>>> > >> > > we post instructions and
> give
> > > > > > ourselves a
> > > > > > > > >> few
> > > > > > > > >> > > > > attempts
> > > > > > > > >> > > > > > > we
> > > > > > > > >> > > > > > > > >> can
> > > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that time would
> work
> > > > for
> > > > > > > > >> me...any
> > > > > > > > >> > > > > > objections?
> > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at
> 8:18
> > > AM,
> > > > > Joe
> > > > > > > > Stein
> > > > > > > > >> <
> > > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly would be great
> maybe
> > > > like
> > > > > > every
> > > > > > > > >> > > Tuesday ~
> > > > > > > > >> > > > > 1pm
> > > > > > > > >> > > > > > > ET
> > > > > > > > >> > > > > > > > /
> > > > > > > > >> > > > > > > > >> 10am
> > > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't mind google
> hangout
> > > but
> > > > > > there
> > > > > > > > is
> > > > > > > > >> > > always
> > > > > > > > >> > > > > some
> > > > > > > > >> > > > > > > > >> issue or
> > > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > we know the apache irc
> > > channel
> > > > > > works.
> > > > > > > > We
> > > > > > > > >> can
> > > > > > > > >> > > > start
> > > > > > > > >> > > > > > > there
> > > > > > > > >> > > > > > > > >> and
> > > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull
> transcripts
> > > > too
> > > > > > and
> > > > > > > > >> > > associate
> > > > > > > > >> > > > to
> > > > > > > > >> > > > > > > > >> tickets if
> > > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at
> 11:10
> > > > AM,
> > > > > > Jay
> > > > > > > > >> Kreps <
> > > > > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked about
> doing a
> > > > > Google
> > > > > > > > >> Hangout to
> > > > > > > > >> > > > chat
> > > > > > > > >> > > > > > > about
> > > > > > > > >> > > > > > > > >> this.
> > > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > generalizing that a
> little
> > > > > > > > further...I
> > > > > > > > >> > > > actually
> > > > > > > > >> > > > > > > think
> > > > > > > > >> > > > > > > > it
> > > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > everyone spending a
> > > > reasonable
> > > > > > chunk
> > > > > > > > of
> > > > > > > > >> > > their
> > > > > > > > >> > > > > week
> > > > > > > > >> > > > > > > on
> > > > > > > > >> > > > > > > > >> Kafka
> > > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I
> > > think
> > > > we
> > > > > > could
> > > > > > > > >> use
> > > > > > > > >> > > time
> > > > > > > > >> > > > > to
> > > > > > > > >> > > > > > > talk
> > > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we
> are on
> > > > top
> > > > > of
> > > > > > > > code
> > > > > > > > >> > > > reviews,
> > > > > > > > >> > > > > > talk
> > > > > > > > >> > > > > > > > >> through
> > > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > We can make it publicly
> > > > > > available so
> > > > > > > > >> that
> > > > > > > > >> > > any
> > > > > > > > >> > > > > one
> > > > > > > > >> > > > > > > can
> > > > > > > > >> > > > > > > > >> follow
> > > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any interest in doing
> this?
> > > > If
> > > > > so
> > > > > > > > I'll
> > > > > > > > >> try
> > > > > > > > >> > > to
> > > > > > > > >> > > > > set
> > > > > > > > >> > > > > > it
> > > > > > > > >> > > > > > > > up
> > > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at
> > > 3:57
> > > > > AM,
> > > > > > > > Andrii
> > > > > > > > >> > > > > Biletskyi
> > > > > > > > >> > > > > > <
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > andrii.biletskyi@stealth.ly>
> > > > > > wrote:
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP
> page,
> > > > fixed
> > > > > /
> > > > > > > > >> aligned
> > > > > > > > >> > > > > document
> > > > > > > > >> > > > > > > > >> structure.
> > > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > very initial
> proposal for
> > > > > > > > >> AdminClient so
> > > > > > > > >> > > we
> > > > > > > > >> > > > > have
> > > > > > > > >> > > > > > > > >> something
> > > > > > > > >> > > > > > > > >> >>>> to
> > > > > > > > >> > > > > > > > >> >>>> > >> start
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > from
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > while
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > > >> > > > > > > > >> >>>>
> > > > > > > > >> > > > > > > > >>
> > > > > > > > >> > > > > > > >
> > > > > > > > >> > > > > > >
> > > > > > > > >> > > > > >
> > > > > > > > >> > > > >
> > > > > > > > >> > > >
> > > > > > > > >> > >
> > > > > > > > >>
> > > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015
> at
> > > > 9:01
> > > > > > PM,
> > > > > > > > >> Andrii
> > > > > > > > >> > > > > > Biletskyi
> > > > > > > > >> > > > > > > <
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > andrii.biletskyi@stealth.ly>
> > > > > > > > wrote:
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re error messages:
> you
> > > > are
> > > > > > right,
> > > > > > > > >> in
> > > > > > > > >> > > most
> > > > > > > > >> > > > > > cases
> > > > > > > > >> > > > > > > > >> client
> > > > > > > > >> > > > > > > > >> >>>> will
> > > > > > > > >> > > > > > > > >> >>>> > >> have
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > enough
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > context to show
> > > > descriptive
> > > > > > error
> > > > > > > > >> > > message.
> > > > > > > > >> > > > > My
> > > > > > > > >> > > > > > > > >> concern is
> > > > > > > > >> > > > > > > > >> >>>> that
> > > > > > > > >> > > > > > > > >> >>>> > >> we
> > > > > > > > >> > > > > > > > >> >>>> > >> > > will
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > have
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > to
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > add lots of new
> error
> > > > codes
> > > > > > for
> > > > > > > > >> each
> > > > > > > > >> > > > > possible
> > > > > > > > >> > > > > > > > >> error. Of
> > > > > > > > >> > > > > > > > >> >>>> course,
> > > > > > > > >> > > > > > > > >> >>>> > >> > we
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > could
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > reuse
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > some of existing
> like
> > > > > > > > >> > > > > > > UknownTopicOrPartitionCode,
> > > > > > > > >> > > > > > > > >> but we
> > > > > > > > >> > > > > > > > >> >>>> will
> > > > > > > > >> > > > > > > > >> >>>> > >> > also
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > need
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > to
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> > > > > > > > >> TopicAlreadyExistsCode,
> > > > > > > > >> > > > > > > > >> >>>> TopicConfigInvalid (both
> > > > > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > topic
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > name and config,
> and
> > > > > probably
> > > > > > > > user
> > > > > > > > >> would
> > > > > > > > >> > > > > like
> > > > > > > > >> > > > > > to
> > > > > > > > >> > > > > > > > >> know
> > > > > > > > >> > > > > > > > >> >>>> what
> > > > > > > > >> > > > > > > > >> >>>> > >> > exactly
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his
> > > config),
> > > > > > > > >> > > > > > > InvalidReplicaAssignment,
> > > > > > > > >> > > > > > > > >> >>>> > >> InternalError
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > (e.g.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure)
> etc.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > And this is only
> for
> > > > > > > > TopicCommand,
> > > > > > > > >> we
> > > > > > > > >> > > will
> > > > > > > > >> > > > > > also
> > > > > > > > >> > > > > > > > >> need to
> > > > > > > > >> > > > > > > > >> >>>> add
> > > > > > > > >> > > > > > > > >> >>>> > >> > similar
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > for
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> > > > > > > > >> PreferredReplica. So
> > > > > > > > >> > > > > we'll
> > > > > > > > >> > > > > > > end
> > > > > > > > >> > > > > > > > >> up
> > > > > > > > >> > > > > > > > >> >>>> with a
> > > > > > > > >> > > > > > > > >> >>>> > >> > large
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > list
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > of
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > error codes, used
> only
> > > in
> > > > > > Admin
> > > > > > > > >> > > protocol.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I
> > > agree
> > > > > my
> > > > > > > > >> proposal is
> > > > > > > > >> > > > not
> > > > > > > > >> > > > > > > > >> consistent
> > > > > > > > >> > > > > > > > >> >>>> with
> > > > > > > > >> > > > > > > > >> >>>> > >> > other
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > cases.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find
> > > better
> > > > > > solution
> > > > > > > > >> or
> > > > > > > > >> > > > > something
> > > > > > > > >> > > > > > > > >> >>>> in-between.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I
> > > think
> > > > it
> > > > > > is a
> > > > > > > > >> great
> > > > > > > > >> > > > idea.
> > > > > > > > >> > > > > > > This
> > > > > > > > >> > > > > > > > >> way we
> > > > > > > > >> > > > > > > > >> >>>> can
> > > > > > > > >> > > > > > > > >> >>>> > >> move
> > > > > > > > >> > > > > > > > >> >>>> > >> > > on
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > faster.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Let's agree
> somehow on
> > > > > > date/time
> > > > > > > > so
> > > > > > > > >> > > people
> > > > > > > > >> > > > > can
> > > > > > > > >> > > > > > > > join.
> > > > > > > > >> > > > > > > > >> >>>> Will work
> > > > > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > > > > >> > > > > > > > >> >>>> > >> > > me
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > and
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > next week almost
> > > anytime
> > > > if
> > > > > > > > agreed
> > > > > > > > >> in
> > > > > > > > >> > > > > advance.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18,
> 2015 at
> > > > > 7:09
> > > > > > PM,
> > > > > > > > >> Jay
> > > > > > > > >> > > > Kreps <
> > > > > > > > >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > wrote:
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can
> do
> > > good
> > > > > > error
> > > > > > > > >> handling
> > > > > > > > >> > > > > > without
> > > > > > > > >> > > > > > > > >> needing
> > > > > > > > >> > > > > > > > >> >>>> custom
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > server-side
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e.
> > > generally
> > > > > the
> > > > > > > > >> client has
> > > > > > > > >> > > > the
> > > > > > > > >> > > > > > > > >> context to
> > > > > > > > >> > > > > > > > >> >>>> know
> > > > > > > > >> > > > > > > > >> >>>> > >> that
> > > > > > > > >> > > > > > > > >> >>>> > >> > > if
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > it
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > got
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> an error that the
> > > topic
> > > > > > doesn't
> > > > > > > > >> exist
> > > > > > > > >> > > to
> > > > > > > > >> > > > > say
> > > > > > > > >> > > > > > > > >> "Topic X
> > > > > > > > >> > > > > > > > >> >>>> doesn't
> > > > > > > > >> > > > > > > > >> >>>> > >> > > exist"
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> rather
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> than "error code
> 14"
> > > (or
> > > > > > > > >> whatever).
> > > > > > > > >> > > Maybe
> > > > > > > > >> > > > > > there
> > > > > > > > >> > > > > > > > are
> > > > > > > > >> > > > > > > > >> >>>> specific
> > > > > > > > >> > > > > > > > >> >>>> > >> > cases
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > where
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If
> we
> > > want
> > > > > to
> > > > > > add
> > > > > > > > >> > > > server-side
> > > > > > > > >> > > > > > > error
> > > > > > > > >> > > > > > > > >> >>>> messages we
> > > > > > > > >> > > > > > > > >> >>>> > >> > > really
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > do
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> need to do this
> in a
> > > > > > consistent
> > > > > > > > >> way
> > > > > > > > >> > > > across
> > > > > > > > >> > > > > > the
> > > > > > > > >> > > > > > > > >> protocol.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> I still have a
> bunch
> > > of
> > > > > open
> > > > > > > > >> questions
> > > > > > > > >> > > > here
> > > > > > > > >> > > > > > > from
> > > > > > > > >> > > > > > > > my
> > > > > > > > >> > > > > > > > >> >>>> previous
> > > > > > > > >> > > > > > > > >> >>>> > >> > > list. I
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > will
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> be out for the
> next
> > > few
> > > > > > days for
> > > > > > > > >> Strata
> > > > > > > > >> > > > > > though.
> > > > > > > > >> > > > > > > > >> Maybe
> > > > > > > > >> > > > > > > > >> >>>> we could
> > > > > > > > >> > > > > > > > >> >>>> > >> > do
> > > > > > > > >> > > > > > > > >> >>>> > >> > > a
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Google
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on
> any
> > > open
> > > > > > issues
> > > > > > > > >> some
> > > > > > > > >> > > time
> > > > > > > > >> > > > > > > towards
> > > > > > > > >> > > > > > > > >> the
> > > > > > > > >> > > > > > > > >> >>>> end of
> > > > > > > > >> > > > > > > > >> >>>> > >> > next
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > week
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > for
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested
> in
> > > > this
> > > > > > > > ticket?
> > > > > > > > >> I
> > > > > > > > >> > > have
> > > > > > > > >> > > > a
> > > > > > > > >> > > > > > > > feeling
> > > > > > > > >> > > > > > > > >> that
> > > > > > > > >> > > > > > > > >> >>>> might
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > progress
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> things a little
> faster
> > > > > than
> > > > > > > > >> email--I
> > > > > > > > >> > > > think
> > > > > > > > >> > > > > we
> > > > > > > > >> > > > > > > > >> could talk
> > > > > > > > >> > > > > > > > >> >>>> > >> through
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > those
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought
> up
> > > > fairly
> > > > > > > > >> quickly...
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18,
> 2015
> > > at
> > > > > > 7:27 AM,
> > > > > > > > >> Andrii
> > > > > > > > >> > > > > > > > Biletskyi <
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > andrii.biletskyi@stealth.ly
> > > > > > >
> > > > > > > > >> wrote:
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to
> > > address
> > > > > > some of
> > > > > > > > >> the
> > > > > > > > >> > > > issues
> > > > > > > > >> > > > > > > which
> > > > > > > > >> > > > > > > > >> were
> > > > > > > > >> > > > > > > > >> >>>> > >> mentioned
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > earlier
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> about
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP
> format.
> > > > One
> > > > > of
> > > > > > > > >> those was
> > > > > > > > >> > > > > about
> > > > > > > > >> > > > > > > > >> batching
> > > > > > > > >> > > > > > > > >> >>>> > >> > operations.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > What
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > if
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> we
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > follow
> TopicCommand
> > > > > > approach
> > > > > > > > >> and let
> > > > > > > > >> > > > > people
> > > > > > > > >> > > > > > > > >> specify
> > > > > > > > >> > > > > > > > >> >>>> > >> topic-name
> > > > > > > > >> > > > > > > > >> >>>> > >> > > by
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover
> > > most
> > > > of
> > > > > > the
> > > > > > > > use
> > > > > > > > >> > > cases?
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is
> what
> > > > > > information
> > > > > > > > >> should
> > > > > > > > >> > > we
> > > > > > > > >> > > > > > > > generally
> > > > > > > > >> > > > > > > > >> >>>> provide in
> > > > > > > > >> > > > > > > > >> >>>> > >> > > Admin
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that
> Admin
> > > > > > commands
> > > > > > > > >> don't
> > > > > > > > >> > > > imply
> > > > > > > > >> > > > > > > they
> > > > > > > > >> > > > > > > > >> will
> > > > > > > > >> > > > > > > > >> >>>> be used
> > > > > > > > >> > > > > > > > >> >>>> > >> > only
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > in
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > CLI
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > but,
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me,
> CLI
> > > > is a
> > > > > > very
> > > > > > > > >> > > important
> > > > > > > > >> > > > > > > client
> > > > > > > > >> > > > > > > > >> of this
> > > > > > > > >> > > > > > > > >> >>>> > >> > feature.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > In
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > case,
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical,
> we
> > > > would
> > > > > > like
> > > > > > > > to
> > > > > > > > >> > > provide
> > > > > > > > >> > > > > > users
> > > > > > > > >> > > > > > > > >> with
> > > > > > > > >> > > > > > > > >> >>>> rich
> > > > > > > > >> > > > > > > > >> >>>> > >> > > experience
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > in
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> terms
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > of
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > getting results
> /
> > > > errors
> > > > > > of
> > > > > > > > the
> > > > > > > > >> > > > executed
> > > > > > > > >> > > > > > > > >> commands.
> > > > > > > > >> > > > > > > > >> >>>> Usually
> > > > > > > > >> > > > > > > > >> >>>> > >> we
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > supply
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> with
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses only
> > > > > errorCode,
> > > > > > > > which
> > > > > > > > >> looks
> > > > > > > > >> > > > > very
> > > > > > > > >> > > > > > > > >> limiting,
> > > > > > > > >> > > > > > > > >> >>>> in case
> > > > > > > > >> > > > > > > > >> >>>> > >> > of
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > CLI
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > we
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> may
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > want to print
> human
> > > > > > readable
> > > > > > > > >> error
> > > > > > > > >> > > > > > > description.
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into
> > > > account
> > > > > > > > >> previous item
> > > > > > > > >> > > > > about
> > > > > > > > >> > > > > > > > >> batching,
> > > > > > > > >> > > > > > > > >> >>>> what
> > > > > > > > >> > > > > > > > >> >>>> > >> do
> > > > > > > > >> > > > > > > > >> >>>> > >> > > you
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > think
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > about
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > having smth
> like:
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > ('create'
> doesn't
> > > > > support
> > > > > > > > >> regexp)
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> CreateTopicRequest
> > > =>
> > > > > > > > TopicName
> > > > > > > > >> > > > > Partitions
> > > > > > > > >> > > > > > > > >> Replicas
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> CreateTopicResponse
> > > =>
> > > > > > > > ErrorCode
> > > > > > > > >> > > > > > > > ErrorDescription
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode =>
> int16
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
>  ErrorDescription
> > > =>
> > > > > > string
> > > > > > > > >> (empty
> > > > > > > > >> > > if
> > > > > > > > >> > > > > > > > >> successful)
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> AlterTopicRequest ->
> > > > > > > > >> TopicNameRegexp
> > > > > > > > >> > > >
> > > > > > > > >
> > > > > > > > > ...
> > > > > > > > >
> > > > > > > > > [Message clipped]
> > > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
>
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Joel Koshy <jj...@gmail.com>.
+1 as well. I think it helps to keep the rerouting approach orthogonal
to this KIP.

On Wed, Mar 18, 2015 at 03:40:48PM -0700, Jay Kreps wrote:
> I'm +1 on Jun's suggestion as long as it can work for all the requests.
> 
> On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <ju...@confluent.io> wrote:
> 
> > Andrii,
> >
> > I think we agreed on the following.
> >
> > (a) Admin requests can be sent to and handled by any broker.
> > (b) Admin requests are processed asynchronously, at least for now. That is,
> > when the client gets a response, it just means that the request is
> > initiated, but not necessarily completed. Then, it's up to the client to
> > issue another request to check the status for completion.
> >
> > To support (a), we were thinking of doing request forwarding to the
> > controller (utilizing KAFKA-1912). I am making an alternative proposal.
> > Basically, the broker can just write to ZooKeeper to inform the controller
> > about the request. For example, to handle partitionReassignment, the broker
> > will just write the requested partitions to /admin/reassign_partitions
> > (like what AdminUtils currently does) and then send a response to the
> > client. This shouldn't take long and the implementation will be simpler
> > than forwarding the requests to the controller through RPC.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Jun,
> > >
> > > I might be wrong but didn't we agree we will let any broker from the
> > > cluster handle *long-running* admin requests (at this time
> > preferredReplica
> > > and
> > > reassignPartitions), via zk admin path. Thus CreateTopics etc should be
> > > sent
> > > only to the controller.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Joel, Andril,
> > > >
> > > > I think we agreed that those admin requests can be issued to any
> > broker.
> > > > Because of that, there doesn't seem to be a strong need to know the
> > > > controller. So, perhaps we can proceed by not making any change to the
> > > > format of TMR right now. When we start using create topic request in
> > the
> > > > producer, we will need a new version of TMR that doesn't trigger auto
> > > topic
> > > > creation. But that can be done later.
> > > >
> > > > As a first cut implementation, I think the broker can just write to ZK
> > > > directly for
> > > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > > requests, instead of forwarding them to the controller. This will
> > > simplify
> > > > the implementation on the broker side.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <jj...@gmail.com>
> > > wrote:
> > > >
> > > > > (Thanks Andrii for the summary)
> > > > >
> > > > > For (1) yes we will circle back on that shortly after syncing up in
> > > > > person. I think it is close to getting committed although development
> > > > > for KAFKA-1927 can probably begin without it.
> > > > >
> > > > > There is one more item we covered at the hangout. i.e., whether we
> > > > > want to add the coordinator to the topic metadata response or provide
> > > > > a clearer ClusterMetadataRequest.
> > > > >
> > > > > There are two reasons I think we should try and avoid adding the
> > > > > field:
> > > > > - It is irrelevant to topic metadata
> > > > > - If we finally do request rerouting in Kafka then the field would
> > add
> > > > >   little to no value. (It still helps to have a separate
> > > > >   ClusterMetadataRequest to query for cluster-wide information such
> > as
> > > > >   'which broker is the controller?' as Joe mentioned.)
> > > > >
> > > > > I think it would be cleaner to have an explicit
> > ClusterMetadataRequest
> > > > > that you can send to any broker in order to obtain the controller
> > (and
> > > > > in the future possibly other cluster-wide information). I think the
> > > > > main argument against doing this and instead adding it to the topic
> > > > > metadata response was convenience - i.e., you don't have to discover
> > > > > the controller in advance. However, I don't see much actual
> > > > > benefit/convenience in this and in fact think it is a non-issue. Let
> > > > > me know if I'm overlooking something here.
> > > > >
> > > > > As an example, say we need to initiate partition reassignment by
> > > > > issuing the new ReassignPartitionsRequest to the controller (assume
> > we
> > > > > already have the desired manual partition assignment).  If we are to
> > > > > augment topic metadata response then the flow be something like this
> > :
> > > > >
> > > > > - Issue topic metadata request to any broker (and discover the
> > > > >   controller
> > > > > - Connect to controller if required (i.e., if the broker above !=
> > > > >   controller)
> > > > > - Issue the partition reassignment request to the controller.
> > > > >
> > > > > With an explicit cluster metadata request it would be:
> > > > > - Issue cluster metadata request to any broker
> > > > > - Connect to controller if required (i.e., if the broker above !=
> > > > >   controller)
> > > > > - Issue the partition reassignment request
> > > > >
> > > > > So it seems to add little practical value and bloats topic metadata
> > > > > response with an irrelevant detail.
> > > > >
> > > > > The other angle to this is the following - is it a matter of naming?
> > > > > Should we just rename topic metadata request/response to just
> > > > > MetadataRequest/Response and add cluster metadata to it? By that same
> > > > > token should we also allow querying for the consumer coordinator (and
> > > > > in future transaction coordinator) as well? This leads to a bloated
> > > > > request which isn't very appealing and altogether confusing.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Joel
> > > > >
> > > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > > > > > Andri,
> > > > > >
> > > > > > Thanks for the summary.
> > > > > >
> > > > > > 1. I just realized that in order to start working on KAFKA-1927, we
> > > > will
> > > > > > need to merge the changes to OffsetCommitRequest (from 0.8.2) to
> > > trunk.
> > > > > > This is planned to be done as part of KAFKA-1634. So, we will need
> > > > > Guozhang
> > > > > > and Joel's help to wrap this up.
> > > > > >
> > > > > > 2. Thinking about this a bit more, if the semantic of those "write"
> > > > > > requests is async (i.e., after the client gets a response, it just
> > > > means
> > > > > > that the operation is initiated, but not necessarily completed), we
> > > > don't
> > > > > > really need to forward the requests to the controller. Instead, the
> > > > > > receiving broker can just write the operation to ZK as the admin
> > > > command
> > > > > > line tool previously does. This will simplify the implementation.
> > > > > >
> > > > > > 8. There is another implementation detail for describe topic.
> > > Ideally,
> > > > we
> > > > > > want to read the topic config from the broker cache, instead of
> > > > > ZooKeeper.
> > > > > > Currently, every broker reads the topic-level config for all
> > topics.
> > > > > > However, it ignores those for topics not hosted on itself. So, we
> > may
> > > > > need
> > > > > > to change TopicConfigManager a bit so that it caches the configs
> > for
> > > > all
> > > > > > topics.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Guys,
> > > > > > >
> > > > > > > Thanks for a great discussion!
> > > > > > > Here are the actions points:
> > > > > > >
> > > > > > > 1. Q: Get rid of all scala requests objects, use java protocol
> > > > > definitions.
> > > > > > >     A: Gwen kindly took that (KAFKA-1927). It's important to
> > speed
> > > up
> > > > > > > review procedure
> > > > > > >          there since this ticket blocks other important changes.
> > > > > > >
> > > > > > > 2. Q: Generic re-reroute facility vs client maintaining cluster
> > > > state.
> > > > > > >     A: Jay has added pseudo code to KAFKA-1912 - need to consider
> > > > > whether
> > > > > > > this will be
> > > > > > >         easy to implement as a server-side feature (comments are
> > > > > > > welcomed!).
> > > > > > >
> > > > > > > 3. Q: Controller field in wire protocol.
> > > > > > >     A: This might be useful for clients, add this to
> > > > > TopicMetadataResponse
> > > > > > > (already in KIP).
> > > > > > >
> > > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > > >     A: I will add proposed by Jun solution (using clientId for
> > > that)
> > > > > to the
> > > > > > > KIP.
> > > > > > >
> > > > > > > 5. Q: Bumping new versions of TMR vs grabbing all protocol
> > changes
> > > in
> > > > > one
> > > > > > > version.
> > > > > > >     A: It was decided to try to gather all changes to protocol
> > > > (before
> > > > > > > release).
> > > > > > >         In case of TMR it worth checking: KAFKA-2020 and KIP-13
> > > > > (quotas)
> > > > > > >
> > > > > > > 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
> > > > > > >     A: Use jackson for that, /tools project is a separate jar so
> > > > > shouldn't
> > > > > > > be a big deal.
> > > > > > >
> > > > > > > 7.  Q: VerifyReassingPartitions vs generic status check command.
> > > > > > >      A: For long-running requests like reassign partitions
> > > *progress*
> > > > > check
> > > > > > > request is useful,
> > > > > > >          it makes sense to introduce it.
> > > > > > >
> > > > > > >  Please add, correct me if I missed something.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >
> > > > > > > > Joel,
> > > > > > > >
> > > > > > > > You are right, I removed ClusterMetadata because we have
> > > partially
> > > > > > > > what we need in TopicMetadata. Also, as Jay pointed out
> > earlier,
> > > we
> > > > > > > > would like to have "orthogonal" API, but at the same time we
> > need
> > > > > > > > to be backward compatible.
> > > > > > > >
> > > > > > > > But I like your idea and even have some other arguments for
> > this
> > > > > option:
> > > > > > > > There is also DescribeTopicRequest which was proposed in this
> > > KIP,
> > > > > > > > it returns topic configs, partitions, replication factor plus
> > > > > partition
> > > > > > > > ISR, ASR,
> > > > > > > > leader replica. The later part is really already there in
> > > > > > > > TopicMetadataRequest.
> > > > > > > > So again we'll have to add stuff to TMR, not to duplicate some
> > > info
> > > > > in
> > > > > > > > newly added requests. However, this way we'll end up with
> > > "monster"
> > > > > > > > request which returns cluster metadata, topic replication and
> > > > config
> > > > > info
> > > > > > > > plus partition replication data. Seems logical to split TMR to
> > > > > > > > - ClusterMetadata (brokers + controller, maybe smth else)
> > > > > > > > - TopicMetadata (topic info + partition details)
> > > > > > > > But since current TMR is involved in lots of places (including
> > > > > network
> > > > > > > > client,
> > > > > > > > as I understand) this might be very serious change and it
> > > probably
> > > > > makes
> > > > > > > > sense to stick with current approach.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Andrii Biletskyi
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <
> > jjkoshy.w@gmail.com
> > > >
> > > > > wrote:
> > > > > > > >
> > > > > > > >> I may be missing some context but hopefully this will also be
> > > > > covered
> > > > > > > >> today: I thought the earlier proposal where there was an
> > > explicit
> > > > > > > >> ClusterMetadata request was clearer and explicit. During the
> > > > course
> > > > > of
> > > > > > > >> this thread I think the conclusion was that the main need was
> > > for
> > > > > > > >> controller information and that can be rolled into the topic
> > > > > metadata
> > > > > > > >> response but that seems a bit irrelevant to topic metadata.
> > > FWIW I
> > > > > > > >> think the full broker-list is also irrelevant to topic
> > metadata,
> > > > but
> > > > > > > >> it is already there and in use. I think there is still room
> > for
> > > an
> > > > > > > >> explicit ClusterMetadata request since there may be other
> > > > > > > >> cluster-level information that we may want to add over time
> > (and
> > > > > that
> > > > > > > >> have nothing to do with topic metadata).
> > > > > > > >>
> > > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi
> > > wrote:
> > > > > > > >> > Jun,
> > > > > > > >> >
> > > > > > > >> > 101. Okay, if you say that such use case is important. I
> > also
> > > > > think
> > > > > > > >> > using clientId for these purposes is fine - if we already
> > have
> > > > > this
> > > > > > > >> field
> > > > > > > >> > as part of all Wire protocol messages, why not use that.
> > > > > > > >> > I will update KIP-4 page if nobody has other ideas (which
> > may
> > > > > come up
> > > > > > > >> > during the call today).
> > > > > > > >> >
> > > > > > > >> > 102.1 Agree, I'll update the KIP accordingly. I think we can
> > > add
> > > > > new,
> > > > > > > >> > fine-grained error codes if some error code received in
> > > specific
> > > > > case
> > > > > > > >> > won't give enough context to return a descriptive error
> > > message
> > > > > for
> > > > > > > >> user.
> > > > > > > >> >
> > > > > > > >> > Look forward to discussing all outstanding issues in detail
> > > > today
> > > > > > > during
> > > > > > > >> > the call.
> > > > > > > >> >
> > > > > > > >> > Thanks,
> > > > > > > >> > Andrii Biletskyi
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <jun@confluent.io
> > >
> > > > > wrote:
> > > > > > > >> >
> > > > > > > >> > > 101. There may be a use case where you only want the
> > topics
> > > to
> > > > > be
> > > > > > > >> created
> > > > > > > >> > > manually by admins. Currently, you can do that by
> > disabling
> > > > auto
> > > > > > > topic
> > > > > > > >> > > creation and issue topic creation from the TopicCommand.
> > If
> > > we
> > > > > > > >> disable auto
> > > > > > > >> > > topic creation completely on the broker and don't have a
> > way
> > > > to
> > > > > > > >> distinguish
> > > > > > > >> > > between topic creation requests from the regular clients
> > and
> > > > the
> > > > > > > >> admin, we
> > > > > > > >> > > can't support manual topic creation any more. I was
> > thinking
> > > > > that
> > > > > > > >> another
> > > > > > > >> > > way of distinguishing the clients making the topic
> > creation
> > > > > requests
> > > > > > > >> is
> > > > > > > >> > > using clientId. For example, the admin tool can set it to
> > > > > something
> > > > > > > >> like
> > > > > > > >> > > admin and the broker can treat that clientId specially.
> > > > > > > >> > >
> > > > > > > >> > > Also, there is a related discussion in KAFKA-2020.
> > > Currently,
> > > > > we do
> > > > > > > >> the
> > > > > > > >> > > following in TopicMetadataResponse:
> > > > > > > >> > >
> > > > > > > >> > > 1. If leader is not available, we set the partition level
> > > > error
> > > > > code
> > > > > > > >> to
> > > > > > > >> > > LeaderNotAvailable.
> > > > > > > >> > > 2. If a non-leader replica is not available, we take that
> > > > > replica
> > > > > > > out
> > > > > > > >> of
> > > > > > > >> > > the assigned replica list and isr in the response. As an
> > > > > indication
> > > > > > > >> for
> > > > > > > >> > > doing that, we set the partition level error code to
> > > > > > > >> ReplicaNotAvailable.
> > > > > > > >> > >
> > > > > > > >> > > This has a few problems. First, ReplicaNotAvailable
> > probably
> > > > > > > >> shouldn't be
> > > > > > > >> > > an error, at least for the normal producer/consumer
> > clients
> > > > that
> > > > > > > just
> > > > > > > >> want
> > > > > > > >> > > to find out the leader. Second, it can happen that both
> > the
> > > > > leader
> > > > > > > and
> > > > > > > >> > > another replica are not available at the same time. There
> > is
> > > > no
> > > > > > > error
> > > > > > > >> code
> > > > > > > >> > > to indicate both. Third, even if a replica is not
> > available,
> > > > > it's
> > > > > > > >> still
> > > > > > > >> > > useful to return its replica id since some clients (e.g.
> > > admin
> > > > > tool)
> > > > > > > >> may
> > > > > > > >> > > still make use of it.
> > > > > > > >> > >
> > > > > > > >> > > One way to address this issue is to always return the
> > > replica
> > > > > id for
> > > > > > > >> > > leader, assigned replicas, and isr regardless of whether
> > the
> > > > > > > >> corresponding
> > > > > > > >> > > broker is live or not. Since we also return the list of
> > live
> > > > > > > brokers,
> > > > > > > >> the
> > > > > > > >> > > client can figure out whether a leader or a replica is
> > live
> > > or
> > > > > not
> > > > > > > >> and act
> > > > > > > >> > > accordingly. This way, we don't need to set the partition
> > > > level
> > > > > > > error
> > > > > > > >> code
> > > > > > > >> > > when the leader or a replica is not available. This
> > doesn't
> > > > > change
> > > > > > > >> the wire
> > > > > > > >> > > protocol, but does change the semantics. Since we are
> > > evolving
> > > > > the
> > > > > > > >> protocol
> > > > > > > >> > > of TopicMetadataRequest here, we can potentially piggyback
> > > the
> > > > > > > change.
> > > > > > > >> > >
> > > > > > > >> > > 102.1 For those types of errors due to invalid input,
> > > > shouldn't
> > > > > we
> > > > > > > >> just
> > > > > > > >> > > guard it at parameter validation time and throw
> > > > > > > >> InvalidArgumentException
> > > > > > > >> > > without even sending the request to the broker?
> > > > > > > >> > >
> > > > > > > >> > > Thanks,
> > > > > > > >> > >
> > > > > > > >> > > Jun
> > > > > > > >> > >
> > > > > > > >> > >
> > > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >> > >
> > > > > > > >> > > > Jun,
> > > > > > > >> > > >
> > > > > > > >> > > > Answering your questions:
> > > > > > > >> > > >
> > > > > > > >> > > > 101. If I understand you correctly, you are saying
> > future
> > > > > producer
> > > > > > > >> > > versions
> > > > > > > >> > > > (which
> > > > > > > >> > > > will be ported to TMR_V1) won't be able to automatically
> > > > > create
> > > > > > > >> topic (if
> > > > > > > >> > > > we
> > > > > > > >> > > > unconditionally remove topic creation from there). But
> > we
> > > > > need to
> > > > > > > >> this
> > > > > > > >> > > > preserve logic.
> > > > > > > >> > > > Ok, about your proposal: I'm not a big fan too, when it
> > > > comes
> > > > > to
> > > > > > > >> > > > differentiating
> > > > > > > >> > > > clients directly in protocol schema. And also I'm not
> > > sure I
> > > > > > > >> understand
> > > > > > > >> > > at
> > > > > > > >> > > > all why
> > > > > > > >> > > > auto.create.topics.enable is a server side
> > configuration.
> > > > Can
> > > > > we
> > > > > > > >> > > deprecate
> > > > > > > >> > > > this setting
> > > > > > > >> > > > in future versions, add this setting to producer and
> > based
> > > > on
> > > > > that
> > > > > > > >> upon
> > > > > > > >> > > > receiving
> > > > > > > >> > > > UnknownTopic create topic explicitly by a separate
> > > producer
> > > > > call
> > > > > > > via
> > > > > > > >> > > > adminClient?
> > > > > > > >> > > >
> > > > > > > >> > > > 102.1. Hm, yes. It's because we want to support batching
> > > and
> > > > > at
> > > > > > > the
> > > > > > > >> same
> > > > > > > >> > > > time we
> > > > > > > >> > > > want to give descriptive error messages for clients.
> > Since
> > > > > > > >> AdminClient
> > > > > > > >> > > > holds the context
> > > > > > > >> > > > to construct such messages (e.g. AdminClient layer can
> > > know
> > > > > that
> > > > > > > >> > > > InvalidArgumentsCode
> > > > > > > >> > > > means two cases: either invalid number - e.g. -1; or
> > > > > > > >> replication-factor
> > > > > > > >> > > was
> > > > > > > >> > > > provided while
> > > > > > > >> > > > partitions argument wasn't) - I wrapped responses in
> > > > > Exceptions.
> > > > > > > >> But I'm
> > > > > > > >> > > > open to any
> > > > > > > >> > > > other ideas, this was just initial version.
> > > > > > > >> > > > 102.2. Yes, I agree. I'll change that to probably some
> > > other
> > > > > dto.
> > > > > > > >> > > >
> > > > > > > >> > > > Thanks,
> > > > > > > >> > > > Andrii Biletskyi
> > > > > > > >> > > >
> > > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <
> > > jun@confluent.io>
> > > > > > > wrote:
> > > > > > > >> > > >
> > > > > > > >> > > > > Andrii,
> > > > > > > >> > > > >
> > > > > > > >> > > > > 101. That's what I was thinking too, but it may not be
> > > > that
> > > > > > > >> simple. In
> > > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > > >> > > > > we can let it not trigger auto topic creation. Then,
> > in
> > > > the
> > > > > > > >> producer
> > > > > > > >> > > > side,
> > > > > > > >> > > > > if it gets an UnknownTopicException, it can explicitly
> > > > > issue a
> > > > > > > >> > > > > createTopicRequest for auto topic creation. On the
> > > > consumer
> > > > > > > side,
> > > > > > > >> it
> > > > > > > >> > > will
> > > > > > > >> > > > > never issue createTopicRequest. This works when auto
> > > topic
> > > > > > > >> creation is
> > > > > > > >> > > > > enabled on the broker side. However, I am not sure how
> > > > > things
> > > > > > > >> will work
> > > > > > > >> > > > > when auto topic creation is disabled on the broker
> > side.
> > > > In
> > > > > this
> > > > > > > >> case,
> > > > > > > >> > > we
> > > > > > > >> > > > > want to have a way to manually create a topic,
> > > potentially
> > > > > > > through
> > > > > > > >> > > admin
> > > > > > > >> > > > > commands. However, then we need a way to distinguish
> > > > > > > >> createTopicRequest
> > > > > > > >> > > > > issued from the producer clients and the admin tools.
> > > May
> > > > > be we
> > > > > > > >> can
> > > > > > > >> > > add a
> > > > > > > >> > > > > new field in createTopicRequest and set it differently
> > > in
> > > > > the
> > > > > > > >> producer
> > > > > > > >> > > > > client and the admin client. However, I am not sure if
> > > > > that's
> > > > > > > the
> > > > > > > >> best
> > > > > > > >> > > > > approach.
> > > > > > > >> > > > >
> > > > > > > >> > > > > 2. Yes, refactoring existing requests is a non-trivial
> > > > > amount of
> > > > > > > >> work.
> > > > > > > >> > > I
> > > > > > > >> > > > > posted some comments in KAFKA-1927. We will probably
> > > have
> > > > > to fix
> > > > > > > >> > > > KAFKA-1927
> > > > > > > >> > > > > first, before adding the new logic in KAFKA-1694.
> > > > > Otherwise, the
> > > > > > > >> > > changes
> > > > > > > >> > > > > will be too big.
> > > > > > > >> > > > >
> > > > > > > >> > > > > 102. About the AdminClient:
> > > > > > > >> > > > > 102.1. It's a bit weird that we return exception in
> > the
> > > > > api. It
> > > > > > > >> seems
> > > > > > > >> > > > that
> > > > > > > >> > > > > we should either return error code or throw an
> > exception
> > > > > when
> > > > > > > >> getting
> > > > > > > >> > > the
> > > > > > > >> > > > > response state.
> > > > > > > >> > > > > 102.2. We probably shouldn't explicitly use the
> > request
> > > > > object
> > > > > > > in
> > > > > > > >> the
> > > > > > > >> > > > api.
> > > > > > > >> > > > > Not every request evolution requires an api change.
> > > > > > > >> > > > >
> > > > > > > >> > > > > Thanks,
> > > > > > > >> > > > >
> > > > > > > >> > > > > Jun
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > > > > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > > > Jun,
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Thanks for you comments. Answers inline:
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > 100. There are a few fields such as
> > ReplicaAssignment,
> > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > >> > > > > > > and PartitionsSerialized that are represented as a
> > > > > string,
> > > > > > > but
> > > > > > > >> > > > contain
> > > > > > > >> > > > > > > composite structures in json. Could we flatten
> > them
> > > > out
> > > > > > > >> directly in
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Yes, now with Admin Client this looks a bit weird.
> > My
> > > > > initial
> > > > > > > >> > > > motivation
> > > > > > > >> > > > > > was:
> > > > > > > >> > > > > > ReassignPartitionCommand accepts input in json, we
> > > want
> > > > to
> > > > > > > >> remain
> > > > > > > >> > > > tools'
> > > > > > > >> > > > > > interfaces unchanged, where possible.
> > > > > > > >> > > > > > If we port it to deserialized format, in CLI (/tools
> > > > > project)
> > > > > > > >> we will
> > > > > > > >> > > > > have
> > > > > > > >> > > > > > to add some
> > > > > > > >> > > > > > json library since /tools is written in java and
> > we'll
> > > > > need to
> > > > > > > >> > > > > deserialize
> > > > > > > >> > > > > > json file
> > > > > > > >> > > > > > provided by a user. Can we quickly agree on what
> > this
> > > > > library
> > > > > > > >> should
> > > > > > > >> > > be
> > > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto
> > > > topic
> > > > > > > >> creation?
> > > > > > > >> > > > This
> > > > > > > >> > > > > > > will be a bit weird now that we have a separate
> > > topic
> > > > > > > >> creation api.
> > > > > > > >> > > > > Have
> > > > > > > >> > > > > > > you thought about how the new createTopicRequest
> > and
> > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > >> > > > > > > v1 will be used in the producer/consumer client,
> > in
> > > > > addition
> > > > > > > >> to
> > > > > > > >> > > admin
> > > > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > > > >> TopicMetadataRequest
> > > > > > > >> > > from
> > > > > > > >> > > > > the
> > > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > I agree, this strange logic should be fixed. I'm not
> > > > > confident
> > > > > > > >> in
> > > > > > > >> > > this
> > > > > > > >> > > > > > Kafka part so
> > > > > > > >> > > > > > correct me if I'm wrong, but it doesn't look like a
> > > hard
> > > > > thing
> > > > > > > >> to
> > > > > > > >> > > do, I
> > > > > > > >> > > > > > think we can
> > > > > > > >> > > > > > leverage AdminClient for that in Producer and
> > > > > unconditionally
> > > > > > > >> remove
> > > > > > > >> > > > > topic
> > > > > > > >> > > > > > creation from the TopicMetadataRequest_V1.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > 2. I think Jay meant getting rid of scala classes
> > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > >> > > did
> > > > > > > >> > > > > > that
> > > > > > > >> > > > > > > as a stop-gap thing when adding the new requests
> > for
> > > > the
> > > > > > > >> consumers.
> > > > > > > >> > > > > > > However, the long term plan is to get rid of all
> > > those
> > > > > and
> > > > > > > >> just
> > > > > > > >> > > reuse
> > > > > > > >> > > > > the
> > > > > > > >> > > > > > > java request/response in the client. Since this
> > KIP
> > > > > proposes
> > > > > > > >> to
> > > > > > > >> > > add a
> > > > > > > >> > > > > > > significant number of new requests, perhaps we
> > > should
> > > > > bite
> > > > > > > the
> > > > > > > >> > > bullet
> > > > > > > >> > > > > to
> > > > > > > >> > > > > > > clean up the existing scala requests first before
> > > > > adding new
> > > > > > > >> ones?
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Yes, looks like I misunderstood the point of
> > > > > > > >> ...RequestAndHeader.
> > > > > > > >> > > > Okay, I
> > > > > > > >> > > > > > will
> > > > > > > >> > > > > > rework that. The only thing is that I don't see any
> > > > > example
> > > > > > > how
> > > > > > > >> it
> > > > > > > >> > > was
> > > > > > > >> > > > > done
> > > > > > > >> > > > > > for at
> > > > > > > >> > > > > > least one existing protocol message. Thus, as I
> > > > > understand, I
> > > > > > > >> have to
> > > > > > > >> > > > > think
> > > > > > > >> > > > > > how we
> > > > > > > >> > > > > > are going to do it.
> > > > > > > >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
> > > > > > > reasonable,
> > > > > > > >> but
> > > > > > > >> > > if
> > > > > > > >> > > > > it's
> > > > > > > >> > > > > > an *obligatory*
> > > > > > > >> > > > > > requirement to have Admin KIP done, I'm afraid this
> > > can
> > > > > be a
> > > > > > > >> serious
> > > > > > > >> > > > > > blocker for us.
> > > > > > > >> > > > > > There are 13 protocol messages and all that would
> > > > require
> > > > > not
> > > > > > > >> only
> > > > > > > >> > > unit
> > > > > > > >> > > > > > tests but quite
> > > > > > > >> > > > > > intensive manual testing, no? I'm afraid I'm not the
> > > > > right guy
> > > > > > > >> to
> > > > > > > >> > > cover
> > > > > > > >> > > > > > pretty much all
> > > > > > > >> > > > > > Kafka core internals :). Let me know your thoughts
> > on
> > > > this
> > > > > > > >> item. Btw
> > > > > > > >> > > > > there
> > > > > > > >> > > > > > is a ticket to
> > > > > > > >> > > > > > follow-up this issue (
> > > > > > > >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > > >> > > > ).
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Thanks,
> > > > > > > >> > > > > > Andrii Biletskyi
> > > > > > > >> > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <
> > > > > jun@confluent.io>
> > > > > > > >> wrote:
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > > Andrii,
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > A few more comments.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > 100. There are a few fields such as
> > > ReplicaAssignment,
> > > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > > >> > > > > > > and PartitionsSerialized that are represented as a
> > > > > string,
> > > > > > > but
> > > > > > > >> > > > contain
> > > > > > > >> > > > > > > composite structures in json. Could we flatten
> > them
> > > > out
> > > > > > > >> directly in
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger
> > auto
> > > > > topic
> > > > > > > >> > > creation?
> > > > > > > >> > > > > This
> > > > > > > >> > > > > > > will be a bit weird now that we have a separate
> > > topic
> > > > > > > >> creation api.
> > > > > > > >> > > > > Have
> > > > > > > >> > > > > > > you thought about how the new createTopicRequest
> > and
> > > > > > > >> > > > > TopicMetadataRequest
> > > > > > > >> > > > > > > v1 will be used in the producer/consumer client,
> > in
> > > > > addition
> > > > > > > >> to
> > > > > > > >> > > admin
> > > > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > > > >> TopicMetadataRequest
> > > > > > > >> > > from
> > > > > > > >> > > > > the
> > > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > 2. I think Jay meant getting rid of scala classes
> > > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > > >> > > did
> > > > > > > >> > > > > > that
> > > > > > > >> > > > > > > as a stop-gap thing when adding the new requests
> > for
> > > > the
> > > > > > > >> consumers.
> > > > > > > >> > > > > > > However, the long term plan is to get rid of all
> > > those
> > > > > and
> > > > > > > >> just
> > > > > > > >> > > reuse
> > > > > > > >> > > > > the
> > > > > > > >> > > > > > > java request/response in the client. Since this
> > KIP
> > > > > proposes
> > > > > > > >> to
> > > > > > > >> > > add a
> > > > > > > >> > > > > > > significant number of new requests, perhaps we
> > > should
> > > > > bite
> > > > > > > the
> > > > > > > >> > > bullet
> > > > > > > >> > > > > to
> > > > > > > >> > > > > > > clean up the existing scala requests first before
> > > > > adding new
> > > > > > > >> ones?
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Thanks,
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Jun
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi
> > <
> > > > > > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > > Hi,
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > As said above - I list again all comments from
> > > this
> > > > > thread
> > > > > > > >> so we
> > > > > > > >> > > > > > > > can see what's left and finalize all pending
> > > issues.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > Comments from Jay:
> > > > > > > >> > > > > > > > 1. This is much needed functionality, but there
> > > are
> > > > a
> > > > > lot
> > > > > > > >> of the
> > > > > > > >> > > so
> > > > > > > >> > > > > > let's
> > > > > > > >> > > > > > > > really think these protocols through. We really
> > > want
> > > > > to
> > > > > > > end
> > > > > > > >> up
> > > > > > > >> > > > with a
> > > > > > > >> > > > > > set
> > > > > > > >> > > > > > > > of well thought-out, orthoganol apis. For this
> > > > reason
> > > > > I
> > > > > > > >> think it
> > > > > > > >> > > is
> > > > > > > >> > > > > > > really
> > > > > > > >> > > > > > > > important to think through the end state even if
> > > > that
> > > > > > > >> includes
> > > > > > > >> > > APIs
> > > > > > > >> > > > > we
> > > > > > > >> > > > > > > > won't implement in the first phase.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: Definitely behind this. Would appreciate if
> > > there
> > > > > are
> > > > > > > >> concrete
> > > > > > > >> > > > > > > comments
> > > > > > > >> > > > > > > > how this can be improved.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 2. Let's please please please wait until we have
> > > > > switched
> > > > > > > >> the
> > > > > > > >> > > > server
> > > > > > > >> > > > > > over
> > > > > > > >> > > > > > > > to the new java protocol definitions. If we add
> > > > upteen
> > > > > > > more
> > > > > > > >> ad
> > > > > > > >> > > hoc
> > > > > > > >> > > > > > scala
> > > > > > > >> > > > > > > > objects that is just generating more work for
> > the
> > > > > > > >> conversion we
> > > > > > > >> > > > know
> > > > > > > >> > > > > we
> > > > > > > >> > > > > > > > have to do.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed scala
> > > > protocol
> > > > > > > >> classes.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 3. This proposal introduces a new type of
> > optional
> > > > > > > >> parameter.
> > > > > > > >> > > This
> > > > > > > >> > > > is
> > > > > > > >> > > > > > > > inconsistent with everything else in the
> > protocol
> > > > > where we
> > > > > > > >> use -1
> > > > > > > >> > > > or
> > > > > > > >> > > > > > some
> > > > > > > >> > > > > > > > other marker value. You could argue either way
> > but
> > > > > let's
> > > > > > > >> stick
> > > > > > > >> > > with
> > > > > > > >> > > > > > that
> > > > > > > >> > > > > > > > for consistency. For clients that implemented
> > the
> > > > > protocol
> > > > > > > >> in a
> > > > > > > >> > > > > better
> > > > > > > >> > > > > > > way
> > > > > > > >> > > > > > > > than our scala code these basic primitives are
> > > hard
> > > > to
> > > > > > > >> change.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf
> > > type
> > > > > and
> > > > > > > >> changed
> > > > > > > >> > > > > > protocol
> > > > > > > >> > > > > > > > accordingly.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> > > > > > > >> TopicMetadataRequest
> > > > > > > >> > > > > which
> > > > > > > >> > > > > > > has
> > > > > > > >> > > > > > > > brokers, topics, and partitions. I think we
> > should
> > > > > rename
> > > > > > > >> that
> > > > > > > >> > > > > request
> > > > > > > >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest)
> > > and
> > > > > > > >> include the
> > > > > > > >> > > id
> > > > > > > >> > > > > of
> > > > > > > >> > > > > > > the
> > > > > > > >> > > > > > > > controller. Or are there other things we could
> > add
> > > > > here?
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: I agree. Updated the KIP. Let's extends
> > > > > TopicMetadata
> > > > > > > to
> > > > > > > >> > > > version 2
> > > > > > > >> > > > > > and
> > > > > > > >> > > > > > > > include controller.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 5. We have a tendency to try to make a lot of
> > > > requests
> > > > > > > that
> > > > > > > >> can
> > > > > > > >> > > > only
> > > > > > > >> > > > > go
> > > > > > > >> > > > > > > to
> > > > > > > >> > > > > > > > particular nodes. This adds a lot of burden for
> > > > client
> > > > > > > >> > > > > implementations
> > > > > > > >> > > > > > > (it
> > > > > > > >> > > > > > > > sounds easy but each discovery can fail in many
> > > > parts
> > > > > so
> > > > > > > it
> > > > > > > >> ends
> > > > > > > >> > > up
> > > > > > > >> > > > > > > being a
> > > > > > > >> > > > > > > > full state machine to do right). I think we
> > should
> > > > > > > consider
> > > > > > > >> > > making
> > > > > > > >> > > > > > admin
> > > > > > > >> > > > > > > > commands and ideally as many of the other apis
> > as
> > > > > possible
> > > > > > > >> > > > available
> > > > > > > >> > > > > on
> > > > > > > >> > > > > > > all
> > > > > > > >> > > > > > > > brokers and just redirect to the controller on
> > the
> > > > > broker
> > > > > > > >> side.
> > > > > > > >> > > > > Perhaps
> > > > > > > >> > > > > > > > there would be a general way to encapsulate this
> > > > > > > re-routing
> > > > > > > >> > > > behavior.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: It's a very interesting idea, but seems there
> > > are
> > > > > some
> > > > > > > >> > > concerns
> > > > > > > >> > > > > > about
> > > > > > > >> > > > > > > > this
> > > > > > > >> > > > > > > > feature (like performance considerations, how
> > this
> > > > > will
> > > > > > > >> > > complicate
> > > > > > > >> > > > > > server
> > > > > > > >> > > > > > > > etc).
> > > > > > > >> > > > > > > > I believe this shouldn't be a blocker. If this
> > > > > feature is
> > > > > > > >> > > > implemented
> > > > > > > >> > > > > > at
> > > > > > > >> > > > > > > > some
> > > > > > > >> > > > > > > > point it won't affect Admin changes - at least
> > no
> > > > > changes
> > > > > > > to
> > > > > > > >> > > public
> > > > > > > >> > > > > API
> > > > > > > >> > > > > > > > will be required.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 6. We should probably normalize the key value
> > > pairs
> > > > > used
> > > > > > > for
> > > > > > > >> > > > configs
> > > > > > > >> > > > > > > rather
> > > > > > > >> > > > > > > > than embedding a new formatting. So two strings
> > > > rather
> > > > > > > than
> > > > > > > >> one
> > > > > > > >> > > > with
> > > > > > > >> > > > > an
> > > > > > > >> > > > > > > > internal equals sign.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: Fixed in the latest patch - normalized
> > configs
> > > > and
> > > > > > > >> changed
> > > > > > > >> > > > > protocol
> > > > > > > >> > > > > > > > accordingly.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 7. Is the postcondition of these APIs that the
> > > > > command has
> > > > > > > >> begun
> > > > > > > >> > > or
> > > > > > > >> > > > > > that
> > > > > > > >> > > > > > > > the command has been completed? It is a lot more
> > > > > usable if
> > > > > > > >> the
> > > > > > > >> > > > > command
> > > > > > > >> > > > > > > has
> > > > > > > >> > > > > > > > been completed so you know that if you create a
> > > > topic
> > > > > and
> > > > > > > >> then
> > > > > > > >> > > > > publish
> > > > > > > >> > > > > > to
> > > > > > > >> > > > > > > > it you won't get an exception about there being
> > no
> > > > > such
> > > > > > > >> topic.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: For long running requests (like reassign
> > > > > partitions) -
> > > > > > > >> the
> > > > > > > >> > > post
> > > > > > > >> > > > > > > > condition is
> > > > > > > >> > > > > > > > command has begun - so we don't block the
> > client.
> > > In
> > > > > case
> > > > > > > >> of your
> > > > > > > >> > > > > > > example -
> > > > > > > >> > > > > > > > topic commands, this will be refactored and
> > topic
> > > > > commands
> > > > > > > >> will
> > > > > > > >> > > be
> > > > > > > >> > > > > > > executed
> > > > > > > >> > > > > > > > immediately, since the Controller will serve
> > Admin
> > > > > > > requests
> > > > > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 8. Describe topic and list topics duplicate a
> > lot
> > > of
> > > > > stuff
> > > > > > > >> in the
> > > > > > > >> > > > > > > metadata
> > > > > > > >> > > > > > > > request. Is there a reason to give back topics
> > > > marked
> > > > > for
> > > > > > > >> > > > deletion? I
> > > > > > > >> > > > > > > feel
> > > > > > > >> > > > > > > > like if we just make the post-condition of the
> > > > delete
> > > > > > > >> command be
> > > > > > > >> > > > that
> > > > > > > >> > > > > > the
> > > > > > > >> > > > > > > > topic is deleted that will get rid of the need
> > for
> > > > > this
> > > > > > > >> right?
> > > > > > > >> > > And
> > > > > > > >> > > > it
> > > > > > > >> > > > > > > will
> > > > > > > >> > > > > > > > be much more intuitive.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: Fixed in the latest patch - removed topics
> > > marked
> > > > > for
> > > > > > > >> deletion
> > > > > > > >> > > > in
> > > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 9. Should we consider batching these requests?
> > We
> > > > have
> > > > > > > >> generally
> > > > > > > >> > > > > tried
> > > > > > > >> > > > > > to
> > > > > > > >> > > > > > > > allow multiple operations to be batched. My
> > > > suspicion
> > > > > is
> > > > > > > >> that
> > > > > > > >> > > > without
> > > > > > > >> > > > > > > this
> > > > > > > >> > > > > > > > we will get a lot of code that does something
> > like
> > > > > > > >> > > > > > > >    for(topic: adminClient.listTopics())
> > > > > > > >> > > > > > > >       adminClient.describeTopic(topic)
> > > > > > > >> > > > > > > > this code will work great when you test on 5
> > > topics
> > > > > but
> > > > > > > not
> > > > > > > >> do as
> > > > > > > >> > > > > well
> > > > > > > >> > > > > > if
> > > > > > > >> > > > > > > > you have 50k.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: Updated the KIP - please check "Topic Admin
> > > > Schema"
> > > > > > > >> section.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 10. I think we should also discuss how we want
> > to
> > > > > expose a
> > > > > > > >> > > > > programmatic
> > > > > > > >> > > > > > > JVM
> > > > > > > >> > > > > > > > client api for these operations. Currently
> > people
> > > > > rely on
> > > > > > > >> > > > AdminUtils
> > > > > > > >> > > > > > > which
> > > > > > > >> > > > > > > > is totally sketchy. I think we probably need
> > > another
> > > > > > > client
> > > > > > > >> under
> > > > > > > >> > > > > > > clients/
> > > > > > > >> > > > > > > > that exposes administrative functionality. We
> > will
> > > > > need
> > > > > > > >> this just
> > > > > > > >> > > > to
> > > > > > > >> > > > > > > > properly test the new apis, I suspect. We should
> > > > > figure
> > > > > > > out
> > > > > > > >> that
> > > > > > > >> > > > API.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: Updated the KIP - please check "Admin Client"
> > > > > section
> > > > > > > >> with an
> > > > > > > >> > > > > > initial
> > > > > > > >> > > > > > > > API proposal.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 11. The other information that would be really
> > > > useful
> > > > > to
> > > > > > > get
> > > > > > > >> > > would
> > > > > > > >> > > > be
> > > > > > > >> > > > > > > > information about partitions--how much data is
> > in
> > > > the
> > > > > > > >> partition,
> > > > > > > >> > > > what
> > > > > > > >> > > > > > are
> > > > > > > >> > > > > > > > the segment offsets, what is the log-end offset
> > > > (i.e.
> > > > > last
> > > > > > > >> > > offset),
> > > > > > > >> > > > > > what
> > > > > > > >> > > > > > > is
> > > > > > > >> > > > > > > > the compaction point, etc. I think that done
> > right
> > > > > this
> > > > > > > >> would be
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > > successor to the very awkward OffsetRequest we
> > > have
> > > > > today.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the
> > > > latest
> > > > > > > >> patch. I
> > > > > > > >> > > > > believe
> > > > > > > >> > > > > > > > this should
> > > > > > > >> > > > > > > > be resolved in a separate KIP / jira ticket.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 12. Generally we can do good error handling
> > > without
> > > > > > > needing
> > > > > > > >> > > custom
> > > > > > > >> > > > > > > > server-side
> > > > > > > >> > > > > > > > messages. I.e. generally the client has the
> > > context
> > > > to
> > > > > > > know
> > > > > > > >> that
> > > > > > > >> > > if
> > > > > > > >> > > > > it
> > > > > > > >> > > > > > > got
> > > > > > > >> > > > > > > > an error that the topic doesn't exist to say
> > > "Topic
> > > > X
> > > > > > > >> doesn't
> > > > > > > >> > > > exist"
> > > > > > > >> > > > > > > rather
> > > > > > > >> > > > > > > > than "error code 14" (or whatever). Maybe there
> > > are
> > > > > > > specific
> > > > > > > >> > > cases
> > > > > > > >> > > > > > where
> > > > > > > >> > > > > > > > this is hard? If we want to add server-side
> > error
> > > > > messages
> > > > > > > >> we
> > > > > > > >> > > > really
> > > > > > > >> > > > > do
> > > > > > > >> > > > > > > > need to do this in a consistent way across the
> > > > > protocol.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: Updated the KIP - please check "Protocol
> > > Errors"
> > > > > > > >> section. I
> > > > > > > >> > > > added
> > > > > > > >> > > > > > the
> > > > > > > >> > > > > > > > comprehensive, fine-grained list of error codes.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > > >> > > > > > > > 13. Describe topic request: it would be great to
> > > go
> > > > > beyond
> > > > > > > >> just
> > > > > > > >> > > > > > batching
> > > > > > > >> > > > > > > on
> > > > > > > >> > > > > > > > topic name regex for this request. For example,
> > a
> > > > very
> > > > > > > >> common use
> > > > > > > >> > > > > case
> > > > > > > >> > > > > > of
> > > > > > > >> > > > > > > > the topic command is to list all topics whose
> > > config
> > > > > A's
> > > > > > > >> value is
> > > > > > > >> > > > B.
> > > > > > > >> > > > > > With
> > > > > > > >> > > > > > > > topic name regex then we have to first retrieve
> > > > > __all__
> > > > > > > >> topics's
> > > > > > > >> > > > > > > > description info and then filter at the client
> > > end,
> > > > > which
> > > > > > > >> will
> > > > > > > >> > > be a
> > > > > > > >> > > > > > huge
> > > > > > > >> > > > > > > > burden on ZK.
> > > > > > > >> > > > > > > > AND
> > > > > > > >> > > > > > > > 14. Config K-Vs in create topic: this is related
> > > to
> > > > > the
> > > > > > > >> previous
> > > > > > > >> > > > > point;
> > > > > > > >> > > > > > > > maybe we can add another metadata K-V or just a
> > > > > metadata
> > > > > > > >> string
> > > > > > > >> > > > along
> > > > > > > >> > > > > > > side
> > > > > > > >> > > > > > > > with config K-V in create topic like we did for
> > > > offset
> > > > > > > >> commit
> > > > > > > >> > > > > request.
> > > > > > > >> > > > > > > This
> > > > > > > >> > > > > > > > field can be quite useful in storing information
> > > > like
> > > > > > > >> "owner" of
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > topic
> > > > > > > >> > > > > > > > who issue the create command, etc, which is
> > quite
> > > > > > > important
> > > > > > > >> for a
> > > > > > > >> > > > > > > > multi-tenant setting. Then in the describe topic
> > > > > request
> > > > > > > we
> > > > > > > >> can
> > > > > > > >> > > > also
> > > > > > > >> > > > > > > batch
> > > > > > > >> > > > > > > > on regex of the metadata field.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: As discussed it is very interesting but can
> > be
> > > > > > > >> implemented
> > > > > > > >> > > later
> > > > > > > >> > > > > > after
> > > > > > > >> > > > > > > > we have some basic functionality there.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > 15. Today all the admin operations are async in
> > > the
> > > > > sense
> > > > > > > >> that
> > > > > > > >> > > > > command
> > > > > > > >> > > > > > > will
> > > > > > > >> > > > > > > > return once it is written in ZK, and that is why
> > > we
> > > > > need
> > > > > > > >> extra
> > > > > > > >> > > > > > > verification
> > > > > > > >> > > > > > > > like testUtil.waitForTopicCreated() / verify
> > > > partition
> > > > > > > >> > > reassignment
> > > > > > > >> > > > > > > > request, etc. With admin requests we could add a
> > > > flag
> > > > > to
> > > > > > > >> enable /
> > > > > > > >> > > > > > disable
> > > > > > > >> > > > > > > > synchronous requests; when it is turned on, the
> > > > > response
> > > > > > > >> will not
> > > > > > > >> > > > > > return
> > > > > > > >> > > > > > > > until the request has been completed. And for
> > > async
> > > > > > > >> requests we
> > > > > > > >> > > can
> > > > > > > >> > > > > > add a
> > > > > > > >> > > > > > > > "token" field in the response, and then only
> > need
> > > a
> > > > > > > general
> > > > > > > >> > > "admin
> > > > > > > >> > > > > > > > verification request" with the given token to
> > > check
> > > > > if the
> > > > > > > >> async
> > > > > > > >> > > > > > request
> > > > > > > >> > > > > > > > has been completed.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: I see your point. My idea was to provide
> > > specific
> > > > > > > >> > > > Verify...Request
> > > > > > > >> > > > > > per
> > > > > > > >> > > > > > > > each
> > > > > > > >> > > > > > > > long running request, where needed. We can do it
> > > the
> > > > > way
> > > > > > > you
> > > > > > > >> > > > suggest.
> > > > > > > >> > > > > > The
> > > > > > > >> > > > > > > > only
> > > > > > > >> > > > > > > > concern is that introducing a token we again
> > will
> > > > make
> > > > > > > >> schema
> > > > > > > >> > > > > > "dynamic".
> > > > > > > >> > > > > > > We
> > > > > > > >> > > > > > > > wanted
> > > > > > > >> > > > > > > > to do similar thing introducing single
> > > AdminRequest
> > > > > for
> > > > > > > all
> > > > > > > >> topic
> > > > > > > >> > > > > > > commands
> > > > > > > >> > > > > > > > but rejected
> > > > > > > >> > > > > > > > this idea because we wanted to have schema
> > > defined.
> > > > So
> > > > > > > this
> > > > > > > >> is
> > > > > > > >> > > > more a
> > > > > > > >> > > > > > > > choice between:
> > > > > > > >> > > > > > > > a) have fixed schema but introduce each time new
> > > > > > > >> Verify...Request
> > > > > > > >> > > > for
> > > > > > > >> > > > > > > > long-running requests
> > > > > > > >> > > > > > > > b) use one request for verification but
> > generalize
> > > > it
> > > > > with
> > > > > > > >> token
> > > > > > > >> > > > > > > > I'm fine with whatever decision community come
> > to.
> > > > > Just
> > > > > > > let
> > > > > > > >> me
> > > > > > > >> > > know
> > > > > > > >> > > > > > your
> > > > > > > >> > > > > > > > thoughts.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > > >> > > > > > > > 16. Specifically for ownership, I think the plan
> > > is
> > > > > to add
> > > > > > > >> ACL
> > > > > > > >> > > (it
> > > > > > > >> > > > > > sounds
> > > > > > > >> > > > > > > > like you are describing ACL) via an external
> > > system
> > > > > > > (Argus,
> > > > > > > >> > > > Sentry).
> > > > > > > >> > > > > > > > I remember KIP-11 described this, but I can't
> > find
> > > > > the KIP
> > > > > > > >> any
> > > > > > > >> > > > > longer.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > A: Okay, no problem. Not sure though how we are
> > > > going
> > > > > to
> > > > > > > >> handle
> > > > > > > >> > > it.
> > > > > > > >> > > > > > Wait
> > > > > > > >> > > > > > > > which KIP
> > > > > > > >> > > > > > > > will be committed first and include changes to
> > > > > > > >> TopicMetadata from
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > later
> > > > > > > >> > > > > > > > one?
> > > > > > > >> > > > > > > > Anyway, I added this note to "Open Questions"
> > > > section
> > > > > so
> > > > > > > we
> > > > > > > >> don't
> > > > > > > >> > > > > miss
> > > > > > > >> > > > > > > this
> > > > > > > >> > > > > > > > piece.
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > Thanks,
> > > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii
> > > Biletskyi <
> > > > > > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > > Hi all,
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > Today I uploaded the patch that covers some of
> > > the
> > > > > > > >> discussed
> > > > > > > >> > > and
> > > > > > > >> > > > > > agreed
> > > > > > > >> > > > > > > > > items:
> > > > > > > >> > > > > > > > > - removed MaybeOf optional type
> > > > > > > >> > > > > > > > > - switched to java protocol definitions
> > > > > > > >> > > > > > > > > - simplified messages (normalized configs,
> > > removed
> > > > > topic
> > > > > > > >> marked
> > > > > > > >> > > > for
> > > > > > > >> > > > > > > > > deletion)
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > I also updated the KIP-4 with respective
> > changes
> > > > and
> > > > > > > >> wrote down
> > > > > > > >> > > > my
> > > > > > > >> > > > > > > > > proposal for
> > > > > > > >> > > > > > > > > pending items:
> > > > > > > >> > > > > > > > > - Batch Admin Operations -> updated Wire
> > > Protocol
> > > > > schema
> > > > > > > >> > > proposal
> > > > > > > >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
> > > > > > > >> > > > TopicMetadataRequest
> > > > > > > >> > > > > > > > > - Admin Client -> updated my initial proposal
> > to
> > > > > reflect
> > > > > > > >> > > batching
> > > > > > > >> > > > > > > > > - Error codes -> proposed fine-grained error
> > > code
> > > > > > > instead
> > > > > > > >> of
> > > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > I will also send a separate email to cover all
> > > > > comments
> > > > > > > >> from
> > > > > > > >> > > this
> > > > > > > >> > > > > > > thread.
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > Thanks,
> > > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira
> > <
> > > > > > > >> > > > > gshapira@cloudera.com
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > > > wrote:
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > > >> > > > > > > > >>
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >>
> > > > > > >
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > >> > > > > > > > >> )
> > > > > > > >> > > > > > > > >> It actually specifies changes to the Metadata
> > > > > protocol,
> > > > > > > >> so
> > > > > > > >> > > > making
> > > > > > > >> > > > > > sure
> > > > > > > >> > > > > > > > >> both KIPs are consistent in this regard will
> > be
> > > > > good.
> > > > > > > >> > > > > > > > >>
> > > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen
> > Shapira
> > > <
> > > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > >> wrote:
> > > > > > > >> > > > > > > > >> > Specifically for ownership, I think the
> > plan
> > > is
> > > > > to
> > > > > > > add
> > > > > > > >> ACL
> > > > > > > >> > > (it
> > > > > > > >> > > > > > > sounds
> > > > > > > >> > > > > > > > >> > like you are describing ACL) via an
> > external
> > > > > system
> > > > > > > >> (Argus,
> > > > > > > >> > > > > > Sentry).
> > > > > > > >> > > > > > > > >> > I remember KIP-11 described this, but I
> > can't
> > > > > find
> > > > > > > the
> > > > > > > >> KIP
> > > > > > > >> > > any
> > > > > > > >> > > > > > > longer.
> > > > > > > >> > > > > > > > >> >
> > > > > > > >> > > > > > > > >> > Regardless, I think KIP-4 focuses on
> > getting
> > > > > > > >> information
> > > > > > > >> > > that
> > > > > > > >> > > > > > > already
> > > > > > > >> > > > > > > > >> > exists from Kafka brokers, not on adding
> > > > > information
> > > > > > > >> that
> > > > > > > >> > > > > perhaps
> > > > > > > >> > > > > > > > >> > should exist but doesn't yet?
> > > > > > > >> > > > > > > > >> >
> > > > > > > >> > > > > > > > >> > Gwen
> > > > > > > >> > > > > > > > >> >
> > > > > > > >> > > > > > > > >> >
> > > > > > > >> > > > > > > > >> >
> > > > > > > >> > > > > > > > >> >
> > > > > > > >> > > > > > > > >> >
> > > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang
> > > Wang
> > > > <
> > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > >> > > > > > > > >> wrote:
> > > > > > > >> > > > > > > > >> >> Folks,
> > > > > > > >> > > > > > > > >> >>
> > > > > > > >> > > > > > > > >> >> Just want to elaborate a bit more on the
> > > > > > > create-topic
> > > > > > > >> > > > metadata
> > > > > > > >> > > > > > and
> > > > > > > >> > > > > > > > >> batching
> > > > > > > >> > > > > > > > >> >> describe-topic based on config / metadata
> > in
> > > > my
> > > > > > > >> previous
> > > > > > > >> > > > email
> > > > > > > >> > > > > as
> > > > > > > >> > > > > > > we
> > > > > > > >> > > > > > > > >> work
> > > > > > > >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to
> > > have
> > > > > some
> > > > > > > >> sort of
> > > > > > > >> > > > > topic
> > > > > > > >> > > > > > > > >> management
> > > > > > > >> > > > > > > > >> >> mechanisms, which I think is quite
> > important
> > > > in
> > > > > a
> > > > > > > >> > > > multi-tenant
> > > > > > > >> > > > > /
> > > > > > > >> > > > > > > > cloud
> > > > > > > >> > > > > > > > >> >> architecture: today anyone can create
> > topics
> > > > in
> > > > > a
> > > > > > > >> shared
> > > > > > > >> > > > Kafka
> > > > > > > >> > > > > > > > >> cluster, but
> > > > > > > >> > > > > > > > >> >> there is no concept or "ownership" of
> > topics
> > > > > that
> > > > > > > are
> > > > > > > >> > > created
> > > > > > > >> > > > > by
> > > > > > > >> > > > > > > > >> different
> > > > > > > >> > > > > > > > >> >> users. For example, at LinkedIn we
> > basically
> > > > > > > >> distinguish
> > > > > > > >> > > > topic
> > > > > > > >> > > > > > > owners
> > > > > > > >> > > > > > > > >> via
> > > > > > > >> > > > > > > > >> >> some casual topic name prefix, which is a
> > > bit
> > > > > > > awkward
> > > > > > > >> and
> > > > > > > >> > > > does
> > > > > > > >> > > > > > not
> > > > > > > >> > > > > > > > fly
> > > > > > > >> > > > > > > > >> as
> > > > > > > >> > > > > > > > >> >> we scale our customers. It would be great
> > to
> > > > use
> > > > > > > >> > > > > describe-topics
> > > > > > > >> > > > > > > such
> > > > > > > >> > > > > > > > >> as:
> > > > > > > >> > > > > > > > >> >>
> > > > > > > >> > > > > > > > >> >> Describe all topics that is created by me.
> > > > > > > >> > > > > > > > >> >>
> > > > > > > >> > > > > > > > >> >> Describe all topics whose retention time
> > is
> > > > > > > overriden
> > > > > > > >> to X.
> > > > > > > >> > > > > > > > >> >>
> > > > > > > >> > > > > > > > >> >> Describe all topics whose writable group
> > > > include
> > > > > > > user
> > > > > > > >> Y
> > > > > > > >> > > (this
> > > > > > > >> > > > > is
> > > > > > > >> > > > > > > > >> related to
> > > > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > > > >> > > > > > > > >> >>
> > > > > > > >> > > > > > > > >> >> One possible way to achieve this is to
> > add a
> > > > > > > metadata
> > > > > > > >> file
> > > > > > > >> > > in
> > > > > > > >> > > > > the
> > > > > > > >> > > > > > > > >> >> create-topic request, whose value will
> > also
> > > be
> > > > > > > >> written ZK
> > > > > > > >> > > as
> > > > > > > >> > > > we
> > > > > > > >> > > > > > > > create
> > > > > > > >> > > > > > > > >> the
> > > > > > > >> > > > > > > > >> >> topic; then describe-topics can choose to
> > > > batch
> > > > > > > topics
> > > > > > > >> > > based
> > > > > > > >> > > > on
> > > > > > > >> > > > > > 1)
> > > > > > > >> > > > > > > > name
> > > > > > > >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata
> > > > > regex,
> > > > > > > etc.
> > > > > > > >> > > > > > > > >> >>
> > > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > > >> > > > > > > > >> >>
> > > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > > >> > > > > > > > >> >>
> > > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang
> > > Wang
> > > > <
> > > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > > >> > > > > > > > >> wrote:
> > > > > > > >> > > > > > > > >> >>
> > > > > > > >> > > > > > > > >> >>> Thanks for the updated wiki. A few
> > comments
> > > > > below:
> > > > > > > >> > > > > > > > >> >>>
> > > > > > > >> > > > > > > > >> >>> 1. Error description in response: I think
> > > if
> > > > > some
> > > > > > > >> > > errorCode
> > > > > > > >> > > > > > could
> > > > > > > >> > > > > > > > >> indicate
> > > > > > > >> > > > > > > > >> >>> several different error cases then we
> > > should
> > > > > really
> > > > > > > >> change
> > > > > > > >> > > > it
> > > > > > > >> > > > > to
> > > > > > > >> > > > > > > > >> multiple
> > > > > > > >> > > > > > > > >> >>> codes. In general the errorCode itself
> > > would
> > > > be
> > > > > > > >> precise
> > > > > > > >> > > and
> > > > > > > >> > > > > > > > >> sufficient for
> > > > > > > >> > > > > > > > >> >>> describing the server side errors.
> > > > > > > >> > > > > > > > >> >>>
> > > > > > > >> > > > > > > > >> >>> 2. Describe topic request: it would be
> > > great
> > > > > to go
> > > > > > > >> beyond
> > > > > > > >> > > > just
> > > > > > > >> > > > > > > > >> batching on
> > > > > > > >> > > > > > > > >> >>> topic name regex for this request. For
> > > > > example, a
> > > > > > > >> very
> > > > > > > >> > > > common
> > > > > > > >> > > > > > use
> > > > > > > >> > > > > > > > >> case of
> > > > > > > >> > > > > > > > >> >>> the topic command is to list all topics
> > > whose
> > > > > > > config
> > > > > > > >> A's
> > > > > > > >> > > > value
> > > > > > > >> > > > > > is
> > > > > > > >> > > > > > > B.
> > > > > > > >> > > > > > > > >> With
> > > > > > > >> > > > > > > > >> >>> topic name regex then we have to first
> > > > retrieve
> > > > > > > >> __all__
> > > > > > > >> > > > > topics's
> > > > > > > >> > > > > > > > >> >>> description info and then filter at the
> > > > client
> > > > > end,
> > > > > > > >> which
> > > > > > > >> > > > will
> > > > > > > >> > > > > > be
> > > > > > > >> > > > > > > a
> > > > > > > >> > > > > > > > >> huge
> > > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > > >> > > > > > > > >> >>>
> > > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is
> > > > > related to
> > > > > > > >> the
> > > > > > > >> > > > > previous
> > > > > > > >> > > > > > > > point;
> > > > > > > >> > > > > > > > >> >>> maybe we can add another metadata K-V or
> > > > just a
> > > > > > > >> metadata
> > > > > > > >> > > > > string
> > > > > > > >> > > > > > > > along
> > > > > > > >> > > > > > > > >> side
> > > > > > > >> > > > > > > > >> >>> with config K-V in create topic like we
> > did
> > > > for
> > > > > > > >> offset
> > > > > > > >> > > > commit
> > > > > > > >> > > > > > > > >> request. This
> > > > > > > >> > > > > > > > >> >>> field can be quite useful in storing
> > > > > information
> > > > > > > like
> > > > > > > >> > > > "owner"
> > > > > > > >> > > > > of
> > > > > > > >> > > > > > > the
> > > > > > > >> > > > > > > > >> topic
> > > > > > > >> > > > > > > > >> >>> who issue the create command, etc, which
> > is
> > > > > quite
> > > > > > > >> > > important
> > > > > > > >> > > > > for
> > > > > > > >> > > > > > a
> > > > > > > >> > > > > > > > >> >>> multi-tenant setting. Then in the
> > describe
> > > > > topic
> > > > > > > >> request
> > > > > > > >> > > we
> > > > > > > >> > > > > can
> > > > > > > >> > > > > > > also
> > > > > > > >> > > > > > > > >> batch
> > > > > > > >> > > > > > > > >> >>> on regex of the metadata field.
> > > > > > > >> > > > > > > > >> >>>
> > > > > > > >> > > > > > > > >> >>> 4. Today all the admin operations are
> > async
> > > > in
> > > > > the
> > > > > > > >> sense
> > > > > > > >> > > > that
> > > > > > > >> > > > > > > > command
> > > > > > > >> > > > > > > > >> will
> > > > > > > >> > > > > > > > >> >>> return once it is written in ZK, and that
> > > is
> > > > > why we
> > > > > > > >> need
> > > > > > > >> > > > extra
> > > > > > > >> > > > > > > > >> verification
> > > > > > > >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() /
> > > verify
> > > > > > > >> partition
> > > > > > > >> > > > > > > reassignment
> > > > > > > >> > > > > > > > >> >>> request, etc. With admin requests we
> > could
> > > > add
> > > > > a
> > > > > > > >> flag to
> > > > > > > >> > > > > enable
> > > > > > > >> > > > > > /
> > > > > > > >> > > > > > > > >> disable
> > > > > > > >> > > > > > > > >> >>> synchronous requests; when it is turned
> > on,
> > > > the
> > > > > > > >> response
> > > > > > > >> > > > will
> > > > > > > >> > > > > > not
> > > > > > > >> > > > > > > > >> return
> > > > > > > >> > > > > > > > >> >>> until the request has been completed. And
> > > for
> > > > > async
> > > > > > > >> > > requests
> > > > > > > >> > > > > we
> > > > > > > >> > > > > > > can
> > > > > > > >> > > > > > > > >> add a
> > > > > > > >> > > > > > > > >> >>> "token" field in the response, and then
> > > only
> > > > > need a
> > > > > > > >> > > general
> > > > > > > >> > > > > > "admin
> > > > > > > >> > > > > > > > >> >>> verification request" with the given
> > token
> > > to
> > > > > check
> > > > > > > >> if the
> > > > > > > >> > > > > async
> > > > > > > >> > > > > > > > >> request
> > > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > > >> > > > > > > > >> >>>
> > > > > > > >> > > > > > > > >> >>> 5. +1 for extending Metadata request to
> > > > include
> > > > > > > >> > > controller /
> > > > > > > >> > > > > > > > >> coordinator
> > > > > > > >> > > > > > > > >> >>> information, and then we can remove the
> > > > > > > >> ConsumerMetadata /
> > > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > > >> > > > > > > > >> >>> requests.
> > > > > > > >> > > > > > > > >> >>>
> > > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > > >> > > > > > > > >> >>>
> > > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel
> > > Koshy <
> > > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > > >> > > > > > > > >> wrote:
> > > > > > > >> > > > > > > > >> >>>
> > > > > > > >> > > > > > > > >> >>>> Thanks for sending that out Joe - I
> > don't
> > > > > think I
> > > > > > > >> will be
> > > > > > > >> > > > > able
> > > > > > > >> > > > > > to
> > > > > > > >> > > > > > > > >> make
> > > > > > > >> > > > > > > > >> >>>> it today, so if notes can be sent out
> > > > > afterward
> > > > > > > that
> > > > > > > >> > > would
> > > > > > > >> > > > be
> > > > > > > >> > > > > > > > great.
> > > > > > > >> > > > > > > > >> >>>>
> > > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM
> > -0800,
> > > > Gwen
> > > > > > > >> Shapira
> > > > > > > >> > > > wrote:
> > > > > > > >> > > > > > > > >> >>>> > Thanks for sending this out Joe.
> > Looking
> > > > > forward
> > > > > > > >> to
> > > > > > > >> > > > > chatting
> > > > > > > >> > > > > > > with
> > > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe
> > > Stein
> > > > <
> > > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > > >> > > > > > > > >> wrote:
> > > > > > > >> > > > > > > > >> >>>> > > Hey, I just sent out a google
> > hangout
> > > > > invite
> > > > > > > to
> > > > > > > >> all
> > > > > > > >> > > > pmc,
> > > > > > > >> > > > > > > > >> committers
> > > > > > > >> > > > > > > > >> >>>> and
> > > > > > > >> > > > > > > > >> >>>> > > everyone I found working on a KIP.
> > If
> > > I
> > > > > missed
> > > > > > > >> anyone
> > > > > > > >> > > > in
> > > > > > > >> > > > > > the
> > > > > > > >> > > > > > > > >> invite
> > > > > > > >> > > > > > > > >> >>>> please
> > > > > > > >> > > > > > > > >> >>>> > > let me know and can update it, np.
> > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > >> > > > > > > > >> >>>> > > We should do this every Tuesday @
> > 2pm
> > > > > Eastern
> > > > > > > >> Time.
> > > > > > > >> > > > Maybe
> > > > > > > >> > > > > > we
> > > > > > > >> > > > > > > > can
> > > > > > > >> > > > > > > > >> get
> > > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > > >> > > > > > > > >> >>>> > > help to make a google account so we
> > > can
> > > > > manage
> > > > > > > >> > > better?
> > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > >> > > > > > > > >> >>>>
> > > > > > > >> > > > > > > > >>
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >>
> > > > > > >
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > >> > > > > > > > >> >>>> > > in progress and related JIRA that
> > are
> > > > > > > >> interdependent
> > > > > > > >> > > > and
> > > > > > > >> > > > > > > common
> > > > > > > >> > > > > > > > >> work.
> > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay
> > > > > Kreps <
> > > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > >> > > > > > > > >> >>>> > >
> > > > > > > >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that
> > > will
> > > > > also
> > > > > > > >> record
> > > > > > > >> > > > and
> > > > > > > >> > > > > > make
> > > > > > > >> > > > > > > > the
> > > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM,
> > > Jeff
> > > > > > > Holoman
> > > > > > > >> <
> > > > > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex
> > for
> > > > > this
> > > > > > > >> purpose.
> > > > > > > >> > > We
> > > > > > > >> > > > > > could
> > > > > > > >> > > > > > > > >> record
> > > > > > > >> > > > > > > > >> >>>> the
> > > > > > > >> > > > > > > > >> >>>> > >> > sessions if there is interest and
> > > > > publish
> > > > > > > >> them
> > > > > > > >> > > out.
> > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM,
> > > Jay
> > > > > > > Kreps <
> > > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > >> > > > > > > > >> >>>> > >> > > Let's try to get the technical
> > > > > hang-ups
> > > > > > > >> sorted
> > > > > > > >> > > > out,
> > > > > > > >> > > > > > > > though.
> > > > > > > >> > > > > > > > >> I
> > > > > > > >> > > > > > > > >> >>>> really
> > > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > > >> > > > > > > > >> >>>> > >> > > there is some benefit to live
> > > > > discussion
> > > > > > > vs
> > > > > > > >> > > > > writing. I
> > > > > > > >> > > > > > > am
> > > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > > >> > > > > > > > >> >>>> > >> > > we post instructions and give
> > > > > ourselves a
> > > > > > > >> few
> > > > > > > >> > > > > attempts
> > > > > > > >> > > > > > > we
> > > > > > > >> > > > > > > > >> can
> > > > > > > >> > > > > > > > >> >>>> get it
> > > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work
> > > for
> > > > > > > >> me...any
> > > > > > > >> > > > > > objections?
> > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18
> > AM,
> > > > Joe
> > > > > > > Stein
> > > > > > > >> <
> > > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > >> > > > > > > > >> >>>> >
> > > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe
> > > like
> > > > > every
> > > > > > > >> > > Tuesday ~
> > > > > > > >> > > > > 1pm
> > > > > > > >> > > > > > > ET
> > > > > > > >> > > > > > > > /
> > > > > > > >> > > > > > > > >> 10am
> > > > > > > >> > > > > > > > >> >>>> PT
> > > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout
> > but
> > > > > there
> > > > > > > is
> > > > > > > >> > > always
> > > > > > > >> > > > > some
> > > > > > > >> > > > > > > > >> issue or
> > > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > > >> > > > > > > > >> >>>> > >> > > > we know the apache irc
> > channel
> > > > > works.
> > > > > > > We
> > > > > > > >> can
> > > > > > > >> > > > start
> > > > > > > >> > > > > > > there
> > > > > > > >> > > > > > > > >> and
> > > > > > > >> > > > > > > > >> >>>> see how
> > > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts
> > > too
> > > > > and
> > > > > > > >> > > associate
> > > > > > > >> > > > to
> > > > > > > >> > > > > > > > >> tickets if
> > > > > > > >> > > > > > > > >> >>>> need be
> > > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > > >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10
> > > AM,
> > > > > Jay
> > > > > > > >> Kreps <
> > > > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a
> > > > Google
> > > > > > > >> Hangout to
> > > > > > > >> > > > chat
> > > > > > > >> > > > > > > about
> > > > > > > >> > > > > > > > >> this.
> > > > > > > >> > > > > > > > >> >>>> What
> > > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
> > > > > > > further...I
> > > > > > > >> > > > actually
> > > > > > > >> > > > > > > think
> > > > > > > >> > > > > > > > it
> > > > > > > >> > > > > > > > >> >>>> would be
> > > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > everyone spending a
> > > reasonable
> > > > > chunk
> > > > > > > of
> > > > > > > >> > > their
> > > > > > > >> > > > > week
> > > > > > > >> > > > > > > on
> > > > > > > >> > > > > > > > >> Kafka
> > > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I
> > think
> > > we
> > > > > could
> > > > > > > >> use
> > > > > > > >> > > time
> > > > > > > >> > > > > to
> > > > > > > >> > > > > > > talk
> > > > > > > >> > > > > > > > >> >>>> through
> > > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on
> > > top
> > > > of
> > > > > > > code
> > > > > > > >> > > > reviews,
> > > > > > > >> > > > > > talk
> > > > > > > >> > > > > > > > >> through
> > > > > > > >> > > > > > > > >> >>>> any
> > > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > We can make it publicly
> > > > > available so
> > > > > > > >> that
> > > > > > > >> > > any
> > > > > > > >> > > > > one
> > > > > > > >> > > > > > > can
> > > > > > > >> > > > > > > > >> follow
> > > > > > > >> > > > > > > > >> >>>> along
> > > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this?
> > > If
> > > > so
> > > > > > > I'll
> > > > > > > >> try
> > > > > > > >> > > to
> > > > > > > >> > > > > set
> > > > > > > >> > > > > > it
> > > > > > > >> > > > > > > > up
> > > > > > > >> > > > > > > > >> >>>> starting
> > > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at
> > 3:57
> > > > AM,
> > > > > > > Andrii
> > > > > > > >> > > > > Biletskyi
> > > > > > > >> > > > > > <
> > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > andrii.biletskyi@stealth.ly>
> > > > > wrote:
> > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page,
> > > fixed
> > > > /
> > > > > > > >> aligned
> > > > > > > >> > > > > document
> > > > > > > >> > > > > > > > >> structure.
> > > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
> > > > > > > >> AdminClient so
> > > > > > > >> > > we
> > > > > > > >> > > > > have
> > > > > > > >> > > > > > > > >> something
> > > > > > > >> > > > > > > > >> >>>> to
> > > > > > > >> > > > > > > > >> >>>> > >> start
> > > > > > > >> > > > > > > > >> >>>> > >> > > > from
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > while
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > > >> > > > > > > > >> >>>>
> > > > > > > >> > > > > > > > >>
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >>
> > > > > > >
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at
> > > 9:01
> > > > > PM,
> > > > > > > >> Andrii
> > > > > > > >> > > > > > Biletskyi
> > > > > > > >> > > > > > > <
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > andrii.biletskyi@stealth.ly>
> > > > > > > wrote:
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you
> > > are
> > > > > right,
> > > > > > > >> in
> > > > > > > >> > > most
> > > > > > > >> > > > > > cases
> > > > > > > >> > > > > > > > >> client
> > > > > > > >> > > > > > > > >> >>>> will
> > > > > > > >> > > > > > > > >> >>>> > >> have
> > > > > > > >> > > > > > > > >> >>>> > >> > > > enough
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > context to show
> > > descriptive
> > > > > error
> > > > > > > >> > > message.
> > > > > > > >> > > > > My
> > > > > > > >> > > > > > > > >> concern is
> > > > > > > >> > > > > > > > >> >>>> that
> > > > > > > >> > > > > > > > >> >>>> > >> we
> > > > > > > >> > > > > > > > >> >>>> > >> > > will
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > have
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > to
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error
> > > codes
> > > > > for
> > > > > > > >> each
> > > > > > > >> > > > > possible
> > > > > > > >> > > > > > > > >> error. Of
> > > > > > > >> > > > > > > > >> >>>> course,
> > > > > > > >> > > > > > > > >> >>>> > >> > we
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > could
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > reuse
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
> > > > > > > >> > > > > > > UknownTopicOrPartitionCode,
> > > > > > > >> > > > > > > > >> but we
> > > > > > > >> > > > > > > > >> >>>> will
> > > > > > > >> > > > > > > > >> >>>> > >> > also
> > > > > > > >> > > > > > > > >> >>>> > >> > > > need
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > to
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> > > > > > > >> TopicAlreadyExistsCode,
> > > > > > > >> > > > > > > > >> >>>> TopicConfigInvalid (both
> > > > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > topic
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > name and config, and
> > > > probably
> > > > > > > user
> > > > > > > >> would
> > > > > > > >> > > > > like
> > > > > > > >> > > > > > to
> > > > > > > >> > > > > > > > >> know
> > > > > > > >> > > > > > > > >> >>>> what
> > > > > > > >> > > > > > > > >> >>>> > >> > exactly
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his
> > config),
> > > > > > > >> > > > > > > InvalidReplicaAssignment,
> > > > > > > >> > > > > > > > >> >>>> > >> InternalError
> > > > > > > >> > > > > > > > >> >>>> > >> > > > (e.g.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
> > > > > > > TopicCommand,
> > > > > > > >> we
> > > > > > > >> > > will
> > > > > > > >> > > > > > also
> > > > > > > >> > > > > > > > >> need to
> > > > > > > >> > > > > > > > >> >>>> add
> > > > > > > >> > > > > > > > >> >>>> > >> > similar
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > for
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> > > > > > > >> PreferredReplica. So
> > > > > > > >> > > > > we'll
> > > > > > > >> > > > > > > end
> > > > > > > >> > > > > > > > >> up
> > > > > > > >> > > > > > > > >> >>>> with a
> > > > > > > >> > > > > > > > >> >>>> > >> > large
> > > > > > > >> > > > > > > > >> >>>> > >> > > > list
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > of
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only
> > in
> > > > > Admin
> > > > > > > >> > > protocol.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I
> > agree
> > > > my
> > > > > > > >> proposal is
> > > > > > > >> > > > not
> > > > > > > >> > > > > > > > >> consistent
> > > > > > > >> > > > > > > > >> >>>> with
> > > > > > > >> > > > > > > > >> >>>> > >> > other
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > cases.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find
> > better
> > > > > solution
> > > > > > > >> or
> > > > > > > >> > > > > something
> > > > > > > >> > > > > > > > >> >>>> in-between.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I
> > think
> > > it
> > > > > is a
> > > > > > > >> great
> > > > > > > >> > > > idea.
> > > > > > > >> > > > > > > This
> > > > > > > >> > > > > > > > >> way we
> > > > > > > >> > > > > > > > >> >>>> can
> > > > > > > >> > > > > > > > >> >>>> > >> move
> > > > > > > >> > > > > > > > >> >>>> > >> > > on
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > faster.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on
> > > > > date/time
> > > > > > > so
> > > > > > > >> > > people
> > > > > > > >> > > > > can
> > > > > > > >> > > > > > > > join.
> > > > > > > >> > > > > > > > >> >>>> Will work
> > > > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > > > >> > > > > > > > >> >>>> > >> > > me
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > and
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > next week almost
> > anytime
> > > if
> > > > > > > agreed
> > > > > > > >> in
> > > > > > > >> > > > > advance.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at
> > > > 7:09
> > > > > PM,
> > > > > > > >> Jay
> > > > > > > >> > > > Kreps <
> > > > > > > >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > wrote:
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do
> > good
> > > > > error
> > > > > > > >> handling
> > > > > > > >> > > > > > without
> > > > > > > >> > > > > > > > >> needing
> > > > > > > >> > > > > > > > >> >>>> custom
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > server-side
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e.
> > generally
> > > > the
> > > > > > > >> client has
> > > > > > > >> > > > the
> > > > > > > >> > > > > > > > >> context to
> > > > > > > >> > > > > > > > >> >>>> know
> > > > > > > >> > > > > > > > >> >>>> > >> that
> > > > > > > >> > > > > > > > >> >>>> > >> > > if
> > > > > > > >> > > > > > > > >> >>>> > >> > > > it
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > got
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> an error that the
> > topic
> > > > > doesn't
> > > > > > > >> exist
> > > > > > > >> > > to
> > > > > > > >> > > > > say
> > > > > > > >> > > > > > > > >> "Topic X
> > > > > > > >> > > > > > > > >> >>>> doesn't
> > > > > > > >> > > > > > > > >> >>>> > >> > > exist"
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> rather
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14"
> > (or
> > > > > > > >> whatever).
> > > > > > > >> > > Maybe
> > > > > > > >> > > > > > there
> > > > > > > >> > > > > > > > are
> > > > > > > >> > > > > > > > >> >>>> specific
> > > > > > > >> > > > > > > > >> >>>> > >> > cases
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > where
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we
> > want
> > > > to
> > > > > add
> > > > > > > >> > > > server-side
> > > > > > > >> > > > > > > error
> > > > > > > >> > > > > > > > >> >>>> messages we
> > > > > > > >> > > > > > > > >> >>>> > >> > > really
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > do
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a
> > > > > consistent
> > > > > > > >> way
> > > > > > > >> > > > across
> > > > > > > >> > > > > > the
> > > > > > > >> > > > > > > > >> protocol.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch
> > of
> > > > open
> > > > > > > >> questions
> > > > > > > >> > > > here
> > > > > > > >> > > > > > > from
> > > > > > > >> > > > > > > > my
> > > > > > > >> > > > > > > > >> >>>> previous
> > > > > > > >> > > > > > > > >> >>>> > >> > > list. I
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > will
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next
> > few
> > > > > days for
> > > > > > > >> Strata
> > > > > > > >> > > > > > though.
> > > > > > > >> > > > > > > > >> Maybe
> > > > > > > >> > > > > > > > >> >>>> we could
> > > > > > > >> > > > > > > > >> >>>> > >> > do
> > > > > > > >> > > > > > > > >> >>>> > >> > > a
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > Google
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any
> > open
> > > > > issues
> > > > > > > >> some
> > > > > > > >> > > time
> > > > > > > >> > > > > > > towards
> > > > > > > >> > > > > > > > >> the
> > > > > > > >> > > > > > > > >> >>>> end of
> > > > > > > >> > > > > > > > >> >>>> > >> > next
> > > > > > > >> > > > > > > > >> >>>> > >> > > > week
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > for
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in
> > > this
> > > > > > > ticket?
> > > > > > > >> I
> > > > > > > >> > > have
> > > > > > > >> > > > a
> > > > > > > >> > > > > > > > feeling
> > > > > > > >> > > > > > > > >> that
> > > > > > > >> > > > > > > > >> >>>> might
> > > > > > > >> > > > > > > > >> >>>> > >> > > > progress
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster
> > > > than
> > > > > > > >> email--I
> > > > > > > >> > > > think
> > > > > > > >> > > > > we
> > > > > > > >> > > > > > > > >> could talk
> > > > > > > >> > > > > > > > >> >>>> > >> through
> > > > > > > >> > > > > > > > >> >>>> > >> > > > those
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up
> > > fairly
> > > > > > > >> quickly...
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015
> > at
> > > > > 7:27 AM,
> > > > > > > >> Andrii
> > > > > > > >> > > > > > > > Biletskyi <
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > andrii.biletskyi@stealth.ly
> > > > > >
> > > > > > > >> wrote:
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to
> > address
> > > > > some of
> > > > > > > >> the
> > > > > > > >> > > > issues
> > > > > > > >> > > > > > > which
> > > > > > > >> > > > > > > > >> were
> > > > > > > >> > > > > > > > >> >>>> > >> mentioned
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > earlier
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> about
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format.
> > > One
> > > > of
> > > > > > > >> those was
> > > > > > > >> > > > > about
> > > > > > > >> > > > > > > > >> batching
> > > > > > > >> > > > > > > > >> >>>> > >> > operations.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > What
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > if
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> we
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand
> > > > > approach
> > > > > > > >> and let
> > > > > > > >> > > > > people
> > > > > > > >> > > > > > > > >> specify
> > > > > > > >> > > > > > > > >> >>>> > >> topic-name
> > > > > > > >> > > > > > > > >> >>>> > >> > > by
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover
> > most
> > > of
> > > > > the
> > > > > > > use
> > > > > > > >> > > cases?
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what
> > > > > information
> > > > > > > >> should
> > > > > > > >> > > we
> > > > > > > >> > > > > > > > generally
> > > > > > > >> > > > > > > > >> >>>> provide in
> > > > > > > >> > > > > > > > >> >>>> > >> > > Admin
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin
> > > > > commands
> > > > > > > >> don't
> > > > > > > >> > > > imply
> > > > > > > >> > > > > > > they
> > > > > > > >> > > > > > > > >> will
> > > > > > > >> > > > > > > > >> >>>> be used
> > > > > > > >> > > > > > > > >> >>>> > >> > only
> > > > > > > >> > > > > > > > >> >>>> > >> > > > in
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > CLI
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > but,
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI
> > > is a
> > > > > very
> > > > > > > >> > > important
> > > > > > > >> > > > > > > client
> > > > > > > >> > > > > > > > >> of this
> > > > > > > >> > > > > > > > >> >>>> > >> > feature.
> > > > > > > >> > > > > > > > >> >>>> > >> > > In
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > case,
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we
> > > would
> > > > > like
> > > > > > > to
> > > > > > > >> > > provide
> > > > > > > >> > > > > > users
> > > > > > > >> > > > > > > > >> with
> > > > > > > >> > > > > > > > >> >>>> rich
> > > > > > > >> > > > > > > > >> >>>> > >> > > experience
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > in
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> terms
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > of
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > getting results /
> > > errors
> > > > > of
> > > > > > > the
> > > > > > > >> > > > executed
> > > > > > > >> > > > > > > > >> commands.
> > > > > > > >> > > > > > > > >> >>>> Usually
> > > > > > > >> > > > > > > > >> >>>> > >> we
> > > > > > > >> > > > > > > > >> >>>> > >> > > > supply
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> with
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses only
> > > > errorCode,
> > > > > > > which
> > > > > > > >> looks
> > > > > > > >> > > > > very
> > > > > > > >> > > > > > > > >> limiting,
> > > > > > > >> > > > > > > > >> >>>> in case
> > > > > > > >> > > > > > > > >> >>>> > >> > of
> > > > > > > >> > > > > > > > >> >>>> > >> > > > CLI
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > we
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> may
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human
> > > > > readable
> > > > > > > >> error
> > > > > > > >> > > > > > > description.
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into
> > > account
> > > > > > > >> previous item
> > > > > > > >> > > > > about
> > > > > > > >> > > > > > > > >> batching,
> > > > > > > >> > > > > > > > >> >>>> what
> > > > > > > >> > > > > > > > >> >>>> > >> do
> > > > > > > >> > > > > > > > >> >>>> > >> > > you
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > think
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > about
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't
> > > > support
> > > > > > > >> regexp)
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest
> > =>
> > > > > > > TopicName
> > > > > > > >> > > > > Partitions
> > > > > > > >> > > > > > > > >> Replicas
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse
> > =>
> > > > > > > ErrorCode
> > > > > > > >> > > > > > > > ErrorDescription
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription
> > =>
> > > > > string
> > > > > > > >> (empty
> > > > > > > >> > > if
> > > > > > > >> > > > > > > > >> successful)
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
> > > > > > > >> TopicNameRegexp
> > > > > > > >> > > >
> > > > > > > >
> > > > > > > > ...
> > > > > > > >
> > > > > > > > [Message clipped]
> > > > > > >
> > > > >
> > > > >
> > > >
> > >
> >


Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Joel,

I'm totally behind your arguments concerning adding irrelevant stuff to
TopicMetadataRequest. And also about having a bloated request.

Personally I'd go with a separate ClusterMetadataRequest (CMR), actually
this was our initial proposal. But since the second part of the request -
brokers
is already present in TopicMetadataResponse (TMR) I agreed to augment
TMR instead of introducing a separate request.

The only thing which should be considered though is kafka producer /
consumer.
If we split TMR to topic metadata and cluster metadata (brokers +
controller) we
need to think whether it's okay if clients would have to issue two separate
requests to maintain Metadata.java (in terms of potential concurrency
issue).
Can someone please clarify this question?

Thanks,
Andrii Biletskyi


On Wed, Mar 18, 2015 at 8:58 PM, Joel Koshy <jj...@gmail.com> wrote:

> (Thanks Andrii for the summary)
>
> For (1) yes we will circle back on that shortly after syncing up in
> person. I think it is close to getting committed although development
> for KAFKA-1927 can probably begin without it.
>
> There is one more item we covered at the hangout. i.e., whether we
> want to add the coordinator to the topic metadata response or provide
> a clearer ClusterMetadataRequest.
>
> There are two reasons I think we should try and avoid adding the
> field:
> - It is irrelevant to topic metadata
> - If we finally do request rerouting in Kafka then the field would add
>   little to no value. (It still helps to have a separate
>   ClusterMetadataRequest to query for cluster-wide information such as
>   'which broker is the controller?' as Joe mentioned.)
>
> I think it would be cleaner to have an explicit ClusterMetadataRequest
> that you can send to any broker in order to obtain the controller (and
> in the future possibly other cluster-wide information). I think the
> main argument against doing this and instead adding it to the topic
> metadata response was convenience - i.e., you don't have to discover
> the controller in advance. However, I don't see much actual
> benefit/convenience in this and in fact think it is a non-issue. Let
> me know if I'm overlooking something here.
>
> As an example, say we need to initiate partition reassignment by
> issuing the new ReassignPartitionsRequest to the controller (assume we
> already have the desired manual partition assignment).  If we are to
> augment topic metadata response then the flow be something like this :
>
> - Issue topic metadata request to any broker (and discover the
>   controller
> - Connect to controller if required (i.e., if the broker above !=
>   controller)
> - Issue the partition reassignment request to the controller.
>
> With an explicit cluster metadata request it would be:
> - Issue cluster metadata request to any broker
> - Connect to controller if required (i.e., if the broker above !=
>   controller)
> - Issue the partition reassignment request
>
> So it seems to add little practical value and bloats topic metadata
> response with an irrelevant detail.
>
> The other angle to this is the following - is it a matter of naming?
> Should we just rename topic metadata request/response to just
> MetadataRequest/Response and add cluster metadata to it? By that same
> token should we also allow querying for the consumer coordinator (and
> in future transaction coordinator) as well? This leads to a bloated
> request which isn't very appealing and altogether confusing.
>
> Thanks,
>
> Joel
>
> On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > Andri,
> >
> > Thanks for the summary.
> >
> > 1. I just realized that in order to start working on KAFKA-1927, we will
> > need to merge the changes to OffsetCommitRequest (from 0.8.2) to trunk.
> > This is planned to be done as part of KAFKA-1634. So, we will need
> Guozhang
> > and Joel's help to wrap this up.
> >
> > 2. Thinking about this a bit more, if the semantic of those "write"
> > requests is async (i.e., after the client gets a response, it just means
> > that the operation is initiated, but not necessarily completed), we don't
> > really need to forward the requests to the controller. Instead, the
> > receiving broker can just write the operation to ZK as the admin command
> > line tool previously does. This will simplify the implementation.
> >
> > 8. There is another implementation detail for describe topic. Ideally, we
> > want to read the topic config from the broker cache, instead of
> ZooKeeper.
> > Currently, every broker reads the topic-level config for all topics.
> > However, it ignores those for topics not hosted on itself. So, we may
> need
> > to change TopicConfigManager a bit so that it caches the configs for all
> > topics.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Guys,
> > >
> > > Thanks for a great discussion!
> > > Here are the actions points:
> > >
> > > 1. Q: Get rid of all scala requests objects, use java protocol
> definitions.
> > >     A: Gwen kindly took that (KAFKA-1927). It's important to speed up
> > > review procedure
> > >          there since this ticket blocks other important changes.
> > >
> > > 2. Q: Generic re-reroute facility vs client maintaining cluster state.
> > >     A: Jay has added pseudo code to KAFKA-1912 - need to consider
> whether
> > > this will be
> > >         easy to implement as a server-side feature (comments are
> > > welcomed!).
> > >
> > > 3. Q: Controller field in wire protocol.
> > >     A: This might be useful for clients, add this to
> TopicMetadataResponse
> > > (already in KIP).
> > >
> > > 4. Q: Decoupling topic creation from TMR.
> > >     A: I will add proposed by Jun solution (using clientId for that)
> to the
> > > KIP.
> > >
> > > 5. Q: Bumping new versions of TMR vs grabbing all protocol changes in
> one
> > > version.
> > >     A: It was decided to try to gather all changes to protocol (before
> > > release).
> > >         In case of TMR it worth checking: KAFKA-2020 and KIP-13
> (quotas)
> > >
> > > 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
> > >     A: Use jackson for that, /tools project is a separate jar so
> shouldn't
> > > be a big deal.
> > >
> > > 7.  Q: VerifyReassingPartitions vs generic status check command.
> > >      A: For long-running requests like reassign partitions *progress*
> check
> > > request is useful,
> > >          it makes sense to introduce it.
> > >
> > >  Please add, correct me if I missed something.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Joel,
> > > >
> > > > You are right, I removed ClusterMetadata because we have partially
> > > > what we need in TopicMetadata. Also, as Jay pointed out earlier, we
> > > > would like to have "orthogonal" API, but at the same time we need
> > > > to be backward compatible.
> > > >
> > > > But I like your idea and even have some other arguments for this
> option:
> > > > There is also DescribeTopicRequest which was proposed in this KIP,
> > > > it returns topic configs, partitions, replication factor plus
> partition
> > > > ISR, ASR,
> > > > leader replica. The later part is really already there in
> > > > TopicMetadataRequest.
> > > > So again we'll have to add stuff to TMR, not to duplicate some info
> in
> > > > newly added requests. However, this way we'll end up with "monster"
> > > > request which returns cluster metadata, topic replication and config
> info
> > > > plus partition replication data. Seems logical to split TMR to
> > > > - ClusterMetadata (brokers + controller, maybe smth else)
> > > > - TopicMetadata (topic info + partition details)
> > > > But since current TMR is involved in lots of places (including
> network
> > > > client,
> > > > as I understand) this might be very serious change and it probably
> makes
> > > > sense to stick with current approach.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > >
> > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jj...@gmail.com>
> wrote:
> > > >
> > > >> I may be missing some context but hopefully this will also be
> covered
> > > >> today: I thought the earlier proposal where there was an explicit
> > > >> ClusterMetadata request was clearer and explicit. During the course
> of
> > > >> this thread I think the conclusion was that the main need was for
> > > >> controller information and that can be rolled into the topic
> metadata
> > > >> response but that seems a bit irrelevant to topic metadata. FWIW I
> > > >> think the full broker-list is also irrelevant to topic metadata, but
> > > >> it is already there and in use. I think there is still room for an
> > > >> explicit ClusterMetadata request since there may be other
> > > >> cluster-level information that we may want to add over time (and
> that
> > > >> have nothing to do with topic metadata).
> > > >>
> > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
> > > >> > Jun,
> > > >> >
> > > >> > 101. Okay, if you say that such use case is important. I also
> think
> > > >> > using clientId for these purposes is fine - if we already have
> this
> > > >> field
> > > >> > as part of all Wire protocol messages, why not use that.
> > > >> > I will update KIP-4 page if nobody has other ideas (which may
> come up
> > > >> > during the call today).
> > > >> >
> > > >> > 102.1 Agree, I'll update the KIP accordingly. I think we can add
> new,
> > > >> > fine-grained error codes if some error code received in specific
> case
> > > >> > won't give enough context to return a descriptive error message
> for
> > > >> user.
> > > >> >
> > > >> > Look forward to discussing all outstanding issues in detail today
> > > during
> > > >> > the call.
> > > >> >
> > > >> > Thanks,
> > > >> > Andrii Biletskyi
> > > >> >
> > > >> >
> > > >> >
> > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > >> >
> > > >> > > 101. There may be a use case where you only want the topics to
> be
> > > >> created
> > > >> > > manually by admins. Currently, you can do that by disabling auto
> > > topic
> > > >> > > creation and issue topic creation from the TopicCommand. If we
> > > >> disable auto
> > > >> > > topic creation completely on the broker and don't have a way to
> > > >> distinguish
> > > >> > > between topic creation requests from the regular clients and the
> > > >> admin, we
> > > >> > > can't support manual topic creation any more. I was thinking
> that
> > > >> another
> > > >> > > way of distinguishing the clients making the topic creation
> requests
> > > >> is
> > > >> > > using clientId. For example, the admin tool can set it to
> something
> > > >> like
> > > >> > > admin and the broker can treat that clientId specially.
> > > >> > >
> > > >> > > Also, there is a related discussion in KAFKA-2020. Currently,
> we do
> > > >> the
> > > >> > > following in TopicMetadataResponse:
> > > >> > >
> > > >> > > 1. If leader is not available, we set the partition level error
> code
> > > >> to
> > > >> > > LeaderNotAvailable.
> > > >> > > 2. If a non-leader replica is not available, we take that
> replica
> > > out
> > > >> of
> > > >> > > the assigned replica list and isr in the response. As an
> indication
> > > >> for
> > > >> > > doing that, we set the partition level error code to
> > > >> ReplicaNotAvailable.
> > > >> > >
> > > >> > > This has a few problems. First, ReplicaNotAvailable probably
> > > >> shouldn't be
> > > >> > > an error, at least for the normal producer/consumer clients that
> > > just
> > > >> want
> > > >> > > to find out the leader. Second, it can happen that both the
> leader
> > > and
> > > >> > > another replica are not available at the same time. There is no
> > > error
> > > >> code
> > > >> > > to indicate both. Third, even if a replica is not available,
> it's
> > > >> still
> > > >> > > useful to return its replica id since some clients (e.g. admin
> tool)
> > > >> may
> > > >> > > still make use of it.
> > > >> > >
> > > >> > > One way to address this issue is to always return the replica
> id for
> > > >> > > leader, assigned replicas, and isr regardless of whether the
> > > >> corresponding
> > > >> > > broker is live or not. Since we also return the list of live
> > > brokers,
> > > >> the
> > > >> > > client can figure out whether a leader or a replica is live or
> not
> > > >> and act
> > > >> > > accordingly. This way, we don't need to set the partition level
> > > error
> > > >> code
> > > >> > > when the leader or a replica is not available. This doesn't
> change
> > > >> the wire
> > > >> > > protocol, but does change the semantics. Since we are evolving
> the
> > > >> protocol
> > > >> > > of TopicMetadataRequest here, we can potentially piggyback the
> > > change.
> > > >> > >
> > > >> > > 102.1 For those types of errors due to invalid input, shouldn't
> we
> > > >> just
> > > >> > > guard it at parameter validation time and throw
> > > >> InvalidArgumentException
> > > >> > > without even sending the request to the broker?
> > > >> > >
> > > >> > > Thanks,
> > > >> > >
> > > >> > > Jun
> > > >> > >
> > > >> > >
> > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > >
> > > >> > > > Jun,
> > > >> > > >
> > > >> > > > Answering your questions:
> > > >> > > >
> > > >> > > > 101. If I understand you correctly, you are saying future
> producer
> > > >> > > versions
> > > >> > > > (which
> > > >> > > > will be ported to TMR_V1) won't be able to automatically
> create
> > > >> topic (if
> > > >> > > > we
> > > >> > > > unconditionally remove topic creation from there). But we
> need to
> > > >> this
> > > >> > > > preserve logic.
> > > >> > > > Ok, about your proposal: I'm not a big fan too, when it comes
> to
> > > >> > > > differentiating
> > > >> > > > clients directly in protocol schema. And also I'm not sure I
> > > >> understand
> > > >> > > at
> > > >> > > > all why
> > > >> > > > auto.create.topics.enable is a server side configuration. Can
> we
> > > >> > > deprecate
> > > >> > > > this setting
> > > >> > > > in future versions, add this setting to producer and based on
> that
> > > >> upon
> > > >> > > > receiving
> > > >> > > > UnknownTopic create topic explicitly by a separate producer
> call
> > > via
> > > >> > > > adminClient?
> > > >> > > >
> > > >> > > > 102.1. Hm, yes. It's because we want to support batching and
> at
> > > the
> > > >> same
> > > >> > > > time we
> > > >> > > > want to give descriptive error messages for clients. Since
> > > >> AdminClient
> > > >> > > > holds the context
> > > >> > > > to construct such messages (e.g. AdminClient layer can know
> that
> > > >> > > > InvalidArgumentsCode
> > > >> > > > means two cases: either invalid number - e.g. -1; or
> > > >> replication-factor
> > > >> > > was
> > > >> > > > provided while
> > > >> > > > partitions argument wasn't) - I wrapped responses in
> Exceptions.
> > > >> But I'm
> > > >> > > > open to any
> > > >> > > > other ideas, this was just initial version.
> > > >> > > > 102.2. Yes, I agree. I'll change that to probably some other
> dto.
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > > Andrii Biletskyi
> > > >> > > >
> > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > >> > > >
> > > >> > > > > Andrii,
> > > >> > > > >
> > > >> > > > > 101. That's what I was thinking too, but it may not be that
> > > >> simple. In
> > > >> > > > > TopicMetadataRequest_V1,
> > > >> > > > > we can let it not trigger auto topic creation. Then, in the
> > > >> producer
> > > >> > > > side,
> > > >> > > > > if it gets an UnknownTopicException, it can explicitly
> issue a
> > > >> > > > > createTopicRequest for auto topic creation. On the consumer
> > > side,
> > > >> it
> > > >> > > will
> > > >> > > > > never issue createTopicRequest. This works when auto topic
> > > >> creation is
> > > >> > > > > enabled on the broker side. However, I am not sure how
> things
> > > >> will work
> > > >> > > > > when auto topic creation is disabled on the broker side. In
> this
> > > >> case,
> > > >> > > we
> > > >> > > > > want to have a way to manually create a topic, potentially
> > > through
> > > >> > > admin
> > > >> > > > > commands. However, then we need a way to distinguish
> > > >> createTopicRequest
> > > >> > > > > issued from the producer clients and the admin tools. May
> be we
> > > >> can
> > > >> > > add a
> > > >> > > > > new field in createTopicRequest and set it differently in
> the
> > > >> producer
> > > >> > > > > client and the admin client. However, I am not sure if
> that's
> > > the
> > > >> best
> > > >> > > > > approach.
> > > >> > > > >
> > > >> > > > > 2. Yes, refactoring existing requests is a non-trivial
> amount of
> > > >> work.
> > > >> > > I
> > > >> > > > > posted some comments in KAFKA-1927. We will probably have
> to fix
> > > >> > > > KAFKA-1927
> > > >> > > > > first, before adding the new logic in KAFKA-1694.
> Otherwise, the
> > > >> > > changes
> > > >> > > > > will be too big.
> > > >> > > > >
> > > >> > > > > 102. About the AdminClient:
> > > >> > > > > 102.1. It's a bit weird that we return exception in the
> api. It
> > > >> seems
> > > >> > > > that
> > > >> > > > > we should either return error code or throw an exception
> when
> > > >> getting
> > > >> > > the
> > > >> > > > > response state.
> > > >> > > > > 102.2. We probably shouldn't explicitly use the request
> object
> > > in
> > > >> the
> > > >> > > > api.
> > > >> > > > > Not every request evolution requires an api change.
> > > >> > > > >
> > > >> > > > > Thanks,
> > > >> > > > >
> > > >> > > > > Jun
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > > > >
> > > >> > > > > > Jun,
> > > >> > > > > >
> > > >> > > > > > Thanks for you comments. Answers inline:
> > > >> > > > > >
> > > >> > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > >> > > > > > > ReassignPartitionRequest,
> > > >> > > > > > > and PartitionsSerialized that are represented as a
> string,
> > > but
> > > >> > > > contain
> > > >> > > > > > > composite structures in json. Could we flatten them out
> > > >> directly in
> > > >> > > > the
> > > >> > > > > > > protocol definition as arrays/records?
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > Yes, now with Admin Client this looks a bit weird. My
> initial
> > > >> > > > motivation
> > > >> > > > > > was:
> > > >> > > > > > ReassignPartitionCommand accepts input in json, we want to
> > > >> remain
> > > >> > > > tools'
> > > >> > > > > > interfaces unchanged, where possible.
> > > >> > > > > > If we port it to deserialized format, in CLI (/tools
> project)
> > > >> we will
> > > >> > > > > have
> > > >> > > > > > to add some
> > > >> > > > > > json library since /tools is written in java and we'll
> need to
> > > >> > > > > deserialize
> > > >> > > > > > json file
> > > >> > > > > > provided by a user. Can we quickly agree on what this
> library
> > > >> should
> > > >> > > be
> > > >> > > > > > (Jackson, GSON, whatever)?
> > > >> > > > > >
> > > >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> > > >> creation?
> > > >> > > > This
> > > >> > > > > > > will be a bit weird now that we have a separate topic
> > > >> creation api.
> > > >> > > > > Have
> > > >> > > > > > > you thought about how the new createTopicRequest and
> > > >> > > > > TopicMetadataRequest
> > > >> > > > > > > v1 will be used in the producer/consumer client, in
> addition
> > > >> to
> > > >> > > admin
> > > >> > > > > > > tools? For example, ideally, we don't want
> > > >> TopicMetadataRequest
> > > >> > > from
> > > >> > > > > the
> > > >> > > > > > > consumer to trigger auto topic creation.
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > I agree, this strange logic should be fixed. I'm not
> confident
> > > >> in
> > > >> > > this
> > > >> > > > > > Kafka part so
> > > >> > > > > > correct me if I'm wrong, but it doesn't look like a hard
> thing
> > > >> to
> > > >> > > do, I
> > > >> > > > > > think we can
> > > >> > > > > > leverage AdminClient for that in Producer and
> unconditionally
> > > >> remove
> > > >> > > > > topic
> > > >> > > > > > creation from the TopicMetadataRequest_V1.
> > > >> > > > > >
> > > >> > > > > > 2. I think Jay meant getting rid of scala classes
> > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > >> HeartbeatResponseAndHeader. We
> > > >> > > did
> > > >> > > > > > that
> > > >> > > > > > > as a stop-gap thing when adding the new requests for the
> > > >> consumers.
> > > >> > > > > > > However, the long term plan is to get rid of all those
> and
> > > >> just
> > > >> > > reuse
> > > >> > > > > the
> > > >> > > > > > > java request/response in the client. Since this KIP
> proposes
> > > >> to
> > > >> > > add a
> > > >> > > > > > > significant number of new requests, perhaps we should
> bite
> > > the
> > > >> > > bullet
> > > >> > > > > to
> > > >> > > > > > > clean up the existing scala requests first before
> adding new
> > > >> ones?
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > > > Yes, looks like I misunderstood the point of
> > > >> ...RequestAndHeader.
> > > >> > > > Okay, I
> > > >> > > > > > will
> > > >> > > > > > rework that. The only thing is that I don't see any
> example
> > > how
> > > >> it
> > > >> > > was
> > > >> > > > > done
> > > >> > > > > > for at
> > > >> > > > > > least one existing protocol message. Thus, as I
> understand, I
> > > >> have to
> > > >> > > > > think
> > > >> > > > > > how we
> > > >> > > > > > are going to do it.
> > > >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
> > > reasonable,
> > > >> but
> > > >> > > if
> > > >> > > > > it's
> > > >> > > > > > an *obligatory*
> > > >> > > > > > requirement to have Admin KIP done, I'm afraid this can
> be a
> > > >> serious
> > > >> > > > > > blocker for us.
> > > >> > > > > > There are 13 protocol messages and all that would require
> not
> > > >> only
> > > >> > > unit
> > > >> > > > > > tests but quite
> > > >> > > > > > intensive manual testing, no? I'm afraid I'm not the
> right guy
> > > >> to
> > > >> > > cover
> > > >> > > > > > pretty much all
> > > >> > > > > > Kafka core internals :). Let me know your thoughts on this
> > > >> item. Btw
> > > >> > > > > there
> > > >> > > > > > is a ticket to
> > > >> > > > > > follow-up this issue (
> > > >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > >> > > > ).
> > > >> > > > > >
> > > >> > > > > > Thanks,
> > > >> > > > > > Andrii Biletskyi
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <
> jun@confluent.io>
> > > >> wrote:
> > > >> > > > > >
> > > >> > > > > > > Andrii,
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > A few more comments.
> > > >> > > > > > >
> > > >> > > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > >> > > > > > > ReassignPartitionRequest,
> > > >> > > > > > > and PartitionsSerialized that are represented as a
> string,
> > > but
> > > >> > > > contain
> > > >> > > > > > > composite structures in json. Could we flatten them out
> > > >> directly in
> > > >> > > > the
> > > >> > > > > > > protocol definition as arrays/records?
> > > >> > > > > > >
> > > >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto
> topic
> > > >> > > creation?
> > > >> > > > > This
> > > >> > > > > > > will be a bit weird now that we have a separate topic
> > > >> creation api.
> > > >> > > > > Have
> > > >> > > > > > > you thought about how the new createTopicRequest and
> > > >> > > > > TopicMetadataRequest
> > > >> > > > > > > v1 will be used in the producer/consumer client, in
> addition
> > > >> to
> > > >> > > admin
> > > >> > > > > > > tools? For example, ideally, we don't want
> > > >> TopicMetadataRequest
> > > >> > > from
> > > >> > > > > the
> > > >> > > > > > > consumer to trigger auto topic creation.
> > > >> > > > > > >
> > > >> > > > > > > 2. I think Jay meant getting rid of scala classes
> > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > >> HeartbeatResponseAndHeader. We
> > > >> > > did
> > > >> > > > > > that
> > > >> > > > > > > as a stop-gap thing when adding the new requests for the
> > > >> consumers.
> > > >> > > > > > > However, the long term plan is to get rid of all those
> and
> > > >> just
> > > >> > > reuse
> > > >> > > > > the
> > > >> > > > > > > java request/response in the client. Since this KIP
> proposes
> > > >> to
> > > >> > > add a
> > > >> > > > > > > significant number of new requests, perhaps we should
> bite
> > > the
> > > >> > > bullet
> > > >> > > > > to
> > > >> > > > > > > clean up the existing scala requests first before
> adding new
> > > >> ones?
> > > >> > > > > > >
> > > >> > > > > > > Thanks,
> > > >> > > > > > >
> > > >> > > > > > > Jun
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Hi,
> > > >> > > > > > > >
> > > >> > > > > > > > As said above - I list again all comments from this
> thread
> > > >> so we
> > > >> > > > > > > > can see what's left and finalize all pending issues.
> > > >> > > > > > > >
> > > >> > > > > > > > Comments from Jay:
> > > >> > > > > > > > 1. This is much needed functionality, but there are a
> lot
> > > >> of the
> > > >> > > so
> > > >> > > > > > let's
> > > >> > > > > > > > really think these protocols through. We really want
> to
> > > end
> > > >> up
> > > >> > > > with a
> > > >> > > > > > set
> > > >> > > > > > > > of well thought-out, orthoganol apis. For this reason
> I
> > > >> think it
> > > >> > > is
> > > >> > > > > > > really
> > > >> > > > > > > > important to think through the end state even if that
> > > >> includes
> > > >> > > APIs
> > > >> > > > > we
> > > >> > > > > > > > won't implement in the first phase.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Definitely behind this. Would appreciate if there
> are
> > > >> concrete
> > > >> > > > > > > comments
> > > >> > > > > > > > how this can be improved.
> > > >> > > > > > > >
> > > >> > > > > > > > 2. Let's please please please wait until we have
> switched
> > > >> the
> > > >> > > > server
> > > >> > > > > > over
> > > >> > > > > > > > to the new java protocol definitions. If we add upteen
> > > more
> > > >> ad
> > > >> > > hoc
> > > >> > > > > > scala
> > > >> > > > > > > > objects that is just generating more work for the
> > > >> conversion we
> > > >> > > > know
> > > >> > > > > we
> > > >> > > > > > > > have to do.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Fixed in the latest patch - removed scala protocol
> > > >> classes.
> > > >> > > > > > > >
> > > >> > > > > > > > 3. This proposal introduces a new type of optional
> > > >> parameter.
> > > >> > > This
> > > >> > > > is
> > > >> > > > > > > > inconsistent with everything else in the protocol
> where we
> > > >> use -1
> > > >> > > > or
> > > >> > > > > > some
> > > >> > > > > > > > other marker value. You could argue either way but
> let's
> > > >> stick
> > > >> > > with
> > > >> > > > > > that
> > > >> > > > > > > > for consistency. For clients that implemented the
> protocol
> > > >> in a
> > > >> > > > > better
> > > >> > > > > > > way
> > > >> > > > > > > > than our scala code these basic primitives are hard to
> > > >> change.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf type
> and
> > > >> changed
> > > >> > > > > > protocol
> > > >> > > > > > > > accordingly.
> > > >> > > > > > > >
> > > >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> > > >> TopicMetadataRequest
> > > >> > > > > which
> > > >> > > > > > > has
> > > >> > > > > > > > brokers, topics, and partitions. I think we should
> rename
> > > >> that
> > > >> > > > > request
> > > >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest) and
> > > >> include the
> > > >> > > id
> > > >> > > > > of
> > > >> > > > > > > the
> > > >> > > > > > > > controller. Or are there other things we could add
> here?
> > > >> > > > > > > >
> > > >> > > > > > > > A: I agree. Updated the KIP. Let's extends
> TopicMetadata
> > > to
> > > >> > > > version 2
> > > >> > > > > > and
> > > >> > > > > > > > include controller.
> > > >> > > > > > > >
> > > >> > > > > > > > 5. We have a tendency to try to make a lot of requests
> > > that
> > > >> can
> > > >> > > > only
> > > >> > > > > go
> > > >> > > > > > > to
> > > >> > > > > > > > particular nodes. This adds a lot of burden for client
> > > >> > > > > implementations
> > > >> > > > > > > (it
> > > >> > > > > > > > sounds easy but each discovery can fail in many parts
> so
> > > it
> > > >> ends
> > > >> > > up
> > > >> > > > > > > being a
> > > >> > > > > > > > full state machine to do right). I think we should
> > > consider
> > > >> > > making
> > > >> > > > > > admin
> > > >> > > > > > > > commands and ideally as many of the other apis as
> possible
> > > >> > > > available
> > > >> > > > > on
> > > >> > > > > > > all
> > > >> > > > > > > > brokers and just redirect to the controller on the
> broker
> > > >> side.
> > > >> > > > > Perhaps
> > > >> > > > > > > > there would be a general way to encapsulate this
> > > re-routing
> > > >> > > > behavior.
> > > >> > > > > > > >
> > > >> > > > > > > > A: It's a very interesting idea, but seems there are
> some
> > > >> > > concerns
> > > >> > > > > > about
> > > >> > > > > > > > this
> > > >> > > > > > > > feature (like performance considerations, how this
> will
> > > >> > > complicate
> > > >> > > > > > server
> > > >> > > > > > > > etc).
> > > >> > > > > > > > I believe this shouldn't be a blocker. If this
> feature is
> > > >> > > > implemented
> > > >> > > > > > at
> > > >> > > > > > > > some
> > > >> > > > > > > > point it won't affect Admin changes - at least no
> changes
> > > to
> > > >> > > public
> > > >> > > > > API
> > > >> > > > > > > > will be required.
> > > >> > > > > > > >
> > > >> > > > > > > > 6. We should probably normalize the key value pairs
> used
> > > for
> > > >> > > > configs
> > > >> > > > > > > rather
> > > >> > > > > > > > than embedding a new formatting. So two strings rather
> > > than
> > > >> one
> > > >> > > > with
> > > >> > > > > an
> > > >> > > > > > > > internal equals sign.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Fixed in the latest patch - normalized configs and
> > > >> changed
> > > >> > > > > protocol
> > > >> > > > > > > > accordingly.
> > > >> > > > > > > >
> > > >> > > > > > > > 7. Is the postcondition of these APIs that the
> command has
> > > >> begun
> > > >> > > or
> > > >> > > > > > that
> > > >> > > > > > > > the command has been completed? It is a lot more
> usable if
> > > >> the
> > > >> > > > > command
> > > >> > > > > > > has
> > > >> > > > > > > > been completed so you know that if you create a topic
> and
> > > >> then
> > > >> > > > > publish
> > > >> > > > > > to
> > > >> > > > > > > > it you won't get an exception about there being no
> such
> > > >> topic.
> > > >> > > > > > > >
> > > >> > > > > > > > A: For long running requests (like reassign
> partitions) -
> > > >> the
> > > >> > > post
> > > >> > > > > > > > condition is
> > > >> > > > > > > > command has begun - so we don't block the client. In
> case
> > > >> of your
> > > >> > > > > > > example -
> > > >> > > > > > > > topic commands, this will be refactored and topic
> commands
> > > >> will
> > > >> > > be
> > > >> > > > > > > executed
> > > >> > > > > > > > immediately, since the Controller will serve Admin
> > > requests
> > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > >> > > > > > > >
> > > >> > > > > > > > 8. Describe topic and list topics duplicate a lot of
> stuff
> > > >> in the
> > > >> > > > > > > metadata
> > > >> > > > > > > > request. Is there a reason to give back topics marked
> for
> > > >> > > > deletion? I
> > > >> > > > > > > feel
> > > >> > > > > > > > like if we just make the post-condition of the delete
> > > >> command be
> > > >> > > > that
> > > >> > > > > > the
> > > >> > > > > > > > topic is deleted that will get rid of the need for
> this
> > > >> right?
> > > >> > > And
> > > >> > > > it
> > > >> > > > > > > will
> > > >> > > > > > > > be much more intuitive.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Fixed in the latest patch - removed topics marked
> for
> > > >> deletion
> > > >> > > > in
> > > >> > > > > > > > ListTopicsRequest.
> > > >> > > > > > > >
> > > >> > > > > > > > 9. Should we consider batching these requests? We have
> > > >> generally
> > > >> > > > > tried
> > > >> > > > > > to
> > > >> > > > > > > > allow multiple operations to be batched. My suspicion
> is
> > > >> that
> > > >> > > > without
> > > >> > > > > > > this
> > > >> > > > > > > > we will get a lot of code that does something like
> > > >> > > > > > > >    for(topic: adminClient.listTopics())
> > > >> > > > > > > >       adminClient.describeTopic(topic)
> > > >> > > > > > > > this code will work great when you test on 5 topics
> but
> > > not
> > > >> do as
> > > >> > > > > well
> > > >> > > > > > if
> > > >> > > > > > > > you have 50k.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Updated the KIP - please check "Topic Admin Schema"
> > > >> section.
> > > >> > > > > > > >
> > > >> > > > > > > > 10. I think we should also discuss how we want to
> expose a
> > > >> > > > > programmatic
> > > >> > > > > > > JVM
> > > >> > > > > > > > client api for these operations. Currently people
> rely on
> > > >> > > > AdminUtils
> > > >> > > > > > > which
> > > >> > > > > > > > is totally sketchy. I think we probably need another
> > > client
> > > >> under
> > > >> > > > > > > clients/
> > > >> > > > > > > > that exposes administrative functionality. We will
> need
> > > >> this just
> > > >> > > > to
> > > >> > > > > > > > properly test the new apis, I suspect. We should
> figure
> > > out
> > > >> that
> > > >> > > > API.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Updated the KIP - please check "Admin Client"
> section
> > > >> with an
> > > >> > > > > > initial
> > > >> > > > > > > > API proposal.
> > > >> > > > > > > >
> > > >> > > > > > > > 11. The other information that would be really useful
> to
> > > get
> > > >> > > would
> > > >> > > > be
> > > >> > > > > > > > information about partitions--how much data is in the
> > > >> partition,
> > > >> > > > what
> > > >> > > > > > are
> > > >> > > > > > > > the segment offsets, what is the log-end offset (i.e.
> last
> > > >> > > offset),
> > > >> > > > > > what
> > > >> > > > > > > is
> > > >> > > > > > > > the compaction point, etc. I think that done right
> this
> > > >> would be
> > > >> > > > the
> > > >> > > > > > > > successor to the very awkward OffsetRequest we have
> today.
> > > >> > > > > > > >
> > > >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest
> > > >> patch. I
> > > >> > > > > believe
> > > >> > > > > > > > this should
> > > >> > > > > > > > be resolved in a separate KIP / jira ticket.
> > > >> > > > > > > >
> > > >> > > > > > > > 12. Generally we can do good error handling without
> > > needing
> > > >> > > custom
> > > >> > > > > > > > server-side
> > > >> > > > > > > > messages. I.e. generally the client has the context to
> > > know
> > > >> that
> > > >> > > if
> > > >> > > > > it
> > > >> > > > > > > got
> > > >> > > > > > > > an error that the topic doesn't exist to say "Topic X
> > > >> doesn't
> > > >> > > > exist"
> > > >> > > > > > > rather
> > > >> > > > > > > > than "error code 14" (or whatever). Maybe there are
> > > specific
> > > >> > > cases
> > > >> > > > > > where
> > > >> > > > > > > > this is hard? If we want to add server-side error
> messages
> > > >> we
> > > >> > > > really
> > > >> > > > > do
> > > >> > > > > > > > need to do this in a consistent way across the
> protocol.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Updated the KIP - please check "Protocol Errors"
> > > >> section. I
> > > >> > > > added
> > > >> > > > > > the
> > > >> > > > > > > > comprehensive, fine-grained list of error codes.
> > > >> > > > > > > >
> > > >> > > > > > > > Comments from Guozhang:
> > > >> > > > > > > > 13. Describe topic request: it would be great to go
> beyond
> > > >> just
> > > >> > > > > > batching
> > > >> > > > > > > on
> > > >> > > > > > > > topic name regex for this request. For example, a very
> > > >> common use
> > > >> > > > > case
> > > >> > > > > > of
> > > >> > > > > > > > the topic command is to list all topics whose config
> A's
> > > >> value is
> > > >> > > > B.
> > > >> > > > > > With
> > > >> > > > > > > > topic name regex then we have to first retrieve
> __all__
> > > >> topics's
> > > >> > > > > > > > description info and then filter at the client end,
> which
> > > >> will
> > > >> > > be a
> > > >> > > > > > huge
> > > >> > > > > > > > burden on ZK.
> > > >> > > > > > > > AND
> > > >> > > > > > > > 14. Config K-Vs in create topic: this is related to
> the
> > > >> previous
> > > >> > > > > point;
> > > >> > > > > > > > maybe we can add another metadata K-V or just a
> metadata
> > > >> string
> > > >> > > > along
> > > >> > > > > > > side
> > > >> > > > > > > > with config K-V in create topic like we did for offset
> > > >> commit
> > > >> > > > > request.
> > > >> > > > > > > This
> > > >> > > > > > > > field can be quite useful in storing information like
> > > >> "owner" of
> > > >> > > > the
> > > >> > > > > > > topic
> > > >> > > > > > > > who issue the create command, etc, which is quite
> > > important
> > > >> for a
> > > >> > > > > > > > multi-tenant setting. Then in the describe topic
> request
> > > we
> > > >> can
> > > >> > > > also
> > > >> > > > > > > batch
> > > >> > > > > > > > on regex of the metadata field.
> > > >> > > > > > > >
> > > >> > > > > > > > A: As discussed it is very interesting but can be
> > > >> implemented
> > > >> > > later
> > > >> > > > > > after
> > > >> > > > > > > > we have some basic functionality there.
> > > >> > > > > > > >
> > > >> > > > > > > > 15. Today all the admin operations are async in the
> sense
> > > >> that
> > > >> > > > > command
> > > >> > > > > > > will
> > > >> > > > > > > > return once it is written in ZK, and that is why we
> need
> > > >> extra
> > > >> > > > > > > verification
> > > >> > > > > > > > like testUtil.waitForTopicCreated() / verify partition
> > > >> > > reassignment
> > > >> > > > > > > > request, etc. With admin requests we could add a flag
> to
> > > >> enable /
> > > >> > > > > > disable
> > > >> > > > > > > > synchronous requests; when it is turned on, the
> response
> > > >> will not
> > > >> > > > > > return
> > > >> > > > > > > > until the request has been completed. And for async
> > > >> requests we
> > > >> > > can
> > > >> > > > > > add a
> > > >> > > > > > > > "token" field in the response, and then only need a
> > > general
> > > >> > > "admin
> > > >> > > > > > > > verification request" with the given token to check
> if the
> > > >> async
> > > >> > > > > > request
> > > >> > > > > > > > has been completed.
> > > >> > > > > > > >
> > > >> > > > > > > > A: I see your point. My idea was to provide specific
> > > >> > > > Verify...Request
> > > >> > > > > > per
> > > >> > > > > > > > each
> > > >> > > > > > > > long running request, where needed. We can do it the
> way
> > > you
> > > >> > > > suggest.
> > > >> > > > > > The
> > > >> > > > > > > > only
> > > >> > > > > > > > concern is that introducing a token we again will make
> > > >> schema
> > > >> > > > > > "dynamic".
> > > >> > > > > > > We
> > > >> > > > > > > > wanted
> > > >> > > > > > > > to do similar thing introducing single AdminRequest
> for
> > > all
> > > >> topic
> > > >> > > > > > > commands
> > > >> > > > > > > > but rejected
> > > >> > > > > > > > this idea because we wanted to have schema defined. So
> > > this
> > > >> is
> > > >> > > > more a
> > > >> > > > > > > > choice between:
> > > >> > > > > > > > a) have fixed schema but introduce each time new
> > > >> Verify...Request
> > > >> > > > for
> > > >> > > > > > > > long-running requests
> > > >> > > > > > > > b) use one request for verification but generalize it
> with
> > > >> token
> > > >> > > > > > > > I'm fine with whatever decision community come to.
> Just
> > > let
> > > >> me
> > > >> > > know
> > > >> > > > > > your
> > > >> > > > > > > > thoughts.
> > > >> > > > > > > >
> > > >> > > > > > > > Comment from Gwen:
> > > >> > > > > > > > 16. Specifically for ownership, I think the plan is
> to add
> > > >> ACL
> > > >> > > (it
> > > >> > > > > > sounds
> > > >> > > > > > > > like you are describing ACL) via an external system
> > > (Argus,
> > > >> > > > Sentry).
> > > >> > > > > > > > I remember KIP-11 described this, but I can't find
> the KIP
> > > >> any
> > > >> > > > > longer.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Okay, no problem. Not sure though how we are going
> to
> > > >> handle
> > > >> > > it.
> > > >> > > > > > Wait
> > > >> > > > > > > > which KIP
> > > >> > > > > > > > will be committed first and include changes to
> > > >> TopicMetadata from
> > > >> > > > the
> > > >> > > > > > > later
> > > >> > > > > > > > one?
> > > >> > > > > > > > Anyway, I added this note to "Open Questions" section
> so
> > > we
> > > >> don't
> > > >> > > > > miss
> > > >> > > > > > > this
> > > >> > > > > > > > piece.
> > > >> > > > > > > >
> > > >> > > > > > > > Thanks,
> > > >> > > > > > > > Andrii Biletskyi
> > > >> > > > > > > >
> > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > > > > > > >
> > > >> > > > > > > > > Hi all,
> > > >> > > > > > > > >
> > > >> > > > > > > > > Today I uploaded the patch that covers some of the
> > > >> discussed
> > > >> > > and
> > > >> > > > > > agreed
> > > >> > > > > > > > > items:
> > > >> > > > > > > > > - removed MaybeOf optional type
> > > >> > > > > > > > > - switched to java protocol definitions
> > > >> > > > > > > > > - simplified messages (normalized configs, removed
> topic
> > > >> marked
> > > >> > > > for
> > > >> > > > > > > > > deletion)
> > > >> > > > > > > > >
> > > >> > > > > > > > > I also updated the KIP-4 with respective changes and
> > > >> wrote down
> > > >> > > > my
> > > >> > > > > > > > > proposal for
> > > >> > > > > > > > > pending items:
> > > >> > > > > > > > > - Batch Admin Operations -> updated Wire Protocol
> schema
> > > >> > > proposal
> > > >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
> > > >> > > > TopicMetadataRequest
> > > >> > > > > > > > > - Admin Client -> updated my initial proposal to
> reflect
> > > >> > > batching
> > > >> > > > > > > > > - Error codes -> proposed fine-grained error code
> > > instead
> > > >> of
> > > >> > > > > > > > > AdminRequestFailed
> > > >> > > > > > > > >
> > > >> > > > > > > > > I will also send a separate email to cover all
> comments
> > > >> from
> > > >> > > this
> > > >> > > > > > > thread.
> > > >> > > > > > > > >
> > > >> > > > > > > > > Thanks,
> > > >> > > > > > > > > Andrii Biletskyi
> > > >> > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> > > >> > > > > gshapira@cloudera.com
> > > >> > > > > > >
> > > >> > > > > > > > > wrote:
> > > >> > > > > > > > >
> > > >> > > > > > > > >> Found KIP-11 (
> > > >> > > > > > > > >>
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > >> > > > > > > > >> )
> > > >> > > > > > > > >> It actually specifies changes to the Metadata
> protocol,
> > > >> so
> > > >> > > > making
> > > >> > > > > > sure
> > > >> > > > > > > > >> both KIPs are consistent in this regard will be
> good.
> > > >> > > > > > > > >>
> > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> > > >> > > > > > gshapira@cloudera.com
> > > >> > > > > > > >
> > > >> > > > > > > > >> wrote:
> > > >> > > > > > > > >> > Specifically for ownership, I think the plan is
> to
> > > add
> > > >> ACL
> > > >> > > (it
> > > >> > > > > > > sounds
> > > >> > > > > > > > >> > like you are describing ACL) via an external
> system
> > > >> (Argus,
> > > >> > > > > > Sentry).
> > > >> > > > > > > > >> > I remember KIP-11 described this, but I can't
> find
> > > the
> > > >> KIP
> > > >> > > any
> > > >> > > > > > > longer.
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting
> > > >> information
> > > >> > > that
> > > >> > > > > > > already
> > > >> > > > > > > > >> > exists from Kafka brokers, not on adding
> information
> > > >> that
> > > >> > > > > perhaps
> > > >> > > > > > > > >> > should exist but doesn't yet?
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> > Gwen
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> > > >> > > > > > wangguoz@gmail.com>
> > > >> > > > > > > > >> wrote:
> > > >> > > > > > > > >> >> Folks,
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Just want to elaborate a bit more on the
> > > create-topic
> > > >> > > > metadata
> > > >> > > > > > and
> > > >> > > > > > > > >> batching
> > > >> > > > > > > > >> >> describe-topic based on config / metadata in my
> > > >> previous
> > > >> > > > email
> > > >> > > > > as
> > > >> > > > > > > we
> > > >> > > > > > > > >> work
> > > >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to have
> some
> > > >> sort of
> > > >> > > > > topic
> > > >> > > > > > > > >> management
> > > >> > > > > > > > >> >> mechanisms, which I think is quite important in
> a
> > > >> > > > multi-tenant
> > > >> > > > > /
> > > >> > > > > > > > cloud
> > > >> > > > > > > > >> >> architecture: today anyone can create topics in
> a
> > > >> shared
> > > >> > > > Kafka
> > > >> > > > > > > > >> cluster, but
> > > >> > > > > > > > >> >> there is no concept or "ownership" of topics
> that
> > > are
> > > >> > > created
> > > >> > > > > by
> > > >> > > > > > > > >> different
> > > >> > > > > > > > >> >> users. For example, at LinkedIn we basically
> > > >> distinguish
> > > >> > > > topic
> > > >> > > > > > > owners
> > > >> > > > > > > > >> via
> > > >> > > > > > > > >> >> some casual topic name prefix, which is a bit
> > > awkward
> > > >> and
> > > >> > > > does
> > > >> > > > > > not
> > > >> > > > > > > > fly
> > > >> > > > > > > > >> as
> > > >> > > > > > > > >> >> we scale our customers. It would be great to use
> > > >> > > > > describe-topics
> > > >> > > > > > > such
> > > >> > > > > > > > >> as:
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Describe all topics that is created by me.
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Describe all topics whose retention time is
> > > overriden
> > > >> to X.
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Describe all topics whose writable group include
> > > user
> > > >> Y
> > > >> > > (this
> > > >> > > > > is
> > > >> > > > > > > > >> related to
> > > >> > > > > > > > >> >> authorization), etc..
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> One possible way to achieve this is to add a
> > > metadata
> > > >> file
> > > >> > > in
> > > >> > > > > the
> > > >> > > > > > > > >> >> create-topic request, whose value will also be
> > > >> written ZK
> > > >> > > as
> > > >> > > > we
> > > >> > > > > > > > create
> > > >> > > > > > > > >> the
> > > >> > > > > > > > >> >> topic; then describe-topics can choose to batch
> > > topics
> > > >> > > based
> > > >> > > > on
> > > >> > > > > > 1)
> > > >> > > > > > > > name
> > > >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata
> regex,
> > > etc.
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Thoughts?
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Guozhang
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> > > >> > > > > > wangguoz@gmail.com>
> > > >> > > > > > > > >> wrote:
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >>> Thanks for the updated wiki. A few comments
> below:
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> 1. Error description in response: I think if
> some
> > > >> > > errorCode
> > > >> > > > > > could
> > > >> > > > > > > > >> indicate
> > > >> > > > > > > > >> >>> several different error cases then we should
> really
> > > >> change
> > > >> > > > it
> > > >> > > > > to
> > > >> > > > > > > > >> multiple
> > > >> > > > > > > > >> >>> codes. In general the errorCode itself would be
> > > >> precise
> > > >> > > and
> > > >> > > > > > > > >> sufficient for
> > > >> > > > > > > > >> >>> describing the server side errors.
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> 2. Describe topic request: it would be great
> to go
> > > >> beyond
> > > >> > > > just
> > > >> > > > > > > > >> batching on
> > > >> > > > > > > > >> >>> topic name regex for this request. For
> example, a
> > > >> very
> > > >> > > > common
> > > >> > > > > > use
> > > >> > > > > > > > >> case of
> > > >> > > > > > > > >> >>> the topic command is to list all topics whose
> > > config
> > > >> A's
> > > >> > > > value
> > > >> > > > > > is
> > > >> > > > > > > B.
> > > >> > > > > > > > >> With
> > > >> > > > > > > > >> >>> topic name regex then we have to first retrieve
> > > >> __all__
> > > >> > > > > topics's
> > > >> > > > > > > > >> >>> description info and then filter at the client
> end,
> > > >> which
> > > >> > > > will
> > > >> > > > > > be
> > > >> > > > > > > a
> > > >> > > > > > > > >> huge
> > > >> > > > > > > > >> >>> burden on ZK.
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is
> related to
> > > >> the
> > > >> > > > > previous
> > > >> > > > > > > > point;
> > > >> > > > > > > > >> >>> maybe we can add another metadata K-V or just a
> > > >> metadata
> > > >> > > > > string
> > > >> > > > > > > > along
> > > >> > > > > > > > >> side
> > > >> > > > > > > > >> >>> with config K-V in create topic like we did for
> > > >> offset
> > > >> > > > commit
> > > >> > > > > > > > >> request. This
> > > >> > > > > > > > >> >>> field can be quite useful in storing
> information
> > > like
> > > >> > > > "owner"
> > > >> > > > > of
> > > >> > > > > > > the
> > > >> > > > > > > > >> topic
> > > >> > > > > > > > >> >>> who issue the create command, etc, which is
> quite
> > > >> > > important
> > > >> > > > > for
> > > >> > > > > > a
> > > >> > > > > > > > >> >>> multi-tenant setting. Then in the describe
> topic
> > > >> request
> > > >> > > we
> > > >> > > > > can
> > > >> > > > > > > also
> > > >> > > > > > > > >> batch
> > > >> > > > > > > > >> >>> on regex of the metadata field.
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> 4. Today all the admin operations are async in
> the
> > > >> sense
> > > >> > > > that
> > > >> > > > > > > > command
> > > >> > > > > > > > >> will
> > > >> > > > > > > > >> >>> return once it is written in ZK, and that is
> why we
> > > >> need
> > > >> > > > extra
> > > >> > > > > > > > >> verification
> > > >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify
> > > >> partition
> > > >> > > > > > > reassignment
> > > >> > > > > > > > >> >>> request, etc. With admin requests we could add
> a
> > > >> flag to
> > > >> > > > > enable
> > > >> > > > > > /
> > > >> > > > > > > > >> disable
> > > >> > > > > > > > >> >>> synchronous requests; when it is turned on, the
> > > >> response
> > > >> > > > will
> > > >> > > > > > not
> > > >> > > > > > > > >> return
> > > >> > > > > > > > >> >>> until the request has been completed. And for
> async
> > > >> > > requests
> > > >> > > > > we
> > > >> > > > > > > can
> > > >> > > > > > > > >> add a
> > > >> > > > > > > > >> >>> "token" field in the response, and then only
> need a
> > > >> > > general
> > > >> > > > > > "admin
> > > >> > > > > > > > >> >>> verification request" with the given token to
> check
> > > >> if the
> > > >> > > > > async
> > > >> > > > > > > > >> request
> > > >> > > > > > > > >> >>> has been completed.
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> 5. +1 for extending Metadata request to include
> > > >> > > controller /
> > > >> > > > > > > > >> coordinator
> > > >> > > > > > > > >> >>> information, and then we can remove the
> > > >> ConsumerMetadata /
> > > >> > > > > > > > >> ClusterMetadata
> > > >> > > > > > > > >> >>> requests.
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> Guozhang
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> > > >> > > > > > jjkoshy.w@gmail.com>
> > > >> > > > > > > > >> wrote:
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't
> think I
> > > >> will be
> > > >> > > > > able
> > > >> > > > > > to
> > > >> > > > > > > > >> make
> > > >> > > > > > > > >> >>>> it today, so if notes can be sent out
> afterward
> > > that
> > > >> > > would
> > > >> > > > be
> > > >> > > > > > > > great.
> > > >> > > > > > > > >> >>>>
> > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen
> > > >> Shapira
> > > >> > > > wrote:
> > > >> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking
> forward
> > > >> to
> > > >> > > > > chatting
> > > >> > > > > > > with
> > > >> > > > > > > > >> >>>> everyone :)
> > > >> > > > > > > > >> >>>> >
> > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> > > >> > > > > > > joe.stein@stealth.ly>
> > > >> > > > > > > > >> wrote:
> > > >> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout
> invite
> > > to
> > > >> all
> > > >> > > > pmc,
> > > >> > > > > > > > >> committers
> > > >> > > > > > > > >> >>>> and
> > > >> > > > > > > > >> >>>> > > everyone I found working on a KIP. If I
> missed
> > > >> anyone
> > > >> > > > in
> > > >> > > > > > the
> > > >> > > > > > > > >> invite
> > > >> > > > > > > > >> >>>> please
> > > >> > > > > > > > >> >>>> > > let me know and can update it, np.
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm
> Eastern
> > > >> Time.
> > > >> > > > Maybe
> > > >> > > > > > we
> > > >> > > > > > > > can
> > > >> > > > > > > > >> get
> > > >> > > > > > > > >> >>>> INFRA
> > > >> > > > > > > > >> >>>> > > help to make a google account so we can
> manage
> > > >> > > better?
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>> > > To discuss
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>>
> > > >> > > > > > > > >>
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > >> > > > > > > > >> >>>> > > in progress and related JIRA that are
> > > >> interdependent
> > > >> > > > and
> > > >> > > > > > > common
> > > >> > > > > > > > >> work.
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay
> Kreps <
> > > >> > > > > > > > jay.kreps@gmail.com>
> > > >> > > > > > > > >> >>>> wrote:
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that will
> also
> > > >> record
> > > >> > > > and
> > > >> > > > > > make
> > > >> > > > > > > > the
> > > >> > > > > > > > >> >>>> sessions
> > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > >> > > > > > > > >> >>>> > >>
> > > >> > > > > > > > >> >>>> > >> -Jay
> > > >> > > > > > > > >> >>>> > >>
> > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff
> > > Holoman
> > > >> <
> > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > >> > > > > > > > >> >>>> > >> wrote:
> > > >> > > > > > > > >> >>>> > >>
> > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for
> this
> > > >> purpose.
> > > >> > > We
> > > >> > > > > > could
> > > >> > > > > > > > >> record
> > > >> > > > > > > > >> >>>> the
> > > >> > > > > > > > >> >>>> > >> > sessions if there is interest and
> publish
> > > >> them
> > > >> > > out.
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >> > Thanks
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >> > Jeff
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay
> > > Kreps <
> > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > >> > > > > > > > >> >>>> wrote:
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >> > > Let's try to get the technical
> hang-ups
> > > >> sorted
> > > >> > > > out,
> > > >> > > > > > > > though.
> > > >> > > > > > > > >> I
> > > >> > > > > > > > >> >>>> really
> > > >> > > > > > > > >> >>>> > >> > think
> > > >> > > > > > > > >> >>>> > >> > > there is some benefit to live
> discussion
> > > vs
> > > >> > > > > writing. I
> > > >> > > > > > > am
> > > >> > > > > > > > >> >>>> hopeful that
> > > >> > > > > > > > >> >>>> > >> if
> > > >> > > > > > > > >> >>>> > >> > > we post instructions and give
> ourselves a
> > > >> few
> > > >> > > > > attempts
> > > >> > > > > > > we
> > > >> > > > > > > > >> can
> > > >> > > > > > > > >> >>>> get it
> > > >> > > > > > > > >> >>>> > >> > > working.
> > > >> > > > > > > > >> >>>> > >> > >
> > > >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work for
> > > >> me...any
> > > >> > > > > > objections?
> > > >> > > > > > > > >> >>>> > >> > >
> > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > >> > > > > > > > >> >>>> > >> > >
> > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe
> > > Stein
> > > >> <
> > > >> > > > > > > > >> joe.stein@stealth.ly
> > > >> > > > > > > > >> >>>> >
> > > >> > > > > > > > >> >>>> > >> wrote:
> > > >> > > > > > > > >> >>>> > >> > >
> > > >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe like
> every
> > > >> > > Tuesday ~
> > > >> > > > > 1pm
> > > >> > > > > > > ET
> > > >> > > > > > > > /
> > > >> > > > > > > > >> 10am
> > > >> > > > > > > > >> >>>> PT
> > > >> > > > > > > > >> >>>> > >> ????
> > > >> > > > > > > > >> >>>> > >> > > >
> > > >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but
> there
> > > is
> > > >> > > always
> > > >> > > > > some
> > > >> > > > > > > > >> issue or
> > > >> > > > > > > > >> >>>> > >> whatever
> > > >> > > > > > > > >> >>>> > >> > so
> > > >> > > > > > > > >> >>>> > >> > > > we know the apache irc channel
> works.
> > > We
> > > >> can
> > > >> > > > start
> > > >> > > > > > > there
> > > >> > > > > > > > >> and
> > > >> > > > > > > > >> >>>> see how
> > > >> > > > > > > > >> >>>> > >> it
> > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too
> and
> > > >> > > associate
> > > >> > > > to
> > > >> > > > > > > > >> tickets if
> > > >> > > > > > > > >> >>>> need be
> > > >> > > > > > > > >> >>>> > >> > > makes
> > > >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> > > >> > > > > > > > >> >>>> > >> > > >
> > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > >> > > > > > > > >> >>>> > >> > > >
> > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM,
> Jay
> > > >> Kreps <
> > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > >> > > > > > > > >> >>>> > >> > wrote:
> > > >> > > > > > > > >> >>>> > >> > > >
> > > >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a Google
> > > >> Hangout to
> > > >> > > > chat
> > > >> > > > > > > about
> > > >> > > > > > > > >> this.
> > > >> > > > > > > > >> >>>> What
> > > >> > > > > > > > >> >>>> > >> > about
> > > >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
> > > further...I
> > > >> > > > actually
> > > >> > > > > > > think
> > > >> > > > > > > > it
> > > >> > > > > > > > >> >>>> would be
> > > >> > > > > > > > >> >>>> > >> > good
> > > >> > > > > > > > >> >>>> > >> > > > for
> > > >> > > > > > > > >> >>>> > >> > > > > everyone spending a reasonable
> chunk
> > > of
> > > >> > > their
> > > >> > > > > week
> > > >> > > > > > > on
> > > >> > > > > > > > >> Kafka
> > > >> > > > > > > > >> >>>> stuff
> > > >> > > > > > > > >> >>>> > >> to
> > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think we
> could
> > > >> use
> > > >> > > time
> > > >> > > > > to
> > > >> > > > > > > talk
> > > >> > > > > > > > >> >>>> through
> > > >> > > > > > > > >> >>>> > >> design
> > > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of
> > > code
> > > >> > > > reviews,
> > > >> > > > > > talk
> > > >> > > > > > > > >> through
> > > >> > > > > > > > >> >>>> any
> > > >> > > > > > > > >> >>>> > >> > tricky
> > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > We can make it publicly
> available so
> > > >> that
> > > >> > > any
> > > >> > > > > one
> > > >> > > > > > > can
> > > >> > > > > > > > >> follow
> > > >> > > > > > > > >> >>>> along
> > > >> > > > > > > > >> >>>> > >> > who
> > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this? If so
> > > I'll
> > > >> try
> > > >> > > to
> > > >> > > > > set
> > > >> > > > > > it
> > > >> > > > > > > > up
> > > >> > > > > > > > >> >>>> starting
> > > >> > > > > > > > >> >>>> > >> next
> > > >> > > > > > > > >> >>>> > >> > > > week.
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM,
> > > Andrii
> > > >> > > > > Biletskyi
> > > >> > > > > > <
> > > >> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly>
> wrote:
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed /
> > > >> aligned
> > > >> > > > > document
> > > >> > > > > > > > >> structure.
> > > >> > > > > > > > >> >>>> Also I
> > > >> > > > > > > > >> >>>> > >> > > added
> > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
> > > >> AdminClient so
> > > >> > > we
> > > >> > > > > have
> > > >> > > > > > > > >> something
> > > >> > > > > > > > >> >>>> to
> > > >> > > > > > > > >> >>>> > >> start
> > > >> > > > > > > > >> >>>> > >> > > > from
> > > >> > > > > > > > >> >>>> > >> > > > > > while
> > > >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > >
> > > >> > > > > > > > >> >>>> > >> > >
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >>
> > > >> > > > > > > > >> >>>>
> > > >> > > > > > > > >>
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > > >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01
> PM,
> > > >> Andrii
> > > >> > > > > > Biletskyi
> > > >> > > > > > > <
> > > >> > > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you are
> right,
> > > >> in
> > > >> > > most
> > > >> > > > > > cases
> > > >> > > > > > > > >> client
> > > >> > > > > > > > >> >>>> will
> > > >> > > > > > > > >> >>>> > >> have
> > > >> > > > > > > > >> >>>> > >> > > > enough
> > > >> > > > > > > > >> >>>> > >> > > > > > > context to show descriptive
> error
> > > >> > > message.
> > > >> > > > > My
> > > >> > > > > > > > >> concern is
> > > >> > > > > > > > >> >>>> that
> > > >> > > > > > > > >> >>>> > >> we
> > > >> > > > > > > > >> >>>> > >> > > will
> > > >> > > > > > > > >> >>>> > >> > > > > > have
> > > >> > > > > > > > >> >>>> > >> > > > > > > to
> > > >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error codes
> for
> > > >> each
> > > >> > > > > possible
> > > >> > > > > > > > >> error. Of
> > > >> > > > > > > > >> >>>> course,
> > > >> > > > > > > > >> >>>> > >> > we
> > > >> > > > > > > > >> >>>> > >> > > > > could
> > > >> > > > > > > > >> >>>> > >> > > > > > > reuse
> > > >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
> > > >> > > > > > > UknownTopicOrPartitionCode,
> > > >> > > > > > > > >> but we
> > > >> > > > > > > > >> >>>> will
> > > >> > > > > > > > >> >>>> > >> > also
> > > >> > > > > > > > >> >>>> > >> > > > need
> > > >> > > > > > > > >> >>>> > >> > > > > > to
> > > >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> > > >> TopicAlreadyExistsCode,
> > > >> > > > > > > > >> >>>> TopicConfigInvalid (both
> > > >> > > > > > > > >> >>>> > >> > for
> > > >> > > > > > > > >> >>>> > >> > > > > topic
> > > >> > > > > > > > >> >>>> > >> > > > > > > name and config, and probably
> > > user
> > > >> would
> > > >> > > > > like
> > > >> > > > > > to
> > > >> > > > > > > > >> know
> > > >> > > > > > > > >> >>>> what
> > > >> > > > > > > > >> >>>> > >> > exactly
> > > >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
> > > >> > > > > > > InvalidReplicaAssignment,
> > > >> > > > > > > > >> >>>> > >> InternalError
> > > >> > > > > > > > >> >>>> > >> > > > (e.g.
> > > >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
> > > TopicCommand,
> > > >> we
> > > >> > > will
> > > >> > > > > > also
> > > >> > > > > > > > >> need to
> > > >> > > > > > > > >> >>>> add
> > > >> > > > > > > > >> >>>> > >> > similar
> > > >> > > > > > > > >> >>>> > >> > > > > stuff
> > > >> > > > > > > > >> >>>> > >> > > > > > > for
> > > >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> > > >> PreferredReplica. So
> > > >> > > > > we'll
> > > >> > > > > > > end
> > > >> > > > > > > > >> up
> > > >> > > > > > > > >> >>>> with a
> > > >> > > > > > > > >> >>>> > >> > large
> > > >> > > > > > > > >> >>>> > >> > > > list
> > > >> > > > > > > > >> >>>> > >> > > > > > of
> > > >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in
> Admin
> > > >> > > protocol.
> > > >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree my
> > > >> proposal is
> > > >> > > > not
> > > >> > > > > > > > >> consistent
> > > >> > > > > > > > >> >>>> with
> > > >> > > > > > > > >> >>>> > >> > other
> > > >> > > > > > > > >> >>>> > >> > > > > cases.
> > > >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better
> solution
> > > >> or
> > > >> > > > > something
> > > >> > > > > > > > >> >>>> in-between.
> > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it
> is a
> > > >> great
> > > >> > > > idea.
> > > >> > > > > > > This
> > > >> > > > > > > > >> way we
> > > >> > > > > > > > >> >>>> can
> > > >> > > > > > > > >> >>>> > >> move
> > > >> > > > > > > > >> >>>> > >> > > on
> > > >> > > > > > > > >> >>>> > >> > > > > > > faster.
> > > >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on
> date/time
> > > so
> > > >> > > people
> > > >> > > > > can
> > > >> > > > > > > > join.
> > > >> > > > > > > > >> >>>> Will work
> > > >> > > > > > > > >> >>>> > >> > for
> > > >> > > > > > > > >> >>>> > >> > > me
> > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > >> > > > > > > > >> >>>> > >> > > > > > > and
> > > >> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime if
> > > agreed
> > > >> in
> > > >> > > > > advance.
> > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09
> PM,
> > > >> Jay
> > > >> > > > Kreps <
> > > >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > >> > > > > > > > >> >>>> > >> > > > > wrote:
> > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good
> error
> > > >> handling
> > > >> > > > > > without
> > > >> > > > > > > > >> needing
> > > >> > > > > > > > >> >>>> custom
> > > >> > > > > > > > >> >>>> > >> > > > > > server-side
> > > >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the
> > > >> client has
> > > >> > > > the
> > > >> > > > > > > > >> context to
> > > >> > > > > > > > >> >>>> know
> > > >> > > > > > > > >> >>>> > >> that
> > > >> > > > > > > > >> >>>> > >> > > if
> > > >> > > > > > > > >> >>>> > >> > > > it
> > > >> > > > > > > > >> >>>> > >> > > > > > got
> > > >> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic
> doesn't
> > > >> exist
> > > >> > > to
> > > >> > > > > say
> > > >> > > > > > > > >> "Topic X
> > > >> > > > > > > > >> >>>> doesn't
> > > >> > > > > > > > >> >>>> > >> > > exist"
> > > >> > > > > > > > >> >>>> > >> > > > > > >> rather
> > > >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or
> > > >> whatever).
> > > >> > > Maybe
> > > >> > > > > > there
> > > >> > > > > > > > are
> > > >> > > > > > > > >> >>>> specific
> > > >> > > > > > > > >> >>>> > >> > cases
> > > >> > > > > > > > >> >>>> > >> > > > > where
> > > >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to
> add
> > > >> > > > server-side
> > > >> > > > > > > error
> > > >> > > > > > > > >> >>>> messages we
> > > >> > > > > > > > >> >>>> > >> > > really
> > > >> > > > > > > > >> >>>> > >> > > > > do
> > > >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a
> consistent
> > > >> way
> > > >> > > > across
> > > >> > > > > > the
> > > >> > > > > > > > >> protocol.
> > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open
> > > >> questions
> > > >> > > > here
> > > >> > > > > > > from
> > > >> > > > > > > > my
> > > >> > > > > > > > >> >>>> previous
> > > >> > > > > > > > >> >>>> > >> > > list. I
> > > >> > > > > > > > >> >>>> > >> > > > > > will
> > > >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few
> days for
> > > >> Strata
> > > >> > > > > > though.
> > > >> > > > > > > > >> Maybe
> > > >> > > > > > > > >> >>>> we could
> > > >> > > > > > > > >> >>>> > >> > do
> > > >> > > > > > > > >> >>>> > >> > > a
> > > >> > > > > > > > >> >>>> > >> > > > > > Google
> > > >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open
> issues
> > > >> some
> > > >> > > time
> > > >> > > > > > > towards
> > > >> > > > > > > > >> the
> > > >> > > > > > > > >> >>>> end of
> > > >> > > > > > > > >> >>>> > >> > next
> > > >> > > > > > > > >> >>>> > >> > > > week
> > > >> > > > > > > > >> >>>> > >> > > > > > for
> > > >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in this
> > > ticket?
> > > >> I
> > > >> > > have
> > > >> > > > a
> > > >> > > > > > > > feeling
> > > >> > > > > > > > >> that
> > > >> > > > > > > > >> >>>> might
> > > >> > > > > > > > >> >>>> > >> > > > progress
> > > >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster than
> > > >> email--I
> > > >> > > > think
> > > >> > > > > we
> > > >> > > > > > > > >> could talk
> > > >> > > > > > > > >> >>>> > >> through
> > > >> > > > > > > > >> >>>> > >> > > > those
> > > >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly
> > > >> quickly...
> > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at
> 7:27 AM,
> > > >> Andrii
> > > >> > > > > > > > Biletskyi <
> > > >> > > > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly
> >
> > > >> wrote:
> > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address
> some of
> > > >> the
> > > >> > > > issues
> > > >> > > > > > > which
> > > >> > > > > > > > >> were
> > > >> > > > > > > > >> >>>> > >> mentioned
> > > >> > > > > > > > >> >>>> > >> > > > > earlier
> > > >> > > > > > > > >> >>>> > >> > > > > > >> about
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of
> > > >> those was
> > > >> > > > > about
> > > >> > > > > > > > >> batching
> > > >> > > > > > > > >> >>>> > >> > operations.
> > > >> > > > > > > > >> >>>> > >> > > > What
> > > >> > > > > > > > >> >>>> > >> > > > > > if
> > > >> > > > > > > > >> >>>> > >> > > > > > >> we
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand
> approach
> > > >> and let
> > > >> > > > > people
> > > >> > > > > > > > >> specify
> > > >> > > > > > > > >> >>>> > >> topic-name
> > > >> > > > > > > > >> >>>> > >> > > by
> > > >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most of
> the
> > > use
> > > >> > > cases?
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what
> information
> > > >> should
> > > >> > > we
> > > >> > > > > > > > generally
> > > >> > > > > > > > >> >>>> provide in
> > > >> > > > > > > > >> >>>> > >> > > Admin
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin
> commands
> > > >> don't
> > > >> > > > imply
> > > >> > > > > > > they
> > > >> > > > > > > > >> will
> > > >> > > > > > > > >> >>>> be used
> > > >> > > > > > > > >> >>>> > >> > only
> > > >> > > > > > > > >> >>>> > >> > > > in
> > > >> > > > > > > > >> >>>> > >> > > > > > CLI
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > but,
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a
> very
> > > >> > > important
> > > >> > > > > > > client
> > > >> > > > > > > > >> of this
> > > >> > > > > > > > >> >>>> > >> > feature.
> > > >> > > > > > > > >> >>>> > >> > > In
> > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > case,
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would
> like
> > > to
> > > >> > > provide
> > > >> > > > > > users
> > > >> > > > > > > > >> with
> > > >> > > > > > > > >> >>>> rich
> > > >> > > > > > > > >> >>>> > >> > > experience
> > > >> > > > > > > > >> >>>> > >> > > > > in
> > > >> > > > > > > > >> >>>> > >> > > > > > >> terms
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > of
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > getting results / errors
> of
> > > the
> > > >> > > > executed
> > > >> > > > > > > > >> commands.
> > > >> > > > > > > > >> >>>> Usually
> > > >> > > > > > > > >> >>>> > >> we
> > > >> > > > > > > > >> >>>> > >> > > > supply
> > > >> > > > > > > > >> >>>> > >> > > > > > >> with
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses only errorCode,
> > > which
> > > >> looks
> > > >> > > > > very
> > > >> > > > > > > > >> limiting,
> > > >> > > > > > > > >> >>>> in case
> > > >> > > > > > > > >> >>>> > >> > of
> > > >> > > > > > > > >> >>>> > >> > > > CLI
> > > >> > > > > > > > >> >>>> > >> > > > > we
> > > >> > > > > > > > >> >>>> > >> > > > > > >> may
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human
> readable
> > > >> error
> > > >> > > > > > > description.
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into account
> > > >> previous item
> > > >> > > > > about
> > > >> > > > > > > > >> batching,
> > > >> > > > > > > > >> >>>> what
> > > >> > > > > > > > >> >>>> > >> do
> > > >> > > > > > > > >> >>>> > >> > > you
> > > >> > > > > > > > >> >>>> > >> > > > > > think
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > about
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support
> > > >> regexp)
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest =>
> > > TopicName
> > > >> > > > > Partitions
> > > >> > > > > > > > >> Replicas
> > > >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse =>
> > > ErrorCode
> > > >> > > > > > > > ErrorDescription
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription =>
> string
> > > >> (empty
> > > >> > > if
> > > >> > > > > > > > >> successful)
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
> > > >> TopicNameRegexp
> > > >> > > >
> > > >
> > > > ...
> > > >
> > > > [Message clipped]
> > >
>
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
I'm +1 on Jun's suggestion as long as it can work for all the requests.

On Wed, Mar 18, 2015 at 3:35 PM, Jun Rao <ju...@confluent.io> wrote:

> Andrii,
>
> I think we agreed on the following.
>
> (a) Admin requests can be sent to and handled by any broker.
> (b) Admin requests are processed asynchronously, at least for now. That is,
> when the client gets a response, it just means that the request is
> initiated, but not necessarily completed. Then, it's up to the client to
> issue another request to check the status for completion.
>
> To support (a), we were thinking of doing request forwarding to the
> controller (utilizing KAFKA-1912). I am making an alternative proposal.
> Basically, the broker can just write to ZooKeeper to inform the controller
> about the request. For example, to handle partitionReassignment, the broker
> will just write the requested partitions to /admin/reassign_partitions
> (like what AdminUtils currently does) and then send a response to the
> client. This shouldn't take long and the implementation will be simpler
> than forwarding the requests to the controller through RPC.
>
> Thanks,
>
> Jun
>
>
> On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Jun,
> >
> > I might be wrong but didn't we agree we will let any broker from the
> > cluster handle *long-running* admin requests (at this time
> preferredReplica
> > and
> > reassignPartitions), via zk admin path. Thus CreateTopics etc should be
> > sent
> > only to the controller.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> > On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Joel, Andril,
> > >
> > > I think we agreed that those admin requests can be issued to any
> broker.
> > > Because of that, there doesn't seem to be a strong need to know the
> > > controller. So, perhaps we can proceed by not making any change to the
> > > format of TMR right now. When we start using create topic request in
> the
> > > producer, we will need a new version of TMR that doesn't trigger auto
> > topic
> > > creation. But that can be done later.
> > >
> > > As a first cut implementation, I think the broker can just write to ZK
> > > directly for
> > > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > > requests, instead of forwarding them to the controller. This will
> > simplify
> > > the implementation on the broker side.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > >
> > > > (Thanks Andrii for the summary)
> > > >
> > > > For (1) yes we will circle back on that shortly after syncing up in
> > > > person. I think it is close to getting committed although development
> > > > for KAFKA-1927 can probably begin without it.
> > > >
> > > > There is one more item we covered at the hangout. i.e., whether we
> > > > want to add the coordinator to the topic metadata response or provide
> > > > a clearer ClusterMetadataRequest.
> > > >
> > > > There are two reasons I think we should try and avoid adding the
> > > > field:
> > > > - It is irrelevant to topic metadata
> > > > - If we finally do request rerouting in Kafka then the field would
> add
> > > >   little to no value. (It still helps to have a separate
> > > >   ClusterMetadataRequest to query for cluster-wide information such
> as
> > > >   'which broker is the controller?' as Joe mentioned.)
> > > >
> > > > I think it would be cleaner to have an explicit
> ClusterMetadataRequest
> > > > that you can send to any broker in order to obtain the controller
> (and
> > > > in the future possibly other cluster-wide information). I think the
> > > > main argument against doing this and instead adding it to the topic
> > > > metadata response was convenience - i.e., you don't have to discover
> > > > the controller in advance. However, I don't see much actual
> > > > benefit/convenience in this and in fact think it is a non-issue. Let
> > > > me know if I'm overlooking something here.
> > > >
> > > > As an example, say we need to initiate partition reassignment by
> > > > issuing the new ReassignPartitionsRequest to the controller (assume
> we
> > > > already have the desired manual partition assignment).  If we are to
> > > > augment topic metadata response then the flow be something like this
> :
> > > >
> > > > - Issue topic metadata request to any broker (and discover the
> > > >   controller
> > > > - Connect to controller if required (i.e., if the broker above !=
> > > >   controller)
> > > > - Issue the partition reassignment request to the controller.
> > > >
> > > > With an explicit cluster metadata request it would be:
> > > > - Issue cluster metadata request to any broker
> > > > - Connect to controller if required (i.e., if the broker above !=
> > > >   controller)
> > > > - Issue the partition reassignment request
> > > >
> > > > So it seems to add little practical value and bloats topic metadata
> > > > response with an irrelevant detail.
> > > >
> > > > The other angle to this is the following - is it a matter of naming?
> > > > Should we just rename topic metadata request/response to just
> > > > MetadataRequest/Response and add cluster metadata to it? By that same
> > > > token should we also allow querying for the consumer coordinator (and
> > > > in future transaction coordinator) as well? This leads to a bloated
> > > > request which isn't very appealing and altogether confusing.
> > > >
> > > > Thanks,
> > > >
> > > > Joel
> > > >
> > > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > > > > Andri,
> > > > >
> > > > > Thanks for the summary.
> > > > >
> > > > > 1. I just realized that in order to start working on KAFKA-1927, we
> > > will
> > > > > need to merge the changes to OffsetCommitRequest (from 0.8.2) to
> > trunk.
> > > > > This is planned to be done as part of KAFKA-1634. So, we will need
> > > > Guozhang
> > > > > and Joel's help to wrap this up.
> > > > >
> > > > > 2. Thinking about this a bit more, if the semantic of those "write"
> > > > > requests is async (i.e., after the client gets a response, it just
> > > means
> > > > > that the operation is initiated, but not necessarily completed), we
> > > don't
> > > > > really need to forward the requests to the controller. Instead, the
> > > > > receiving broker can just write the operation to ZK as the admin
> > > command
> > > > > line tool previously does. This will simplify the implementation.
> > > > >
> > > > > 8. There is another implementation detail for describe topic.
> > Ideally,
> > > we
> > > > > want to read the topic config from the broker cache, instead of
> > > > ZooKeeper.
> > > > > Currently, every broker reads the topic-level config for all
> topics.
> > > > > However, it ignores those for topics not hosted on itself. So, we
> may
> > > > need
> > > > > to change TopicConfigManager a bit so that it caches the configs
> for
> > > all
> > > > > topics.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Guys,
> > > > > >
> > > > > > Thanks for a great discussion!
> > > > > > Here are the actions points:
> > > > > >
> > > > > > 1. Q: Get rid of all scala requests objects, use java protocol
> > > > definitions.
> > > > > >     A: Gwen kindly took that (KAFKA-1927). It's important to
> speed
> > up
> > > > > > review procedure
> > > > > >          there since this ticket blocks other important changes.
> > > > > >
> > > > > > 2. Q: Generic re-reroute facility vs client maintaining cluster
> > > state.
> > > > > >     A: Jay has added pseudo code to KAFKA-1912 - need to consider
> > > > whether
> > > > > > this will be
> > > > > >         easy to implement as a server-side feature (comments are
> > > > > > welcomed!).
> > > > > >
> > > > > > 3. Q: Controller field in wire protocol.
> > > > > >     A: This might be useful for clients, add this to
> > > > TopicMetadataResponse
> > > > > > (already in KIP).
> > > > > >
> > > > > > 4. Q: Decoupling topic creation from TMR.
> > > > > >     A: I will add proposed by Jun solution (using clientId for
> > that)
> > > > to the
> > > > > > KIP.
> > > > > >
> > > > > > 5. Q: Bumping new versions of TMR vs grabbing all protocol
> changes
> > in
> > > > one
> > > > > > version.
> > > > > >     A: It was decided to try to gather all changes to protocol
> > > (before
> > > > > > release).
> > > > > >         In case of TMR it worth checking: KAFKA-2020 and KIP-13
> > > > (quotas)
> > > > > >
> > > > > > 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
> > > > > >     A: Use jackson for that, /tools project is a separate jar so
> > > > shouldn't
> > > > > > be a big deal.
> > > > > >
> > > > > > 7.  Q: VerifyReassingPartitions vs generic status check command.
> > > > > >      A: For long-running requests like reassign partitions
> > *progress*
> > > > check
> > > > > > request is useful,
> > > > > >          it makes sense to introduce it.
> > > > > >
> > > > > >  Please add, correct me if I missed something.
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Joel,
> > > > > > >
> > > > > > > You are right, I removed ClusterMetadata because we have
> > partially
> > > > > > > what we need in TopicMetadata. Also, as Jay pointed out
> earlier,
> > we
> > > > > > > would like to have "orthogonal" API, but at the same time we
> need
> > > > > > > to be backward compatible.
> > > > > > >
> > > > > > > But I like your idea and even have some other arguments for
> this
> > > > option:
> > > > > > > There is also DescribeTopicRequest which was proposed in this
> > KIP,
> > > > > > > it returns topic configs, partitions, replication factor plus
> > > > partition
> > > > > > > ISR, ASR,
> > > > > > > leader replica. The later part is really already there in
> > > > > > > TopicMetadataRequest.
> > > > > > > So again we'll have to add stuff to TMR, not to duplicate some
> > info
> > > > in
> > > > > > > newly added requests. However, this way we'll end up with
> > "monster"
> > > > > > > request which returns cluster metadata, topic replication and
> > > config
> > > > info
> > > > > > > plus partition replication data. Seems logical to split TMR to
> > > > > > > - ClusterMetadata (brokers + controller, maybe smth else)
> > > > > > > - TopicMetadata (topic info + partition details)
> > > > > > > But since current TMR is involved in lots of places (including
> > > > network
> > > > > > > client,
> > > > > > > as I understand) this might be very serious change and it
> > probably
> > > > makes
> > > > > > > sense to stick with current approach.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <
> jjkoshy.w@gmail.com
> > >
> > > > wrote:
> > > > > > >
> > > > > > >> I may be missing some context but hopefully this will also be
> > > > covered
> > > > > > >> today: I thought the earlier proposal where there was an
> > explicit
> > > > > > >> ClusterMetadata request was clearer and explicit. During the
> > > course
> > > > of
> > > > > > >> this thread I think the conclusion was that the main need was
> > for
> > > > > > >> controller information and that can be rolled into the topic
> > > > metadata
> > > > > > >> response but that seems a bit irrelevant to topic metadata.
> > FWIW I
> > > > > > >> think the full broker-list is also irrelevant to topic
> metadata,
> > > but
> > > > > > >> it is already there and in use. I think there is still room
> for
> > an
> > > > > > >> explicit ClusterMetadata request since there may be other
> > > > > > >> cluster-level information that we may want to add over time
> (and
> > > > that
> > > > > > >> have nothing to do with topic metadata).
> > > > > > >>
> > > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi
> > wrote:
> > > > > > >> > Jun,
> > > > > > >> >
> > > > > > >> > 101. Okay, if you say that such use case is important. I
> also
> > > > think
> > > > > > >> > using clientId for these purposes is fine - if we already
> have
> > > > this
> > > > > > >> field
> > > > > > >> > as part of all Wire protocol messages, why not use that.
> > > > > > >> > I will update KIP-4 page if nobody has other ideas (which
> may
> > > > come up
> > > > > > >> > during the call today).
> > > > > > >> >
> > > > > > >> > 102.1 Agree, I'll update the KIP accordingly. I think we can
> > add
> > > > new,
> > > > > > >> > fine-grained error codes if some error code received in
> > specific
> > > > case
> > > > > > >> > won't give enough context to return a descriptive error
> > message
> > > > for
> > > > > > >> user.
> > > > > > >> >
> > > > > > >> > Look forward to discussing all outstanding issues in detail
> > > today
> > > > > > during
> > > > > > >> > the call.
> > > > > > >> >
> > > > > > >> > Thanks,
> > > > > > >> > Andrii Biletskyi
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <jun@confluent.io
> >
> > > > wrote:
> > > > > > >> >
> > > > > > >> > > 101. There may be a use case where you only want the
> topics
> > to
> > > > be
> > > > > > >> created
> > > > > > >> > > manually by admins. Currently, you can do that by
> disabling
> > > auto
> > > > > > topic
> > > > > > >> > > creation and issue topic creation from the TopicCommand.
> If
> > we
> > > > > > >> disable auto
> > > > > > >> > > topic creation completely on the broker and don't have a
> way
> > > to
> > > > > > >> distinguish
> > > > > > >> > > between topic creation requests from the regular clients
> and
> > > the
> > > > > > >> admin, we
> > > > > > >> > > can't support manual topic creation any more. I was
> thinking
> > > > that
> > > > > > >> another
> > > > > > >> > > way of distinguishing the clients making the topic
> creation
> > > > requests
> > > > > > >> is
> > > > > > >> > > using clientId. For example, the admin tool can set it to
> > > > something
> > > > > > >> like
> > > > > > >> > > admin and the broker can treat that clientId specially.
> > > > > > >> > >
> > > > > > >> > > Also, there is a related discussion in KAFKA-2020.
> > Currently,
> > > > we do
> > > > > > >> the
> > > > > > >> > > following in TopicMetadataResponse:
> > > > > > >> > >
> > > > > > >> > > 1. If leader is not available, we set the partition level
> > > error
> > > > code
> > > > > > >> to
> > > > > > >> > > LeaderNotAvailable.
> > > > > > >> > > 2. If a non-leader replica is not available, we take that
> > > > replica
> > > > > > out
> > > > > > >> of
> > > > > > >> > > the assigned replica list and isr in the response. As an
> > > > indication
> > > > > > >> for
> > > > > > >> > > doing that, we set the partition level error code to
> > > > > > >> ReplicaNotAvailable.
> > > > > > >> > >
> > > > > > >> > > This has a few problems. First, ReplicaNotAvailable
> probably
> > > > > > >> shouldn't be
> > > > > > >> > > an error, at least for the normal producer/consumer
> clients
> > > that
> > > > > > just
> > > > > > >> want
> > > > > > >> > > to find out the leader. Second, it can happen that both
> the
> > > > leader
> > > > > > and
> > > > > > >> > > another replica are not available at the same time. There
> is
> > > no
> > > > > > error
> > > > > > >> code
> > > > > > >> > > to indicate both. Third, even if a replica is not
> available,
> > > > it's
> > > > > > >> still
> > > > > > >> > > useful to return its replica id since some clients (e.g.
> > admin
> > > > tool)
> > > > > > >> may
> > > > > > >> > > still make use of it.
> > > > > > >> > >
> > > > > > >> > > One way to address this issue is to always return the
> > replica
> > > > id for
> > > > > > >> > > leader, assigned replicas, and isr regardless of whether
> the
> > > > > > >> corresponding
> > > > > > >> > > broker is live or not. Since we also return the list of
> live
> > > > > > brokers,
> > > > > > >> the
> > > > > > >> > > client can figure out whether a leader or a replica is
> live
> > or
> > > > not
> > > > > > >> and act
> > > > > > >> > > accordingly. This way, we don't need to set the partition
> > > level
> > > > > > error
> > > > > > >> code
> > > > > > >> > > when the leader or a replica is not available. This
> doesn't
> > > > change
> > > > > > >> the wire
> > > > > > >> > > protocol, but does change the semantics. Since we are
> > evolving
> > > > the
> > > > > > >> protocol
> > > > > > >> > > of TopicMetadataRequest here, we can potentially piggyback
> > the
> > > > > > change.
> > > > > > >> > >
> > > > > > >> > > 102.1 For those types of errors due to invalid input,
> > > shouldn't
> > > > we
> > > > > > >> just
> > > > > > >> > > guard it at parameter validation time and throw
> > > > > > >> InvalidArgumentException
> > > > > > >> > > without even sending the request to the broker?
> > > > > > >> > >
> > > > > > >> > > Thanks,
> > > > > > >> > >
> > > > > > >> > > Jun
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >> > >
> > > > > > >> > > > Jun,
> > > > > > >> > > >
> > > > > > >> > > > Answering your questions:
> > > > > > >> > > >
> > > > > > >> > > > 101. If I understand you correctly, you are saying
> future
> > > > producer
> > > > > > >> > > versions
> > > > > > >> > > > (which
> > > > > > >> > > > will be ported to TMR_V1) won't be able to automatically
> > > > create
> > > > > > >> topic (if
> > > > > > >> > > > we
> > > > > > >> > > > unconditionally remove topic creation from there). But
> we
> > > > need to
> > > > > > >> this
> > > > > > >> > > > preserve logic.
> > > > > > >> > > > Ok, about your proposal: I'm not a big fan too, when it
> > > comes
> > > > to
> > > > > > >> > > > differentiating
> > > > > > >> > > > clients directly in protocol schema. And also I'm not
> > sure I
> > > > > > >> understand
> > > > > > >> > > at
> > > > > > >> > > > all why
> > > > > > >> > > > auto.create.topics.enable is a server side
> configuration.
> > > Can
> > > > we
> > > > > > >> > > deprecate
> > > > > > >> > > > this setting
> > > > > > >> > > > in future versions, add this setting to producer and
> based
> > > on
> > > > that
> > > > > > >> upon
> > > > > > >> > > > receiving
> > > > > > >> > > > UnknownTopic create topic explicitly by a separate
> > producer
> > > > call
> > > > > > via
> > > > > > >> > > > adminClient?
> > > > > > >> > > >
> > > > > > >> > > > 102.1. Hm, yes. It's because we want to support batching
> > and
> > > > at
> > > > > > the
> > > > > > >> same
> > > > > > >> > > > time we
> > > > > > >> > > > want to give descriptive error messages for clients.
> Since
> > > > > > >> AdminClient
> > > > > > >> > > > holds the context
> > > > > > >> > > > to construct such messages (e.g. AdminClient layer can
> > know
> > > > that
> > > > > > >> > > > InvalidArgumentsCode
> > > > > > >> > > > means two cases: either invalid number - e.g. -1; or
> > > > > > >> replication-factor
> > > > > > >> > > was
> > > > > > >> > > > provided while
> > > > > > >> > > > partitions argument wasn't) - I wrapped responses in
> > > > Exceptions.
> > > > > > >> But I'm
> > > > > > >> > > > open to any
> > > > > > >> > > > other ideas, this was just initial version.
> > > > > > >> > > > 102.2. Yes, I agree. I'll change that to probably some
> > other
> > > > dto.
> > > > > > >> > > >
> > > > > > >> > > > Thanks,
> > > > > > >> > > > Andrii Biletskyi
> > > > > > >> > > >
> > > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <
> > jun@confluent.io>
> > > > > > wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Andrii,
> > > > > > >> > > > >
> > > > > > >> > > > > 101. That's what I was thinking too, but it may not be
> > > that
> > > > > > >> simple. In
> > > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > > >> > > > > we can let it not trigger auto topic creation. Then,
> in
> > > the
> > > > > > >> producer
> > > > > > >> > > > side,
> > > > > > >> > > > > if it gets an UnknownTopicException, it can explicitly
> > > > issue a
> > > > > > >> > > > > createTopicRequest for auto topic creation. On the
> > > consumer
> > > > > > side,
> > > > > > >> it
> > > > > > >> > > will
> > > > > > >> > > > > never issue createTopicRequest. This works when auto
> > topic
> > > > > > >> creation is
> > > > > > >> > > > > enabled on the broker side. However, I am not sure how
> > > > things
> > > > > > >> will work
> > > > > > >> > > > > when auto topic creation is disabled on the broker
> side.
> > > In
> > > > this
> > > > > > >> case,
> > > > > > >> > > we
> > > > > > >> > > > > want to have a way to manually create a topic,
> > potentially
> > > > > > through
> > > > > > >> > > admin
> > > > > > >> > > > > commands. However, then we need a way to distinguish
> > > > > > >> createTopicRequest
> > > > > > >> > > > > issued from the producer clients and the admin tools.
> > May
> > > > be we
> > > > > > >> can
> > > > > > >> > > add a
> > > > > > >> > > > > new field in createTopicRequest and set it differently
> > in
> > > > the
> > > > > > >> producer
> > > > > > >> > > > > client and the admin client. However, I am not sure if
> > > > that's
> > > > > > the
> > > > > > >> best
> > > > > > >> > > > > approach.
> > > > > > >> > > > >
> > > > > > >> > > > > 2. Yes, refactoring existing requests is a non-trivial
> > > > amount of
> > > > > > >> work.
> > > > > > >> > > I
> > > > > > >> > > > > posted some comments in KAFKA-1927. We will probably
> > have
> > > > to fix
> > > > > > >> > > > KAFKA-1927
> > > > > > >> > > > > first, before adding the new logic in KAFKA-1694.
> > > > Otherwise, the
> > > > > > >> > > changes
> > > > > > >> > > > > will be too big.
> > > > > > >> > > > >
> > > > > > >> > > > > 102. About the AdminClient:
> > > > > > >> > > > > 102.1. It's a bit weird that we return exception in
> the
> > > > api. It
> > > > > > >> seems
> > > > > > >> > > > that
> > > > > > >> > > > > we should either return error code or throw an
> exception
> > > > when
> > > > > > >> getting
> > > > > > >> > > the
> > > > > > >> > > > > response state.
> > > > > > >> > > > > 102.2. We probably shouldn't explicitly use the
> request
> > > > object
> > > > > > in
> > > > > > >> the
> > > > > > >> > > > api.
> > > > > > >> > > > > Not every request evolution requires an api change.
> > > > > > >> > > > >
> > > > > > >> > > > > Thanks,
> > > > > > >> > > > >
> > > > > > >> > > > > Jun
> > > > > > >> > > > >
> > > > > > >> > > > >
> > > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > > > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >> > > > >
> > > > > > >> > > > > > Jun,
> > > > > > >> > > > > >
> > > > > > >> > > > > > Thanks for you comments. Answers inline:
> > > > > > >> > > > > >
> > > > > > >> > > > > > 100. There are a few fields such as
> ReplicaAssignment,
> > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > >> > > > > > > and PartitionsSerialized that are represented as a
> > > > string,
> > > > > > but
> > > > > > >> > > > contain
> > > > > > >> > > > > > > composite structures in json. Could we flatten
> them
> > > out
> > > > > > >> directly in
> > > > > > >> > > > the
> > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > Yes, now with Admin Client this looks a bit weird.
> My
> > > > initial
> > > > > > >> > > > motivation
> > > > > > >> > > > > > was:
> > > > > > >> > > > > > ReassignPartitionCommand accepts input in json, we
> > want
> > > to
> > > > > > >> remain
> > > > > > >> > > > tools'
> > > > > > >> > > > > > interfaces unchanged, where possible.
> > > > > > >> > > > > > If we port it to deserialized format, in CLI (/tools
> > > > project)
> > > > > > >> we will
> > > > > > >> > > > > have
> > > > > > >> > > > > > to add some
> > > > > > >> > > > > > json library since /tools is written in java and
> we'll
> > > > need to
> > > > > > >> > > > > deserialize
> > > > > > >> > > > > > json file
> > > > > > >> > > > > > provided by a user. Can we quickly agree on what
> this
> > > > library
> > > > > > >> should
> > > > > > >> > > be
> > > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > > >> > > > > >
> > > > > > >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto
> > > topic
> > > > > > >> creation?
> > > > > > >> > > > This
> > > > > > >> > > > > > > will be a bit weird now that we have a separate
> > topic
> > > > > > >> creation api.
> > > > > > >> > > > > Have
> > > > > > >> > > > > > > you thought about how the new createTopicRequest
> and
> > > > > > >> > > > > TopicMetadataRequest
> > > > > > >> > > > > > > v1 will be used in the producer/consumer client,
> in
> > > > addition
> > > > > > >> to
> > > > > > >> > > admin
> > > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > > >> TopicMetadataRequest
> > > > > > >> > > from
> > > > > > >> > > > > the
> > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > I agree, this strange logic should be fixed. I'm not
> > > > confident
> > > > > > >> in
> > > > > > >> > > this
> > > > > > >> > > > > > Kafka part so
> > > > > > >> > > > > > correct me if I'm wrong, but it doesn't look like a
> > hard
> > > > thing
> > > > > > >> to
> > > > > > >> > > do, I
> > > > > > >> > > > > > think we can
> > > > > > >> > > > > > leverage AdminClient for that in Producer and
> > > > unconditionally
> > > > > > >> remove
> > > > > > >> > > > > topic
> > > > > > >> > > > > > creation from the TopicMetadataRequest_V1.
> > > > > > >> > > > > >
> > > > > > >> > > > > > 2. I think Jay meant getting rid of scala classes
> > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > >> > > did
> > > > > > >> > > > > > that
> > > > > > >> > > > > > > as a stop-gap thing when adding the new requests
> for
> > > the
> > > > > > >> consumers.
> > > > > > >> > > > > > > However, the long term plan is to get rid of all
> > those
> > > > and
> > > > > > >> just
> > > > > > >> > > reuse
> > > > > > >> > > > > the
> > > > > > >> > > > > > > java request/response in the client. Since this
> KIP
> > > > proposes
> > > > > > >> to
> > > > > > >> > > add a
> > > > > > >> > > > > > > significant number of new requests, perhaps we
> > should
> > > > bite
> > > > > > the
> > > > > > >> > > bullet
> > > > > > >> > > > > to
> > > > > > >> > > > > > > clean up the existing scala requests first before
> > > > adding new
> > > > > > >> ones?
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > Yes, looks like I misunderstood the point of
> > > > > > >> ...RequestAndHeader.
> > > > > > >> > > > Okay, I
> > > > > > >> > > > > > will
> > > > > > >> > > > > > rework that. The only thing is that I don't see any
> > > > example
> > > > > > how
> > > > > > >> it
> > > > > > >> > > was
> > > > > > >> > > > > done
> > > > > > >> > > > > > for at
> > > > > > >> > > > > > least one existing protocol message. Thus, as I
> > > > understand, I
> > > > > > >> have to
> > > > > > >> > > > > think
> > > > > > >> > > > > > how we
> > > > > > >> > > > > > are going to do it.
> > > > > > >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
> > > > > > reasonable,
> > > > > > >> but
> > > > > > >> > > if
> > > > > > >> > > > > it's
> > > > > > >> > > > > > an *obligatory*
> > > > > > >> > > > > > requirement to have Admin KIP done, I'm afraid this
> > can
> > > > be a
> > > > > > >> serious
> > > > > > >> > > > > > blocker for us.
> > > > > > >> > > > > > There are 13 protocol messages and all that would
> > > require
> > > > not
> > > > > > >> only
> > > > > > >> > > unit
> > > > > > >> > > > > > tests but quite
> > > > > > >> > > > > > intensive manual testing, no? I'm afraid I'm not the
> > > > right guy
> > > > > > >> to
> > > > > > >> > > cover
> > > > > > >> > > > > > pretty much all
> > > > > > >> > > > > > Kafka core internals :). Let me know your thoughts
> on
> > > this
> > > > > > >> item. Btw
> > > > > > >> > > > > there
> > > > > > >> > > > > > is a ticket to
> > > > > > >> > > > > > follow-up this issue (
> > > > > > >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > > >> > > > ).
> > > > > > >> > > > > >
> > > > > > >> > > > > > Thanks,
> > > > > > >> > > > > > Andrii Biletskyi
> > > > > > >> > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <
> > > > jun@confluent.io>
> > > > > > >> wrote:
> > > > > > >> > > > > >
> > > > > > >> > > > > > > Andrii,
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > A few more comments.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > 100. There are a few fields such as
> > ReplicaAssignment,
> > > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > > >> > > > > > > and PartitionsSerialized that are represented as a
> > > > string,
> > > > > > but
> > > > > > >> > > > contain
> > > > > > >> > > > > > > composite structures in json. Could we flatten
> them
> > > out
> > > > > > >> directly in
> > > > > > >> > > > the
> > > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger
> auto
> > > > topic
> > > > > > >> > > creation?
> > > > > > >> > > > > This
> > > > > > >> > > > > > > will be a bit weird now that we have a separate
> > topic
> > > > > > >> creation api.
> > > > > > >> > > > > Have
> > > > > > >> > > > > > > you thought about how the new createTopicRequest
> and
> > > > > > >> > > > > TopicMetadataRequest
> > > > > > >> > > > > > > v1 will be used in the producer/consumer client,
> in
> > > > addition
> > > > > > >> to
> > > > > > >> > > admin
> > > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > > >> TopicMetadataRequest
> > > > > > >> > > from
> > > > > > >> > > > > the
> > > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > 2. I think Jay meant getting rid of scala classes
> > > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > > >> HeartbeatResponseAndHeader. We
> > > > > > >> > > did
> > > > > > >> > > > > > that
> > > > > > >> > > > > > > as a stop-gap thing when adding the new requests
> for
> > > the
> > > > > > >> consumers.
> > > > > > >> > > > > > > However, the long term plan is to get rid of all
> > those
> > > > and
> > > > > > >> just
> > > > > > >> > > reuse
> > > > > > >> > > > > the
> > > > > > >> > > > > > > java request/response in the client. Since this
> KIP
> > > > proposes
> > > > > > >> to
> > > > > > >> > > add a
> > > > > > >> > > > > > > significant number of new requests, perhaps we
> > should
> > > > bite
> > > > > > the
> > > > > > >> > > bullet
> > > > > > >> > > > > to
> > > > > > >> > > > > > > clean up the existing scala requests first before
> > > > adding new
> > > > > > >> ones?
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Thanks,
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > Jun
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi
> <
> > > > > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > > Hi,
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > As said above - I list again all comments from
> > this
> > > > thread
> > > > > > >> so we
> > > > > > >> > > > > > > > can see what's left and finalize all pending
> > issues.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Comments from Jay:
> > > > > > >> > > > > > > > 1. This is much needed functionality, but there
> > are
> > > a
> > > > lot
> > > > > > >> of the
> > > > > > >> > > so
> > > > > > >> > > > > > let's
> > > > > > >> > > > > > > > really think these protocols through. We really
> > want
> > > > to
> > > > > > end
> > > > > > >> up
> > > > > > >> > > > with a
> > > > > > >> > > > > > set
> > > > > > >> > > > > > > > of well thought-out, orthoganol apis. For this
> > > reason
> > > > I
> > > > > > >> think it
> > > > > > >> > > is
> > > > > > >> > > > > > > really
> > > > > > >> > > > > > > > important to think through the end state even if
> > > that
> > > > > > >> includes
> > > > > > >> > > APIs
> > > > > > >> > > > > we
> > > > > > >> > > > > > > > won't implement in the first phase.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: Definitely behind this. Would appreciate if
> > there
> > > > are
> > > > > > >> concrete
> > > > > > >> > > > > > > comments
> > > > > > >> > > > > > > > how this can be improved.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 2. Let's please please please wait until we have
> > > > switched
> > > > > > >> the
> > > > > > >> > > > server
> > > > > > >> > > > > > over
> > > > > > >> > > > > > > > to the new java protocol definitions. If we add
> > > upteen
> > > > > > more
> > > > > > >> ad
> > > > > > >> > > hoc
> > > > > > >> > > > > > scala
> > > > > > >> > > > > > > > objects that is just generating more work for
> the
> > > > > > >> conversion we
> > > > > > >> > > > know
> > > > > > >> > > > > we
> > > > > > >> > > > > > > > have to do.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: Fixed in the latest patch - removed scala
> > > protocol
> > > > > > >> classes.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 3. This proposal introduces a new type of
> optional
> > > > > > >> parameter.
> > > > > > >> > > This
> > > > > > >> > > > is
> > > > > > >> > > > > > > > inconsistent with everything else in the
> protocol
> > > > where we
> > > > > > >> use -1
> > > > > > >> > > > or
> > > > > > >> > > > > > some
> > > > > > >> > > > > > > > other marker value. You could argue either way
> but
> > > > let's
> > > > > > >> stick
> > > > > > >> > > with
> > > > > > >> > > > > > that
> > > > > > >> > > > > > > > for consistency. For clients that implemented
> the
> > > > protocol
> > > > > > >> in a
> > > > > > >> > > > > better
> > > > > > >> > > > > > > way
> > > > > > >> > > > > > > > than our scala code these basic primitives are
> > hard
> > > to
> > > > > > >> change.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf
> > type
> > > > and
> > > > > > >> changed
> > > > > > >> > > > > > protocol
> > > > > > >> > > > > > > > accordingly.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> > > > > > >> TopicMetadataRequest
> > > > > > >> > > > > which
> > > > > > >> > > > > > > has
> > > > > > >> > > > > > > > brokers, topics, and partitions. I think we
> should
> > > > rename
> > > > > > >> that
> > > > > > >> > > > > request
> > > > > > >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest)
> > and
> > > > > > >> include the
> > > > > > >> > > id
> > > > > > >> > > > > of
> > > > > > >> > > > > > > the
> > > > > > >> > > > > > > > controller. Or are there other things we could
> add
> > > > here?
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: I agree. Updated the KIP. Let's extends
> > > > TopicMetadata
> > > > > > to
> > > > > > >> > > > version 2
> > > > > > >> > > > > > and
> > > > > > >> > > > > > > > include controller.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 5. We have a tendency to try to make a lot of
> > > requests
> > > > > > that
> > > > > > >> can
> > > > > > >> > > > only
> > > > > > >> > > > > go
> > > > > > >> > > > > > > to
> > > > > > >> > > > > > > > particular nodes. This adds a lot of burden for
> > > client
> > > > > > >> > > > > implementations
> > > > > > >> > > > > > > (it
> > > > > > >> > > > > > > > sounds easy but each discovery can fail in many
> > > parts
> > > > so
> > > > > > it
> > > > > > >> ends
> > > > > > >> > > up
> > > > > > >> > > > > > > being a
> > > > > > >> > > > > > > > full state machine to do right). I think we
> should
> > > > > > consider
> > > > > > >> > > making
> > > > > > >> > > > > > admin
> > > > > > >> > > > > > > > commands and ideally as many of the other apis
> as
> > > > possible
> > > > > > >> > > > available
> > > > > > >> > > > > on
> > > > > > >> > > > > > > all
> > > > > > >> > > > > > > > brokers and just redirect to the controller on
> the
> > > > broker
> > > > > > >> side.
> > > > > > >> > > > > Perhaps
> > > > > > >> > > > > > > > there would be a general way to encapsulate this
> > > > > > re-routing
> > > > > > >> > > > behavior.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: It's a very interesting idea, but seems there
> > are
> > > > some
> > > > > > >> > > concerns
> > > > > > >> > > > > > about
> > > > > > >> > > > > > > > this
> > > > > > >> > > > > > > > feature (like performance considerations, how
> this
> > > > will
> > > > > > >> > > complicate
> > > > > > >> > > > > > server
> > > > > > >> > > > > > > > etc).
> > > > > > >> > > > > > > > I believe this shouldn't be a blocker. If this
> > > > feature is
> > > > > > >> > > > implemented
> > > > > > >> > > > > > at
> > > > > > >> > > > > > > > some
> > > > > > >> > > > > > > > point it won't affect Admin changes - at least
> no
> > > > changes
> > > > > > to
> > > > > > >> > > public
> > > > > > >> > > > > API
> > > > > > >> > > > > > > > will be required.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 6. We should probably normalize the key value
> > pairs
> > > > used
> > > > > > for
> > > > > > >> > > > configs
> > > > > > >> > > > > > > rather
> > > > > > >> > > > > > > > than embedding a new formatting. So two strings
> > > rather
> > > > > > than
> > > > > > >> one
> > > > > > >> > > > with
> > > > > > >> > > > > an
> > > > > > >> > > > > > > > internal equals sign.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: Fixed in the latest patch - normalized
> configs
> > > and
> > > > > > >> changed
> > > > > > >> > > > > protocol
> > > > > > >> > > > > > > > accordingly.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 7. Is the postcondition of these APIs that the
> > > > command has
> > > > > > >> begun
> > > > > > >> > > or
> > > > > > >> > > > > > that
> > > > > > >> > > > > > > > the command has been completed? It is a lot more
> > > > usable if
> > > > > > >> the
> > > > > > >> > > > > command
> > > > > > >> > > > > > > has
> > > > > > >> > > > > > > > been completed so you know that if you create a
> > > topic
> > > > and
> > > > > > >> then
> > > > > > >> > > > > publish
> > > > > > >> > > > > > to
> > > > > > >> > > > > > > > it you won't get an exception about there being
> no
> > > > such
> > > > > > >> topic.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: For long running requests (like reassign
> > > > partitions) -
> > > > > > >> the
> > > > > > >> > > post
> > > > > > >> > > > > > > > condition is
> > > > > > >> > > > > > > > command has begun - so we don't block the
> client.
> > In
> > > > case
> > > > > > >> of your
> > > > > > >> > > > > > > example -
> > > > > > >> > > > > > > > topic commands, this will be refactored and
> topic
> > > > commands
> > > > > > >> will
> > > > > > >> > > be
> > > > > > >> > > > > > > executed
> > > > > > >> > > > > > > > immediately, since the Controller will serve
> Admin
> > > > > > requests
> > > > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 8. Describe topic and list topics duplicate a
> lot
> > of
> > > > stuff
> > > > > > >> in the
> > > > > > >> > > > > > > metadata
> > > > > > >> > > > > > > > request. Is there a reason to give back topics
> > > marked
> > > > for
> > > > > > >> > > > deletion? I
> > > > > > >> > > > > > > feel
> > > > > > >> > > > > > > > like if we just make the post-condition of the
> > > delete
> > > > > > >> command be
> > > > > > >> > > > that
> > > > > > >> > > > > > the
> > > > > > >> > > > > > > > topic is deleted that will get rid of the need
> for
> > > > this
> > > > > > >> right?
> > > > > > >> > > And
> > > > > > >> > > > it
> > > > > > >> > > > > > > will
> > > > > > >> > > > > > > > be much more intuitive.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: Fixed in the latest patch - removed topics
> > marked
> > > > for
> > > > > > >> deletion
> > > > > > >> > > > in
> > > > > > >> > > > > > > > ListTopicsRequest.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 9. Should we consider batching these requests?
> We
> > > have
> > > > > > >> generally
> > > > > > >> > > > > tried
> > > > > > >> > > > > > to
> > > > > > >> > > > > > > > allow multiple operations to be batched. My
> > > suspicion
> > > > is
> > > > > > >> that
> > > > > > >> > > > without
> > > > > > >> > > > > > > this
> > > > > > >> > > > > > > > we will get a lot of code that does something
> like
> > > > > > >> > > > > > > >    for(topic: adminClient.listTopics())
> > > > > > >> > > > > > > >       adminClient.describeTopic(topic)
> > > > > > >> > > > > > > > this code will work great when you test on 5
> > topics
> > > > but
> > > > > > not
> > > > > > >> do as
> > > > > > >> > > > > well
> > > > > > >> > > > > > if
> > > > > > >> > > > > > > > you have 50k.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: Updated the KIP - please check "Topic Admin
> > > Schema"
> > > > > > >> section.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 10. I think we should also discuss how we want
> to
> > > > expose a
> > > > > > >> > > > > programmatic
> > > > > > >> > > > > > > JVM
> > > > > > >> > > > > > > > client api for these operations. Currently
> people
> > > > rely on
> > > > > > >> > > > AdminUtils
> > > > > > >> > > > > > > which
> > > > > > >> > > > > > > > is totally sketchy. I think we probably need
> > another
> > > > > > client
> > > > > > >> under
> > > > > > >> > > > > > > clients/
> > > > > > >> > > > > > > > that exposes administrative functionality. We
> will
> > > > need
> > > > > > >> this just
> > > > > > >> > > > to
> > > > > > >> > > > > > > > properly test the new apis, I suspect. We should
> > > > figure
> > > > > > out
> > > > > > >> that
> > > > > > >> > > > API.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: Updated the KIP - please check "Admin Client"
> > > > section
> > > > > > >> with an
> > > > > > >> > > > > > initial
> > > > > > >> > > > > > > > API proposal.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 11. The other information that would be really
> > > useful
> > > > to
> > > > > > get
> > > > > > >> > > would
> > > > > > >> > > > be
> > > > > > >> > > > > > > > information about partitions--how much data is
> in
> > > the
> > > > > > >> partition,
> > > > > > >> > > > what
> > > > > > >> > > > > > are
> > > > > > >> > > > > > > > the segment offsets, what is the log-end offset
> > > (i.e.
> > > > last
> > > > > > >> > > offset),
> > > > > > >> > > > > > what
> > > > > > >> > > > > > > is
> > > > > > >> > > > > > > > the compaction point, etc. I think that done
> right
> > > > this
> > > > > > >> would be
> > > > > > >> > > > the
> > > > > > >> > > > > > > > successor to the very awkward OffsetRequest we
> > have
> > > > today.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the
> > > latest
> > > > > > >> patch. I
> > > > > > >> > > > > believe
> > > > > > >> > > > > > > > this should
> > > > > > >> > > > > > > > be resolved in a separate KIP / jira ticket.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 12. Generally we can do good error handling
> > without
> > > > > > needing
> > > > > > >> > > custom
> > > > > > >> > > > > > > > server-side
> > > > > > >> > > > > > > > messages. I.e. generally the client has the
> > context
> > > to
> > > > > > know
> > > > > > >> that
> > > > > > >> > > if
> > > > > > >> > > > > it
> > > > > > >> > > > > > > got
> > > > > > >> > > > > > > > an error that the topic doesn't exist to say
> > "Topic
> > > X
> > > > > > >> doesn't
> > > > > > >> > > > exist"
> > > > > > >> > > > > > > rather
> > > > > > >> > > > > > > > than "error code 14" (or whatever). Maybe there
> > are
> > > > > > specific
> > > > > > >> > > cases
> > > > > > >> > > > > > where
> > > > > > >> > > > > > > > this is hard? If we want to add server-side
> error
> > > > messages
> > > > > > >> we
> > > > > > >> > > > really
> > > > > > >> > > > > do
> > > > > > >> > > > > > > > need to do this in a consistent way across the
> > > > protocol.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: Updated the KIP - please check "Protocol
> > Errors"
> > > > > > >> section. I
> > > > > > >> > > > added
> > > > > > >> > > > > > the
> > > > > > >> > > > > > > > comprehensive, fine-grained list of error codes.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Comments from Guozhang:
> > > > > > >> > > > > > > > 13. Describe topic request: it would be great to
> > go
> > > > beyond
> > > > > > >> just
> > > > > > >> > > > > > batching
> > > > > > >> > > > > > > on
> > > > > > >> > > > > > > > topic name regex for this request. For example,
> a
> > > very
> > > > > > >> common use
> > > > > > >> > > > > case
> > > > > > >> > > > > > of
> > > > > > >> > > > > > > > the topic command is to list all topics whose
> > config
> > > > A's
> > > > > > >> value is
> > > > > > >> > > > B.
> > > > > > >> > > > > > With
> > > > > > >> > > > > > > > topic name regex then we have to first retrieve
> > > > __all__
> > > > > > >> topics's
> > > > > > >> > > > > > > > description info and then filter at the client
> > end,
> > > > which
> > > > > > >> will
> > > > > > >> > > be a
> > > > > > >> > > > > > huge
> > > > > > >> > > > > > > > burden on ZK.
> > > > > > >> > > > > > > > AND
> > > > > > >> > > > > > > > 14. Config K-Vs in create topic: this is related
> > to
> > > > the
> > > > > > >> previous
> > > > > > >> > > > > point;
> > > > > > >> > > > > > > > maybe we can add another metadata K-V or just a
> > > > metadata
> > > > > > >> string
> > > > > > >> > > > along
> > > > > > >> > > > > > > side
> > > > > > >> > > > > > > > with config K-V in create topic like we did for
> > > offset
> > > > > > >> commit
> > > > > > >> > > > > request.
> > > > > > >> > > > > > > This
> > > > > > >> > > > > > > > field can be quite useful in storing information
> > > like
> > > > > > >> "owner" of
> > > > > > >> > > > the
> > > > > > >> > > > > > > topic
> > > > > > >> > > > > > > > who issue the create command, etc, which is
> quite
> > > > > > important
> > > > > > >> for a
> > > > > > >> > > > > > > > multi-tenant setting. Then in the describe topic
> > > > request
> > > > > > we
> > > > > > >> can
> > > > > > >> > > > also
> > > > > > >> > > > > > > batch
> > > > > > >> > > > > > > > on regex of the metadata field.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: As discussed it is very interesting but can
> be
> > > > > > >> implemented
> > > > > > >> > > later
> > > > > > >> > > > > > after
> > > > > > >> > > > > > > > we have some basic functionality there.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > 15. Today all the admin operations are async in
> > the
> > > > sense
> > > > > > >> that
> > > > > > >> > > > > command
> > > > > > >> > > > > > > will
> > > > > > >> > > > > > > > return once it is written in ZK, and that is why
> > we
> > > > need
> > > > > > >> extra
> > > > > > >> > > > > > > verification
> > > > > > >> > > > > > > > like testUtil.waitForTopicCreated() / verify
> > > partition
> > > > > > >> > > reassignment
> > > > > > >> > > > > > > > request, etc. With admin requests we could add a
> > > flag
> > > > to
> > > > > > >> enable /
> > > > > > >> > > > > > disable
> > > > > > >> > > > > > > > synchronous requests; when it is turned on, the
> > > > response
> > > > > > >> will not
> > > > > > >> > > > > > return
> > > > > > >> > > > > > > > until the request has been completed. And for
> > async
> > > > > > >> requests we
> > > > > > >> > > can
> > > > > > >> > > > > > add a
> > > > > > >> > > > > > > > "token" field in the response, and then only
> need
> > a
> > > > > > general
> > > > > > >> > > "admin
> > > > > > >> > > > > > > > verification request" with the given token to
> > check
> > > > if the
> > > > > > >> async
> > > > > > >> > > > > > request
> > > > > > >> > > > > > > > has been completed.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: I see your point. My idea was to provide
> > specific
> > > > > > >> > > > Verify...Request
> > > > > > >> > > > > > per
> > > > > > >> > > > > > > > each
> > > > > > >> > > > > > > > long running request, where needed. We can do it
> > the
> > > > way
> > > > > > you
> > > > > > >> > > > suggest.
> > > > > > >> > > > > > The
> > > > > > >> > > > > > > > only
> > > > > > >> > > > > > > > concern is that introducing a token we again
> will
> > > make
> > > > > > >> schema
> > > > > > >> > > > > > "dynamic".
> > > > > > >> > > > > > > We
> > > > > > >> > > > > > > > wanted
> > > > > > >> > > > > > > > to do similar thing introducing single
> > AdminRequest
> > > > for
> > > > > > all
> > > > > > >> topic
> > > > > > >> > > > > > > commands
> > > > > > >> > > > > > > > but rejected
> > > > > > >> > > > > > > > this idea because we wanted to have schema
> > defined.
> > > So
> > > > > > this
> > > > > > >> is
> > > > > > >> > > > more a
> > > > > > >> > > > > > > > choice between:
> > > > > > >> > > > > > > > a) have fixed schema but introduce each time new
> > > > > > >> Verify...Request
> > > > > > >> > > > for
> > > > > > >> > > > > > > > long-running requests
> > > > > > >> > > > > > > > b) use one request for verification but
> generalize
> > > it
> > > > with
> > > > > > >> token
> > > > > > >> > > > > > > > I'm fine with whatever decision community come
> to.
> > > > Just
> > > > > > let
> > > > > > >> me
> > > > > > >> > > know
> > > > > > >> > > > > > your
> > > > > > >> > > > > > > > thoughts.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Comment from Gwen:
> > > > > > >> > > > > > > > 16. Specifically for ownership, I think the plan
> > is
> > > > to add
> > > > > > >> ACL
> > > > > > >> > > (it
> > > > > > >> > > > > > sounds
> > > > > > >> > > > > > > > like you are describing ACL) via an external
> > system
> > > > > > (Argus,
> > > > > > >> > > > Sentry).
> > > > > > >> > > > > > > > I remember KIP-11 described this, but I can't
> find
> > > > the KIP
> > > > > > >> any
> > > > > > >> > > > > longer.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > A: Okay, no problem. Not sure though how we are
> > > going
> > > > to
> > > > > > >> handle
> > > > > > >> > > it.
> > > > > > >> > > > > > Wait
> > > > > > >> > > > > > > > which KIP
> > > > > > >> > > > > > > > will be committed first and include changes to
> > > > > > >> TopicMetadata from
> > > > > > >> > > > the
> > > > > > >> > > > > > > later
> > > > > > >> > > > > > > > one?
> > > > > > >> > > > > > > > Anyway, I added this note to "Open Questions"
> > > section
> > > > so
> > > > > > we
> > > > > > >> don't
> > > > > > >> > > > > miss
> > > > > > >> > > > > > > this
> > > > > > >> > > > > > > > piece.
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > Thanks,
> > > > > > >> > > > > > > > Andrii Biletskyi
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii
> > Biletskyi <
> > > > > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > > Hi all,
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > Today I uploaded the patch that covers some of
> > the
> > > > > > >> discussed
> > > > > > >> > > and
> > > > > > >> > > > > > agreed
> > > > > > >> > > > > > > > > items:
> > > > > > >> > > > > > > > > - removed MaybeOf optional type
> > > > > > >> > > > > > > > > - switched to java protocol definitions
> > > > > > >> > > > > > > > > - simplified messages (normalized configs,
> > removed
> > > > topic
> > > > > > >> marked
> > > > > > >> > > > for
> > > > > > >> > > > > > > > > deletion)
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > I also updated the KIP-4 with respective
> changes
> > > and
> > > > > > >> wrote down
> > > > > > >> > > > my
> > > > > > >> > > > > > > > > proposal for
> > > > > > >> > > > > > > > > pending items:
> > > > > > >> > > > > > > > > - Batch Admin Operations -> updated Wire
> > Protocol
> > > > schema
> > > > > > >> > > proposal
> > > > > > >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
> > > > > > >> > > > TopicMetadataRequest
> > > > > > >> > > > > > > > > - Admin Client -> updated my initial proposal
> to
> > > > reflect
> > > > > > >> > > batching
> > > > > > >> > > > > > > > > - Error codes -> proposed fine-grained error
> > code
> > > > > > instead
> > > > > > >> of
> > > > > > >> > > > > > > > > AdminRequestFailed
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > I will also send a separate email to cover all
> > > > comments
> > > > > > >> from
> > > > > > >> > > this
> > > > > > >> > > > > > > thread.
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > Thanks,
> > > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira
> <
> > > > > > >> > > > > gshapira@cloudera.com
> > > > > > >> > > > > > >
> > > > > > >> > > > > > > > > wrote:
> > > > > > >> > > > > > > > >
> > > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > > >> > > > > > > > >>
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >>
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > >> > > > > > > > >> )
> > > > > > >> > > > > > > > >> It actually specifies changes to the Metadata
> > > > protocol,
> > > > > > >> so
> > > > > > >> > > > making
> > > > > > >> > > > > > sure
> > > > > > >> > > > > > > > >> both KIPs are consistent in this regard will
> be
> > > > good.
> > > > > > >> > > > > > > > >>
> > > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen
> Shapira
> > <
> > > > > > >> > > > > > gshapira@cloudera.com
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > > > >> wrote:
> > > > > > >> > > > > > > > >> > Specifically for ownership, I think the
> plan
> > is
> > > > to
> > > > > > add
> > > > > > >> ACL
> > > > > > >> > > (it
> > > > > > >> > > > > > > sounds
> > > > > > >> > > > > > > > >> > like you are describing ACL) via an
> external
> > > > system
> > > > > > >> (Argus,
> > > > > > >> > > > > > Sentry).
> > > > > > >> > > > > > > > >> > I remember KIP-11 described this, but I
> can't
> > > > find
> > > > > > the
> > > > > > >> KIP
> > > > > > >> > > any
> > > > > > >> > > > > > > longer.
> > > > > > >> > > > > > > > >> >
> > > > > > >> > > > > > > > >> > Regardless, I think KIP-4 focuses on
> getting
> > > > > > >> information
> > > > > > >> > > that
> > > > > > >> > > > > > > already
> > > > > > >> > > > > > > > >> > exists from Kafka brokers, not on adding
> > > > information
> > > > > > >> that
> > > > > > >> > > > > perhaps
> > > > > > >> > > > > > > > >> > should exist but doesn't yet?
> > > > > > >> > > > > > > > >> >
> > > > > > >> > > > > > > > >> > Gwen
> > > > > > >> > > > > > > > >> >
> > > > > > >> > > > > > > > >> >
> > > > > > >> > > > > > > > >> >
> > > > > > >> > > > > > > > >> >
> > > > > > >> > > > > > > > >> >
> > > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang
> > Wang
> > > <
> > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > >> > > > > > > > >> wrote:
> > > > > > >> > > > > > > > >> >> Folks,
> > > > > > >> > > > > > > > >> >>
> > > > > > >> > > > > > > > >> >> Just want to elaborate a bit more on the
> > > > > > create-topic
> > > > > > >> > > > metadata
> > > > > > >> > > > > > and
> > > > > > >> > > > > > > > >> batching
> > > > > > >> > > > > > > > >> >> describe-topic based on config / metadata
> in
> > > my
> > > > > > >> previous
> > > > > > >> > > > email
> > > > > > >> > > > > as
> > > > > > >> > > > > > > we
> > > > > > >> > > > > > > > >> work
> > > > > > >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to
> > have
> > > > some
> > > > > > >> sort of
> > > > > > >> > > > > topic
> > > > > > >> > > > > > > > >> management
> > > > > > >> > > > > > > > >> >> mechanisms, which I think is quite
> important
> > > in
> > > > a
> > > > > > >> > > > multi-tenant
> > > > > > >> > > > > /
> > > > > > >> > > > > > > > cloud
> > > > > > >> > > > > > > > >> >> architecture: today anyone can create
> topics
> > > in
> > > > a
> > > > > > >> shared
> > > > > > >> > > > Kafka
> > > > > > >> > > > > > > > >> cluster, but
> > > > > > >> > > > > > > > >> >> there is no concept or "ownership" of
> topics
> > > > that
> > > > > > are
> > > > > > >> > > created
> > > > > > >> > > > > by
> > > > > > >> > > > > > > > >> different
> > > > > > >> > > > > > > > >> >> users. For example, at LinkedIn we
> basically
> > > > > > >> distinguish
> > > > > > >> > > > topic
> > > > > > >> > > > > > > owners
> > > > > > >> > > > > > > > >> via
> > > > > > >> > > > > > > > >> >> some casual topic name prefix, which is a
> > bit
> > > > > > awkward
> > > > > > >> and
> > > > > > >> > > > does
> > > > > > >> > > > > > not
> > > > > > >> > > > > > > > fly
> > > > > > >> > > > > > > > >> as
> > > > > > >> > > > > > > > >> >> we scale our customers. It would be great
> to
> > > use
> > > > > > >> > > > > describe-topics
> > > > > > >> > > > > > > such
> > > > > > >> > > > > > > > >> as:
> > > > > > >> > > > > > > > >> >>
> > > > > > >> > > > > > > > >> >> Describe all topics that is created by me.
> > > > > > >> > > > > > > > >> >>
> > > > > > >> > > > > > > > >> >> Describe all topics whose retention time
> is
> > > > > > overriden
> > > > > > >> to X.
> > > > > > >> > > > > > > > >> >>
> > > > > > >> > > > > > > > >> >> Describe all topics whose writable group
> > > include
> > > > > > user
> > > > > > >> Y
> > > > > > >> > > (this
> > > > > > >> > > > > is
> > > > > > >> > > > > > > > >> related to
> > > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > > >> > > > > > > > >> >>
> > > > > > >> > > > > > > > >> >> One possible way to achieve this is to
> add a
> > > > > > metadata
> > > > > > >> file
> > > > > > >> > > in
> > > > > > >> > > > > the
> > > > > > >> > > > > > > > >> >> create-topic request, whose value will
> also
> > be
> > > > > > >> written ZK
> > > > > > >> > > as
> > > > > > >> > > > we
> > > > > > >> > > > > > > > create
> > > > > > >> > > > > > > > >> the
> > > > > > >> > > > > > > > >> >> topic; then describe-topics can choose to
> > > batch
> > > > > > topics
> > > > > > >> > > based
> > > > > > >> > > > on
> > > > > > >> > > > > > 1)
> > > > > > >> > > > > > > > name
> > > > > > >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata
> > > > regex,
> > > > > > etc.
> > > > > > >> > > > > > > > >> >>
> > > > > > >> > > > > > > > >> >> Thoughts?
> > > > > > >> > > > > > > > >> >>
> > > > > > >> > > > > > > > >> >> Guozhang
> > > > > > >> > > > > > > > >> >>
> > > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang
> > Wang
> > > <
> > > > > > >> > > > > > wangguoz@gmail.com>
> > > > > > >> > > > > > > > >> wrote:
> > > > > > >> > > > > > > > >> >>
> > > > > > >> > > > > > > > >> >>> Thanks for the updated wiki. A few
> comments
> > > > below:
> > > > > > >> > > > > > > > >> >>>
> > > > > > >> > > > > > > > >> >>> 1. Error description in response: I think
> > if
> > > > some
> > > > > > >> > > errorCode
> > > > > > >> > > > > > could
> > > > > > >> > > > > > > > >> indicate
> > > > > > >> > > > > > > > >> >>> several different error cases then we
> > should
> > > > really
> > > > > > >> change
> > > > > > >> > > > it
> > > > > > >> > > > > to
> > > > > > >> > > > > > > > >> multiple
> > > > > > >> > > > > > > > >> >>> codes. In general the errorCode itself
> > would
> > > be
> > > > > > >> precise
> > > > > > >> > > and
> > > > > > >> > > > > > > > >> sufficient for
> > > > > > >> > > > > > > > >> >>> describing the server side errors.
> > > > > > >> > > > > > > > >> >>>
> > > > > > >> > > > > > > > >> >>> 2. Describe topic request: it would be
> > great
> > > > to go
> > > > > > >> beyond
> > > > > > >> > > > just
> > > > > > >> > > > > > > > >> batching on
> > > > > > >> > > > > > > > >> >>> topic name regex for this request. For
> > > > example, a
> > > > > > >> very
> > > > > > >> > > > common
> > > > > > >> > > > > > use
> > > > > > >> > > > > > > > >> case of
> > > > > > >> > > > > > > > >> >>> the topic command is to list all topics
> > whose
> > > > > > config
> > > > > > >> A's
> > > > > > >> > > > value
> > > > > > >> > > > > > is
> > > > > > >> > > > > > > B.
> > > > > > >> > > > > > > > >> With
> > > > > > >> > > > > > > > >> >>> topic name regex then we have to first
> > > retrieve
> > > > > > >> __all__
> > > > > > >> > > > > topics's
> > > > > > >> > > > > > > > >> >>> description info and then filter at the
> > > client
> > > > end,
> > > > > > >> which
> > > > > > >> > > > will
> > > > > > >> > > > > > be
> > > > > > >> > > > > > > a
> > > > > > >> > > > > > > > >> huge
> > > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > > >> > > > > > > > >> >>>
> > > > > > >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is
> > > > related to
> > > > > > >> the
> > > > > > >> > > > > previous
> > > > > > >> > > > > > > > point;
> > > > > > >> > > > > > > > >> >>> maybe we can add another metadata K-V or
> > > just a
> > > > > > >> metadata
> > > > > > >> > > > > string
> > > > > > >> > > > > > > > along
> > > > > > >> > > > > > > > >> side
> > > > > > >> > > > > > > > >> >>> with config K-V in create topic like we
> did
> > > for
> > > > > > >> offset
> > > > > > >> > > > commit
> > > > > > >> > > > > > > > >> request. This
> > > > > > >> > > > > > > > >> >>> field can be quite useful in storing
> > > > information
> > > > > > like
> > > > > > >> > > > "owner"
> > > > > > >> > > > > of
> > > > > > >> > > > > > > the
> > > > > > >> > > > > > > > >> topic
> > > > > > >> > > > > > > > >> >>> who issue the create command, etc, which
> is
> > > > quite
> > > > > > >> > > important
> > > > > > >> > > > > for
> > > > > > >> > > > > > a
> > > > > > >> > > > > > > > >> >>> multi-tenant setting. Then in the
> describe
> > > > topic
> > > > > > >> request
> > > > > > >> > > we
> > > > > > >> > > > > can
> > > > > > >> > > > > > > also
> > > > > > >> > > > > > > > >> batch
> > > > > > >> > > > > > > > >> >>> on regex of the metadata field.
> > > > > > >> > > > > > > > >> >>>
> > > > > > >> > > > > > > > >> >>> 4. Today all the admin operations are
> async
> > > in
> > > > the
> > > > > > >> sense
> > > > > > >> > > > that
> > > > > > >> > > > > > > > command
> > > > > > >> > > > > > > > >> will
> > > > > > >> > > > > > > > >> >>> return once it is written in ZK, and that
> > is
> > > > why we
> > > > > > >> need
> > > > > > >> > > > extra
> > > > > > >> > > > > > > > >> verification
> > > > > > >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() /
> > verify
> > > > > > >> partition
> > > > > > >> > > > > > > reassignment
> > > > > > >> > > > > > > > >> >>> request, etc. With admin requests we
> could
> > > add
> > > > a
> > > > > > >> flag to
> > > > > > >> > > > > enable
> > > > > > >> > > > > > /
> > > > > > >> > > > > > > > >> disable
> > > > > > >> > > > > > > > >> >>> synchronous requests; when it is turned
> on,
> > > the
> > > > > > >> response
> > > > > > >> > > > will
> > > > > > >> > > > > > not
> > > > > > >> > > > > > > > >> return
> > > > > > >> > > > > > > > >> >>> until the request has been completed. And
> > for
> > > > async
> > > > > > >> > > requests
> > > > > > >> > > > > we
> > > > > > >> > > > > > > can
> > > > > > >> > > > > > > > >> add a
> > > > > > >> > > > > > > > >> >>> "token" field in the response, and then
> > only
> > > > need a
> > > > > > >> > > general
> > > > > > >> > > > > > "admin
> > > > > > >> > > > > > > > >> >>> verification request" with the given
> token
> > to
> > > > check
> > > > > > >> if the
> > > > > > >> > > > > async
> > > > > > >> > > > > > > > >> request
> > > > > > >> > > > > > > > >> >>> has been completed.
> > > > > > >> > > > > > > > >> >>>
> > > > > > >> > > > > > > > >> >>> 5. +1 for extending Metadata request to
> > > include
> > > > > > >> > > controller /
> > > > > > >> > > > > > > > >> coordinator
> > > > > > >> > > > > > > > >> >>> information, and then we can remove the
> > > > > > >> ConsumerMetadata /
> > > > > > >> > > > > > > > >> ClusterMetadata
> > > > > > >> > > > > > > > >> >>> requests.
> > > > > > >> > > > > > > > >> >>>
> > > > > > >> > > > > > > > >> >>> Guozhang
> > > > > > >> > > > > > > > >> >>>
> > > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel
> > Koshy <
> > > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > > >> > > > > > > > >> wrote:
> > > > > > >> > > > > > > > >> >>>
> > > > > > >> > > > > > > > >> >>>> Thanks for sending that out Joe - I
> don't
> > > > think I
> > > > > > >> will be
> > > > > > >> > > > > able
> > > > > > >> > > > > > to
> > > > > > >> > > > > > > > >> make
> > > > > > >> > > > > > > > >> >>>> it today, so if notes can be sent out
> > > > afterward
> > > > > > that
> > > > > > >> > > would
> > > > > > >> > > > be
> > > > > > >> > > > > > > > great.
> > > > > > >> > > > > > > > >> >>>>
> > > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM
> -0800,
> > > Gwen
> > > > > > >> Shapira
> > > > > > >> > > > wrote:
> > > > > > >> > > > > > > > >> >>>> > Thanks for sending this out Joe.
> Looking
> > > > forward
> > > > > > >> to
> > > > > > >> > > > > chatting
> > > > > > >> > > > > > > with
> > > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > > >> > > > > > > > >> >>>> >
> > > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe
> > Stein
> > > <
> > > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > > >> > > > > > > > >> wrote:
> > > > > > >> > > > > > > > >> >>>> > > Hey, I just sent out a google
> hangout
> > > > invite
> > > > > > to
> > > > > > >> all
> > > > > > >> > > > pmc,
> > > > > > >> > > > > > > > >> committers
> > > > > > >> > > > > > > > >> >>>> and
> > > > > > >> > > > > > > > >> >>>> > > everyone I found working on a KIP.
> If
> > I
> > > > missed
> > > > > > >> anyone
> > > > > > >> > > > in
> > > > > > >> > > > > > the
> > > > > > >> > > > > > > > >> invite
> > > > > > >> > > > > > > > >> >>>> please
> > > > > > >> > > > > > > > >> >>>> > > let me know and can update it, np.
> > > > > > >> > > > > > > > >> >>>> > >
> > > > > > >> > > > > > > > >> >>>> > > We should do this every Tuesday @
> 2pm
> > > > Eastern
> > > > > > >> Time.
> > > > > > >> > > > Maybe
> > > > > > >> > > > > > we
> > > > > > >> > > > > > > > can
> > > > > > >> > > > > > > > >> get
> > > > > > >> > > > > > > > >> >>>> INFRA
> > > > > > >> > > > > > > > >> >>>> > > help to make a google account so we
> > can
> > > > manage
> > > > > > >> > > better?
> > > > > > >> > > > > > > > >> >>>> > >
> > > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > > >> > > > > > > > >> >>>> > >
> > > > > > >> > > > > > > > >> >>>>
> > > > > > >> > > > > > > > >>
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >>
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > >> > > > > > > > >> >>>> > > in progress and related JIRA that
> are
> > > > > > >> interdependent
> > > > > > >> > > > and
> > > > > > >> > > > > > > common
> > > > > > >> > > > > > > > >> work.
> > > > > > >> > > > > > > > >> >>>> > >
> > > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > >> > > > > > > > >> >>>> > >
> > > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay
> > > > Kreps <
> > > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > >> > > > > > > > >> >>>> > >
> > > > > > >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that
> > will
> > > > also
> > > > > > >> record
> > > > > > >> > > > and
> > > > > > >> > > > > > make
> > > > > > >> > > > > > > > the
> > > > > > >> > > > > > > > >> >>>> sessions
> > > > > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM,
> > Jeff
> > > > > > Holoman
> > > > > > >> <
> > > > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex
> for
> > > > this
> > > > > > >> purpose.
> > > > > > >> > > We
> > > > > > >> > > > > > could
> > > > > > >> > > > > > > > >> record
> > > > > > >> > > > > > > > >> >>>> the
> > > > > > >> > > > > > > > >> >>>> > >> > sessions if there is interest and
> > > > publish
> > > > > > >> them
> > > > > > >> > > out.
> > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM,
> > Jay
> > > > > > Kreps <
> > > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > >> > > > > > > > >> >>>> wrote:
> > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > >> > > > > > > > >> >>>> > >> > > Let's try to get the technical
> > > > hang-ups
> > > > > > >> sorted
> > > > > > >> > > > out,
> > > > > > >> > > > > > > > though.
> > > > > > >> > > > > > > > >> I
> > > > > > >> > > > > > > > >> >>>> really
> > > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > > >> > > > > > > > >> >>>> > >> > > there is some benefit to live
> > > > discussion
> > > > > > vs
> > > > > > >> > > > > writing. I
> > > > > > >> > > > > > > am
> > > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > > >> > > > > > > > >> >>>> > >> if
> > > > > > >> > > > > > > > >> >>>> > >> > > we post instructions and give
> > > > ourselves a
> > > > > > >> few
> > > > > > >> > > > > attempts
> > > > > > >> > > > > > > we
> > > > > > >> > > > > > > > >> can
> > > > > > >> > > > > > > > >> >>>> get it
> > > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work
> > for
> > > > > > >> me...any
> > > > > > >> > > > > > objections?
> > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18
> AM,
> > > Joe
> > > > > > Stein
> > > > > > >> <
> > > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > > >> > > > > > > > >> >>>> >
> > > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe
> > like
> > > > every
> > > > > > >> > > Tuesday ~
> > > > > > >> > > > > 1pm
> > > > > > >> > > > > > > ET
> > > > > > >> > > > > > > > /
> > > > > > >> > > > > > > > >> 10am
> > > > > > >> > > > > > > > >> >>>> PT
> > > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout
> but
> > > > there
> > > > > > is
> > > > > > >> > > always
> > > > > > >> > > > > some
> > > > > > >> > > > > > > > >> issue or
> > > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > > >> > > > > > > > >> >>>> > >> > > > we know the apache irc
> channel
> > > > works.
> > > > > > We
> > > > > > >> can
> > > > > > >> > > > start
> > > > > > >> > > > > > > there
> > > > > > >> > > > > > > > >> and
> > > > > > >> > > > > > > > >> >>>> see how
> > > > > > >> > > > > > > > >> >>>> > >> it
> > > > > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts
> > too
> > > > and
> > > > > > >> > > associate
> > > > > > >> > > > to
> > > > > > >> > > > > > > > >> tickets if
> > > > > > >> > > > > > > > >> >>>> need be
> > > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > > >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10
> > AM,
> > > > Jay
> > > > > > >> Kreps <
> > > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a
> > > Google
> > > > > > >> Hangout to
> > > > > > >> > > > chat
> > > > > > >> > > > > > > about
> > > > > > >> > > > > > > > >> this.
> > > > > > >> > > > > > > > >> >>>> What
> > > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > > >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
> > > > > > further...I
> > > > > > >> > > > actually
> > > > > > >> > > > > > > think
> > > > > > >> > > > > > > > it
> > > > > > >> > > > > > > > >> >>>> would be
> > > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > > >> > > > > > > > >> >>>> > >> > > > > everyone spending a
> > reasonable
> > > > chunk
> > > > > > of
> > > > > > >> > > their
> > > > > > >> > > > > week
> > > > > > >> > > > > > > on
> > > > > > >> > > > > > > > >> Kafka
> > > > > > >> > > > > > > > >> >>>> stuff
> > > > > > >> > > > > > > > >> >>>> > >> to
> > > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I
> think
> > we
> > > > could
> > > > > > >> use
> > > > > > >> > > time
> > > > > > >> > > > > to
> > > > > > >> > > > > > > talk
> > > > > > >> > > > > > > > >> >>>> through
> > > > > > >> > > > > > > > >> >>>> > >> design
> > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on
> > top
> > > of
> > > > > > code
> > > > > > >> > > > reviews,
> > > > > > >> > > > > > talk
> > > > > > >> > > > > > > > >> through
> > > > > > >> > > > > > > > >> >>>> any
> > > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > We can make it publicly
> > > > available so
> > > > > > >> that
> > > > > > >> > > any
> > > > > > >> > > > > one
> > > > > > >> > > > > > > can
> > > > > > >> > > > > > > > >> follow
> > > > > > >> > > > > > > > >> >>>> along
> > > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this?
> > If
> > > so
> > > > > > I'll
> > > > > > >> try
> > > > > > >> > > to
> > > > > > >> > > > > set
> > > > > > >> > > > > > it
> > > > > > >> > > > > > > > up
> > > > > > >> > > > > > > > >> >>>> starting
> > > > > > >> > > > > > > > >> >>>> > >> next
> > > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at
> 3:57
> > > AM,
> > > > > > Andrii
> > > > > > >> > > > > Biletskyi
> > > > > > >> > > > > > <
> > > > > > >> > > > > > > > >> >>>> > >> > > > >
> andrii.biletskyi@stealth.ly>
> > > > wrote:
> > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page,
> > fixed
> > > /
> > > > > > >> aligned
> > > > > > >> > > > > document
> > > > > > >> > > > > > > > >> structure.
> > > > > > >> > > > > > > > >> >>>> Also I
> > > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
> > > > > > >> AdminClient so
> > > > > > >> > > we
> > > > > > >> > > > > have
> > > > > > >> > > > > > > > >> something
> > > > > > >> > > > > > > > >> >>>> to
> > > > > > >> > > > > > > > >> >>>> > >> start
> > > > > > >> > > > > > > > >> >>>> > >> > > > from
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > while
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > > >> > > > > > > > >> >>>> > >> >
> > > > > > >> > > > > > > > >> >>>> > >>
> > > > > > >> > > > > > > > >> >>>>
> > > > > > >> > > > > > > > >>
> > > > > > >> > > > > > > >
> > > > > > >> > > > > > >
> > > > > > >> > > > > >
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >>
> > > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at
> > 9:01
> > > > PM,
> > > > > > >> Andrii
> > > > > > >> > > > > > Biletskyi
> > > > > > >> > > > > > > <
> > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > andrii.biletskyi@stealth.ly>
> > > > > > wrote:
> > > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you
> > are
> > > > right,
> > > > > > >> in
> > > > > > >> > > most
> > > > > > >> > > > > > cases
> > > > > > >> > > > > > > > >> client
> > > > > > >> > > > > > > > >> >>>> will
> > > > > > >> > > > > > > > >> >>>> > >> have
> > > > > > >> > > > > > > > >> >>>> > >> > > > enough
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > context to show
> > descriptive
> > > > error
> > > > > > >> > > message.
> > > > > > >> > > > > My
> > > > > > >> > > > > > > > >> concern is
> > > > > > >> > > > > > > > >> >>>> that
> > > > > > >> > > > > > > > >> >>>> > >> we
> > > > > > >> > > > > > > > >> >>>> > >> > > will
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > have
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > to
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error
> > codes
> > > > for
> > > > > > >> each
> > > > > > >> > > > > possible
> > > > > > >> > > > > > > > >> error. Of
> > > > > > >> > > > > > > > >> >>>> course,
> > > > > > >> > > > > > > > >> >>>> > >> > we
> > > > > > >> > > > > > > > >> >>>> > >> > > > > could
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > reuse
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
> > > > > > >> > > > > > > UknownTopicOrPartitionCode,
> > > > > > >> > > > > > > > >> but we
> > > > > > >> > > > > > > > >> >>>> will
> > > > > > >> > > > > > > > >> >>>> > >> > also
> > > > > > >> > > > > > > > >> >>>> > >> > > > need
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > to
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> > > > > > >> TopicAlreadyExistsCode,
> > > > > > >> > > > > > > > >> >>>> TopicConfigInvalid (both
> > > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > > >> > > > > > > > >> >>>> > >> > > > > topic
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > name and config, and
> > > probably
> > > > > > user
> > > > > > >> would
> > > > > > >> > > > > like
> > > > > > >> > > > > > to
> > > > > > >> > > > > > > > >> know
> > > > > > >> > > > > > > > >> >>>> what
> > > > > > >> > > > > > > > >> >>>> > >> > exactly
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his
> config),
> > > > > > >> > > > > > > InvalidReplicaAssignment,
> > > > > > >> > > > > > > > >> >>>> > >> InternalError
> > > > > > >> > > > > > > > >> >>>> > >> > > > (e.g.
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
> > > > > > TopicCommand,
> > > > > > >> we
> > > > > > >> > > will
> > > > > > >> > > > > > also
> > > > > > >> > > > > > > > >> need to
> > > > > > >> > > > > > > > >> >>>> add
> > > > > > >> > > > > > > > >> >>>> > >> > similar
> > > > > > >> > > > > > > > >> >>>> > >> > > > > stuff
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > for
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> > > > > > >> PreferredReplica. So
> > > > > > >> > > > > we'll
> > > > > > >> > > > > > > end
> > > > > > >> > > > > > > > >> up
> > > > > > >> > > > > > > > >> >>>> with a
> > > > > > >> > > > > > > > >> >>>> > >> > large
> > > > > > >> > > > > > > > >> >>>> > >> > > > list
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > of
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only
> in
> > > > Admin
> > > > > > >> > > protocol.
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I
> agree
> > > my
> > > > > > >> proposal is
> > > > > > >> > > > not
> > > > > > >> > > > > > > > >> consistent
> > > > > > >> > > > > > > > >> >>>> with
> > > > > > >> > > > > > > > >> >>>> > >> > other
> > > > > > >> > > > > > > > >> >>>> > >> > > > > cases.
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find
> better
> > > > solution
> > > > > > >> or
> > > > > > >> > > > > something
> > > > > > >> > > > > > > > >> >>>> in-between.
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I
> think
> > it
> > > > is a
> > > > > > >> great
> > > > > > >> > > > idea.
> > > > > > >> > > > > > > This
> > > > > > >> > > > > > > > >> way we
> > > > > > >> > > > > > > > >> >>>> can
> > > > > > >> > > > > > > > >> >>>> > >> move
> > > > > > >> > > > > > > > >> >>>> > >> > > on
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > faster.
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on
> > > > date/time
> > > > > > so
> > > > > > >> > > people
> > > > > > >> > > > > can
> > > > > > >> > > > > > > > join.
> > > > > > >> > > > > > > > >> >>>> Will work
> > > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > > >> > > > > > > > >> >>>> > >> > > me
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > and
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > next week almost
> anytime
> > if
> > > > > > agreed
> > > > > > >> in
> > > > > > >> > > > > advance.
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at
> > > 7:09
> > > > PM,
> > > > > > >> Jay
> > > > > > >> > > > Kreps <
> > > > > > >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > > > >> > > > > > > > >> >>>> > >> > > > > wrote:
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do
> good
> > > > error
> > > > > > >> handling
> > > > > > >> > > > > > without
> > > > > > >> > > > > > > > >> needing
> > > > > > >> > > > > > > > >> >>>> custom
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > server-side
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e.
> generally
> > > the
> > > > > > >> client has
> > > > > > >> > > > the
> > > > > > >> > > > > > > > >> context to
> > > > > > >> > > > > > > > >> >>>> know
> > > > > > >> > > > > > > > >> >>>> > >> that
> > > > > > >> > > > > > > > >> >>>> > >> > > if
> > > > > > >> > > > > > > > >> >>>> > >> > > > it
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > got
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> an error that the
> topic
> > > > doesn't
> > > > > > >> exist
> > > > > > >> > > to
> > > > > > >> > > > > say
> > > > > > >> > > > > > > > >> "Topic X
> > > > > > >> > > > > > > > >> >>>> doesn't
> > > > > > >> > > > > > > > >> >>>> > >> > > exist"
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> rather
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14"
> (or
> > > > > > >> whatever).
> > > > > > >> > > Maybe
> > > > > > >> > > > > > there
> > > > > > >> > > > > > > > are
> > > > > > >> > > > > > > > >> >>>> specific
> > > > > > >> > > > > > > > >> >>>> > >> > cases
> > > > > > >> > > > > > > > >> >>>> > >> > > > > where
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we
> want
> > > to
> > > > add
> > > > > > >> > > > server-side
> > > > > > >> > > > > > > error
> > > > > > >> > > > > > > > >> >>>> messages we
> > > > > > >> > > > > > > > >> >>>> > >> > > really
> > > > > > >> > > > > > > > >> >>>> > >> > > > > do
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a
> > > > consistent
> > > > > > >> way
> > > > > > >> > > > across
> > > > > > >> > > > > > the
> > > > > > >> > > > > > > > >> protocol.
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch
> of
> > > open
> > > > > > >> questions
> > > > > > >> > > > here
> > > > > > >> > > > > > > from
> > > > > > >> > > > > > > > my
> > > > > > >> > > > > > > > >> >>>> previous
> > > > > > >> > > > > > > > >> >>>> > >> > > list. I
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > will
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next
> few
> > > > days for
> > > > > > >> Strata
> > > > > > >> > > > > > though.
> > > > > > >> > > > > > > > >> Maybe
> > > > > > >> > > > > > > > >> >>>> we could
> > > > > > >> > > > > > > > >> >>>> > >> > do
> > > > > > >> > > > > > > > >> >>>> > >> > > a
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > Google
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any
> open
> > > > issues
> > > > > > >> some
> > > > > > >> > > time
> > > > > > >> > > > > > > towards
> > > > > > >> > > > > > > > >> the
> > > > > > >> > > > > > > > >> >>>> end of
> > > > > > >> > > > > > > > >> >>>> > >> > next
> > > > > > >> > > > > > > > >> >>>> > >> > > > week
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > for
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in
> > this
> > > > > > ticket?
> > > > > > >> I
> > > > > > >> > > have
> > > > > > >> > > > a
> > > > > > >> > > > > > > > feeling
> > > > > > >> > > > > > > > >> that
> > > > > > >> > > > > > > > >> >>>> might
> > > > > > >> > > > > > > > >> >>>> > >> > > > progress
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster
> > > than
> > > > > > >> email--I
> > > > > > >> > > > think
> > > > > > >> > > > > we
> > > > > > >> > > > > > > > >> could talk
> > > > > > >> > > > > > > > >> >>>> > >> through
> > > > > > >> > > > > > > > >> >>>> > >> > > > those
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up
> > fairly
> > > > > > >> quickly...
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015
> at
> > > > 7:27 AM,
> > > > > > >> Andrii
> > > > > > >> > > > > > > > Biletskyi <
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > andrii.biletskyi@stealth.ly
> > > > >
> > > > > > >> wrote:
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to
> address
> > > > some of
> > > > > > >> the
> > > > > > >> > > > issues
> > > > > > >> > > > > > > which
> > > > > > >> > > > > > > > >> were
> > > > > > >> > > > > > > > >> >>>> > >> mentioned
> > > > > > >> > > > > > > > >> >>>> > >> > > > > earlier
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> about
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format.
> > One
> > > of
> > > > > > >> those was
> > > > > > >> > > > > about
> > > > > > >> > > > > > > > >> batching
> > > > > > >> > > > > > > > >> >>>> > >> > operations.
> > > > > > >> > > > > > > > >> >>>> > >> > > > What
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > if
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> we
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand
> > > > approach
> > > > > > >> and let
> > > > > > >> > > > > people
> > > > > > >> > > > > > > > >> specify
> > > > > > >> > > > > > > > >> >>>> > >> topic-name
> > > > > > >> > > > > > > > >> >>>> > >> > > by
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover
> most
> > of
> > > > the
> > > > > > use
> > > > > > >> > > cases?
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what
> > > > information
> > > > > > >> should
> > > > > > >> > > we
> > > > > > >> > > > > > > > generally
> > > > > > >> > > > > > > > >> >>>> provide in
> > > > > > >> > > > > > > > >> >>>> > >> > > Admin
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin
> > > > commands
> > > > > > >> don't
> > > > > > >> > > > imply
> > > > > > >> > > > > > > they
> > > > > > >> > > > > > > > >> will
> > > > > > >> > > > > > > > >> >>>> be used
> > > > > > >> > > > > > > > >> >>>> > >> > only
> > > > > > >> > > > > > > > >> >>>> > >> > > > in
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > CLI
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > but,
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI
> > is a
> > > > very
> > > > > > >> > > important
> > > > > > >> > > > > > > client
> > > > > > >> > > > > > > > >> of this
> > > > > > >> > > > > > > > >> >>>> > >> > feature.
> > > > > > >> > > > > > > > >> >>>> > >> > > In
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > case,
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we
> > would
> > > > like
> > > > > > to
> > > > > > >> > > provide
> > > > > > >> > > > > > users
> > > > > > >> > > > > > > > >> with
> > > > > > >> > > > > > > > >> >>>> rich
> > > > > > >> > > > > > > > >> >>>> > >> > > experience
> > > > > > >> > > > > > > > >> >>>> > >> > > > > in
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> terms
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > of
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > getting results /
> > errors
> > > > of
> > > > > > the
> > > > > > >> > > > executed
> > > > > > >> > > > > > > > >> commands.
> > > > > > >> > > > > > > > >> >>>> Usually
> > > > > > >> > > > > > > > >> >>>> > >> we
> > > > > > >> > > > > > > > >> >>>> > >> > > > supply
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> with
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses only
> > > errorCode,
> > > > > > which
> > > > > > >> looks
> > > > > > >> > > > > very
> > > > > > >> > > > > > > > >> limiting,
> > > > > > >> > > > > > > > >> >>>> in case
> > > > > > >> > > > > > > > >> >>>> > >> > of
> > > > > > >> > > > > > > > >> >>>> > >> > > > CLI
> > > > > > >> > > > > > > > >> >>>> > >> > > > > we
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> may
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human
> > > > readable
> > > > > > >> error
> > > > > > >> > > > > > > description.
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into
> > account
> > > > > > >> previous item
> > > > > > >> > > > > about
> > > > > > >> > > > > > > > >> batching,
> > > > > > >> > > > > > > > >> >>>> what
> > > > > > >> > > > > > > > >> >>>> > >> do
> > > > > > >> > > > > > > > >> >>>> > >> > > you
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > think
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > about
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't
> > > support
> > > > > > >> regexp)
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest
> =>
> > > > > > TopicName
> > > > > > >> > > > > Partitions
> > > > > > >> > > > > > > > >> Replicas
> > > > > > >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse
> =>
> > > > > > ErrorCode
> > > > > > >> > > > > > > > ErrorDescription
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription
> =>
> > > > string
> > > > > > >> (empty
> > > > > > >> > > if
> > > > > > >> > > > > > > > >> successful)
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
> > > > > > >> TopicNameRegexp
> > > > > > >> > > >
> > > > > > >
> > > > > > > ...
> > > > > > >
> > > > > > > [Message clipped]
> > > > > >
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
Andrii,

I think we agreed on the following.

(a) Admin requests can be sent to and handled by any broker.
(b) Admin requests are processed asynchronously, at least for now. That is,
when the client gets a response, it just means that the request is
initiated, but not necessarily completed. Then, it's up to the client to
issue another request to check the status for completion.

To support (a), we were thinking of doing request forwarding to the
controller (utilizing KAFKA-1912). I am making an alternative proposal.
Basically, the broker can just write to ZooKeeper to inform the controller
about the request. For example, to handle partitionReassignment, the broker
will just write the requested partitions to /admin/reassign_partitions
(like what AdminUtils currently does) and then send a response to the
client. This shouldn't take long and the implementation will be simpler
than forwarding the requests to the controller through RPC.

Thanks,

Jun


On Wed, Mar 18, 2015 at 3:03 PM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Jun,
>
> I might be wrong but didn't we agree we will let any broker from the
> cluster handle *long-running* admin requests (at this time preferredReplica
> and
> reassignPartitions), via zk admin path. Thus CreateTopics etc should be
> sent
> only to the controller.
>
> Thanks,
> Andrii Biletskyi
>
> On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Joel, Andril,
> >
> > I think we agreed that those admin requests can be issued to any broker.
> > Because of that, there doesn't seem to be a strong need to know the
> > controller. So, perhaps we can proceed by not making any change to the
> > format of TMR right now. When we start using create topic request in the
> > producer, we will need a new version of TMR that doesn't trigger auto
> topic
> > creation. But that can be done later.
> >
> > As a first cut implementation, I think the broker can just write to ZK
> > directly for
> > createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> > requests, instead of forwarding them to the controller. This will
> simplify
> > the implementation on the broker side.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <jj...@gmail.com>
> wrote:
> >
> > > (Thanks Andrii for the summary)
> > >
> > > For (1) yes we will circle back on that shortly after syncing up in
> > > person. I think it is close to getting committed although development
> > > for KAFKA-1927 can probably begin without it.
> > >
> > > There is one more item we covered at the hangout. i.e., whether we
> > > want to add the coordinator to the topic metadata response or provide
> > > a clearer ClusterMetadataRequest.
> > >
> > > There are two reasons I think we should try and avoid adding the
> > > field:
> > > - It is irrelevant to topic metadata
> > > - If we finally do request rerouting in Kafka then the field would add
> > >   little to no value. (It still helps to have a separate
> > >   ClusterMetadataRequest to query for cluster-wide information such as
> > >   'which broker is the controller?' as Joe mentioned.)
> > >
> > > I think it would be cleaner to have an explicit ClusterMetadataRequest
> > > that you can send to any broker in order to obtain the controller (and
> > > in the future possibly other cluster-wide information). I think the
> > > main argument against doing this and instead adding it to the topic
> > > metadata response was convenience - i.e., you don't have to discover
> > > the controller in advance. However, I don't see much actual
> > > benefit/convenience in this and in fact think it is a non-issue. Let
> > > me know if I'm overlooking something here.
> > >
> > > As an example, say we need to initiate partition reassignment by
> > > issuing the new ReassignPartitionsRequest to the controller (assume we
> > > already have the desired manual partition assignment).  If we are to
> > > augment topic metadata response then the flow be something like this :
> > >
> > > - Issue topic metadata request to any broker (and discover the
> > >   controller
> > > - Connect to controller if required (i.e., if the broker above !=
> > >   controller)
> > > - Issue the partition reassignment request to the controller.
> > >
> > > With an explicit cluster metadata request it would be:
> > > - Issue cluster metadata request to any broker
> > > - Connect to controller if required (i.e., if the broker above !=
> > >   controller)
> > > - Issue the partition reassignment request
> > >
> > > So it seems to add little practical value and bloats topic metadata
> > > response with an irrelevant detail.
> > >
> > > The other angle to this is the following - is it a matter of naming?
> > > Should we just rename topic metadata request/response to just
> > > MetadataRequest/Response and add cluster metadata to it? By that same
> > > token should we also allow querying for the consumer coordinator (and
> > > in future transaction coordinator) as well? This leads to a bloated
> > > request which isn't very appealing and altogether confusing.
> > >
> > > Thanks,
> > >
> > > Joel
> > >
> > > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > > > Andri,
> > > >
> > > > Thanks for the summary.
> > > >
> > > > 1. I just realized that in order to start working on KAFKA-1927, we
> > will
> > > > need to merge the changes to OffsetCommitRequest (from 0.8.2) to
> trunk.
> > > > This is planned to be done as part of KAFKA-1634. So, we will need
> > > Guozhang
> > > > and Joel's help to wrap this up.
> > > >
> > > > 2. Thinking about this a bit more, if the semantic of those "write"
> > > > requests is async (i.e., after the client gets a response, it just
> > means
> > > > that the operation is initiated, but not necessarily completed), we
> > don't
> > > > really need to forward the requests to the controller. Instead, the
> > > > receiving broker can just write the operation to ZK as the admin
> > command
> > > > line tool previously does. This will simplify the implementation.
> > > >
> > > > 8. There is another implementation detail for describe topic.
> Ideally,
> > we
> > > > want to read the topic config from the broker cache, instead of
> > > ZooKeeper.
> > > > Currently, every broker reads the topic-level config for all topics.
> > > > However, it ignores those for topics not hosted on itself. So, we may
> > > need
> > > > to change TopicConfigManager a bit so that it caches the configs for
> > all
> > > > topics.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Guys,
> > > > >
> > > > > Thanks for a great discussion!
> > > > > Here are the actions points:
> > > > >
> > > > > 1. Q: Get rid of all scala requests objects, use java protocol
> > > definitions.
> > > > >     A: Gwen kindly took that (KAFKA-1927). It's important to speed
> up
> > > > > review procedure
> > > > >          there since this ticket blocks other important changes.
> > > > >
> > > > > 2. Q: Generic re-reroute facility vs client maintaining cluster
> > state.
> > > > >     A: Jay has added pseudo code to KAFKA-1912 - need to consider
> > > whether
> > > > > this will be
> > > > >         easy to implement as a server-side feature (comments are
> > > > > welcomed!).
> > > > >
> > > > > 3. Q: Controller field in wire protocol.
> > > > >     A: This might be useful for clients, add this to
> > > TopicMetadataResponse
> > > > > (already in KIP).
> > > > >
> > > > > 4. Q: Decoupling topic creation from TMR.
> > > > >     A: I will add proposed by Jun solution (using clientId for
> that)
> > > to the
> > > > > KIP.
> > > > >
> > > > > 5. Q: Bumping new versions of TMR vs grabbing all protocol changes
> in
> > > one
> > > > > version.
> > > > >     A: It was decided to try to gather all changes to protocol
> > (before
> > > > > release).
> > > > >         In case of TMR it worth checking: KAFKA-2020 and KIP-13
> > > (quotas)
> > > > >
> > > > > 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
> > > > >     A: Use jackson for that, /tools project is a separate jar so
> > > shouldn't
> > > > > be a big deal.
> > > > >
> > > > > 7.  Q: VerifyReassingPartitions vs generic status check command.
> > > > >      A: For long-running requests like reassign partitions
> *progress*
> > > check
> > > > > request is useful,
> > > > >          it makes sense to introduce it.
> > > > >
> > > > >  Please add, correct me if I missed something.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Joel,
> > > > > >
> > > > > > You are right, I removed ClusterMetadata because we have
> partially
> > > > > > what we need in TopicMetadata. Also, as Jay pointed out earlier,
> we
> > > > > > would like to have "orthogonal" API, but at the same time we need
> > > > > > to be backward compatible.
> > > > > >
> > > > > > But I like your idea and even have some other arguments for this
> > > option:
> > > > > > There is also DescribeTopicRequest which was proposed in this
> KIP,
> > > > > > it returns topic configs, partitions, replication factor plus
> > > partition
> > > > > > ISR, ASR,
> > > > > > leader replica. The later part is really already there in
> > > > > > TopicMetadataRequest.
> > > > > > So again we'll have to add stuff to TMR, not to duplicate some
> info
> > > in
> > > > > > newly added requests. However, this way we'll end up with
> "monster"
> > > > > > request which returns cluster metadata, topic replication and
> > config
> > > info
> > > > > > plus partition replication data. Seems logical to split TMR to
> > > > > > - ClusterMetadata (brokers + controller, maybe smth else)
> > > > > > - TopicMetadata (topic info + partition details)
> > > > > > But since current TMR is involved in lots of places (including
> > > network
> > > > > > client,
> > > > > > as I understand) this might be very serious change and it
> probably
> > > makes
> > > > > > sense to stick with current approach.
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > >
> > > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jjkoshy.w@gmail.com
> >
> > > wrote:
> > > > > >
> > > > > >> I may be missing some context but hopefully this will also be
> > > covered
> > > > > >> today: I thought the earlier proposal where there was an
> explicit
> > > > > >> ClusterMetadata request was clearer and explicit. During the
> > course
> > > of
> > > > > >> this thread I think the conclusion was that the main need was
> for
> > > > > >> controller information and that can be rolled into the topic
> > > metadata
> > > > > >> response but that seems a bit irrelevant to topic metadata.
> FWIW I
> > > > > >> think the full broker-list is also irrelevant to topic metadata,
> > but
> > > > > >> it is already there and in use. I think there is still room for
> an
> > > > > >> explicit ClusterMetadata request since there may be other
> > > > > >> cluster-level information that we may want to add over time (and
> > > that
> > > > > >> have nothing to do with topic metadata).
> > > > > >>
> > > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi
> wrote:
> > > > > >> > Jun,
> > > > > >> >
> > > > > >> > 101. Okay, if you say that such use case is important. I also
> > > think
> > > > > >> > using clientId for these purposes is fine - if we already have
> > > this
> > > > > >> field
> > > > > >> > as part of all Wire protocol messages, why not use that.
> > > > > >> > I will update KIP-4 page if nobody has other ideas (which may
> > > come up
> > > > > >> > during the call today).
> > > > > >> >
> > > > > >> > 102.1 Agree, I'll update the KIP accordingly. I think we can
> add
> > > new,
> > > > > >> > fine-grained error codes if some error code received in
> specific
> > > case
> > > > > >> > won't give enough context to return a descriptive error
> message
> > > for
> > > > > >> user.
> > > > > >> >
> > > > > >> > Look forward to discussing all outstanding issues in detail
> > today
> > > > > during
> > > > > >> > the call.
> > > > > >> >
> > > > > >> > Thanks,
> > > > > >> > Andrii Biletskyi
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > > > >> >
> > > > > >> > > 101. There may be a use case where you only want the topics
> to
> > > be
> > > > > >> created
> > > > > >> > > manually by admins. Currently, you can do that by disabling
> > auto
> > > > > topic
> > > > > >> > > creation and issue topic creation from the TopicCommand. If
> we
> > > > > >> disable auto
> > > > > >> > > topic creation completely on the broker and don't have a way
> > to
> > > > > >> distinguish
> > > > > >> > > between topic creation requests from the regular clients and
> > the
> > > > > >> admin, we
> > > > > >> > > can't support manual topic creation any more. I was thinking
> > > that
> > > > > >> another
> > > > > >> > > way of distinguishing the clients making the topic creation
> > > requests
> > > > > >> is
> > > > > >> > > using clientId. For example, the admin tool can set it to
> > > something
> > > > > >> like
> > > > > >> > > admin and the broker can treat that clientId specially.
> > > > > >> > >
> > > > > >> > > Also, there is a related discussion in KAFKA-2020.
> Currently,
> > > we do
> > > > > >> the
> > > > > >> > > following in TopicMetadataResponse:
> > > > > >> > >
> > > > > >> > > 1. If leader is not available, we set the partition level
> > error
> > > code
> > > > > >> to
> > > > > >> > > LeaderNotAvailable.
> > > > > >> > > 2. If a non-leader replica is not available, we take that
> > > replica
> > > > > out
> > > > > >> of
> > > > > >> > > the assigned replica list and isr in the response. As an
> > > indication
> > > > > >> for
> > > > > >> > > doing that, we set the partition level error code to
> > > > > >> ReplicaNotAvailable.
> > > > > >> > >
> > > > > >> > > This has a few problems. First, ReplicaNotAvailable probably
> > > > > >> shouldn't be
> > > > > >> > > an error, at least for the normal producer/consumer clients
> > that
> > > > > just
> > > > > >> want
> > > > > >> > > to find out the leader. Second, it can happen that both the
> > > leader
> > > > > and
> > > > > >> > > another replica are not available at the same time. There is
> > no
> > > > > error
> > > > > >> code
> > > > > >> > > to indicate both. Third, even if a replica is not available,
> > > it's
> > > > > >> still
> > > > > >> > > useful to return its replica id since some clients (e.g.
> admin
> > > tool)
> > > > > >> may
> > > > > >> > > still make use of it.
> > > > > >> > >
> > > > > >> > > One way to address this issue is to always return the
> replica
> > > id for
> > > > > >> > > leader, assigned replicas, and isr regardless of whether the
> > > > > >> corresponding
> > > > > >> > > broker is live or not. Since we also return the list of live
> > > > > brokers,
> > > > > >> the
> > > > > >> > > client can figure out whether a leader or a replica is live
> or
> > > not
> > > > > >> and act
> > > > > >> > > accordingly. This way, we don't need to set the partition
> > level
> > > > > error
> > > > > >> code
> > > > > >> > > when the leader or a replica is not available. This doesn't
> > > change
> > > > > >> the wire
> > > > > >> > > protocol, but does change the semantics. Since we are
> evolving
> > > the
> > > > > >> protocol
> > > > > >> > > of TopicMetadataRequest here, we can potentially piggyback
> the
> > > > > change.
> > > > > >> > >
> > > > > >> > > 102.1 For those types of errors due to invalid input,
> > shouldn't
> > > we
> > > > > >> just
> > > > > >> > > guard it at parameter validation time and throw
> > > > > >> InvalidArgumentException
> > > > > >> > > without even sending the request to the broker?
> > > > > >> > >
> > > > > >> > > Thanks,
> > > > > >> > >
> > > > > >> > > Jun
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >> > >
> > > > > >> > > > Jun,
> > > > > >> > > >
> > > > > >> > > > Answering your questions:
> > > > > >> > > >
> > > > > >> > > > 101. If I understand you correctly, you are saying future
> > > producer
> > > > > >> > > versions
> > > > > >> > > > (which
> > > > > >> > > > will be ported to TMR_V1) won't be able to automatically
> > > create
> > > > > >> topic (if
> > > > > >> > > > we
> > > > > >> > > > unconditionally remove topic creation from there). But we
> > > need to
> > > > > >> this
> > > > > >> > > > preserve logic.
> > > > > >> > > > Ok, about your proposal: I'm not a big fan too, when it
> > comes
> > > to
> > > > > >> > > > differentiating
> > > > > >> > > > clients directly in protocol schema. And also I'm not
> sure I
> > > > > >> understand
> > > > > >> > > at
> > > > > >> > > > all why
> > > > > >> > > > auto.create.topics.enable is a server side configuration.
> > Can
> > > we
> > > > > >> > > deprecate
> > > > > >> > > > this setting
> > > > > >> > > > in future versions, add this setting to producer and based
> > on
> > > that
> > > > > >> upon
> > > > > >> > > > receiving
> > > > > >> > > > UnknownTopic create topic explicitly by a separate
> producer
> > > call
> > > > > via
> > > > > >> > > > adminClient?
> > > > > >> > > >
> > > > > >> > > > 102.1. Hm, yes. It's because we want to support batching
> and
> > > at
> > > > > the
> > > > > >> same
> > > > > >> > > > time we
> > > > > >> > > > want to give descriptive error messages for clients. Since
> > > > > >> AdminClient
> > > > > >> > > > holds the context
> > > > > >> > > > to construct such messages (e.g. AdminClient layer can
> know
> > > that
> > > > > >> > > > InvalidArgumentsCode
> > > > > >> > > > means two cases: either invalid number - e.g. -1; or
> > > > > >> replication-factor
> > > > > >> > > was
> > > > > >> > > > provided while
> > > > > >> > > > partitions argument wasn't) - I wrapped responses in
> > > Exceptions.
> > > > > >> But I'm
> > > > > >> > > > open to any
> > > > > >> > > > other ideas, this was just initial version.
> > > > > >> > > > 102.2. Yes, I agree. I'll change that to probably some
> other
> > > dto.
> > > > > >> > > >
> > > > > >> > > > Thanks,
> > > > > >> > > > Andrii Biletskyi
> > > > > >> > > >
> > > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <
> jun@confluent.io>
> > > > > wrote:
> > > > > >> > > >
> > > > > >> > > > > Andrii,
> > > > > >> > > > >
> > > > > >> > > > > 101. That's what I was thinking too, but it may not be
> > that
> > > > > >> simple. In
> > > > > >> > > > > TopicMetadataRequest_V1,
> > > > > >> > > > > we can let it not trigger auto topic creation. Then, in
> > the
> > > > > >> producer
> > > > > >> > > > side,
> > > > > >> > > > > if it gets an UnknownTopicException, it can explicitly
> > > issue a
> > > > > >> > > > > createTopicRequest for auto topic creation. On the
> > consumer
> > > > > side,
> > > > > >> it
> > > > > >> > > will
> > > > > >> > > > > never issue createTopicRequest. This works when auto
> topic
> > > > > >> creation is
> > > > > >> > > > > enabled on the broker side. However, I am not sure how
> > > things
> > > > > >> will work
> > > > > >> > > > > when auto topic creation is disabled on the broker side.
> > In
> > > this
> > > > > >> case,
> > > > > >> > > we
> > > > > >> > > > > want to have a way to manually create a topic,
> potentially
> > > > > through
> > > > > >> > > admin
> > > > > >> > > > > commands. However, then we need a way to distinguish
> > > > > >> createTopicRequest
> > > > > >> > > > > issued from the producer clients and the admin tools.
> May
> > > be we
> > > > > >> can
> > > > > >> > > add a
> > > > > >> > > > > new field in createTopicRequest and set it differently
> in
> > > the
> > > > > >> producer
> > > > > >> > > > > client and the admin client. However, I am not sure if
> > > that's
> > > > > the
> > > > > >> best
> > > > > >> > > > > approach.
> > > > > >> > > > >
> > > > > >> > > > > 2. Yes, refactoring existing requests is a non-trivial
> > > amount of
> > > > > >> work.
> > > > > >> > > I
> > > > > >> > > > > posted some comments in KAFKA-1927. We will probably
> have
> > > to fix
> > > > > >> > > > KAFKA-1927
> > > > > >> > > > > first, before adding the new logic in KAFKA-1694.
> > > Otherwise, the
> > > > > >> > > changes
> > > > > >> > > > > will be too big.
> > > > > >> > > > >
> > > > > >> > > > > 102. About the AdminClient:
> > > > > >> > > > > 102.1. It's a bit weird that we return exception in the
> > > api. It
> > > > > >> seems
> > > > > >> > > > that
> > > > > >> > > > > we should either return error code or throw an exception
> > > when
> > > > > >> getting
> > > > > >> > > the
> > > > > >> > > > > response state.
> > > > > >> > > > > 102.2. We probably shouldn't explicitly use the request
> > > object
> > > > > in
> > > > > >> the
> > > > > >> > > > api.
> > > > > >> > > > > Not every request evolution requires an api change.
> > > > > >> > > > >
> > > > > >> > > > > Thanks,
> > > > > >> > > > >
> > > > > >> > > > > Jun
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >> > > > >
> > > > > >> > > > > > Jun,
> > > > > >> > > > > >
> > > > > >> > > > > > Thanks for you comments. Answers inline:
> > > > > >> > > > > >
> > > > > >> > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > >> > > > > > > and PartitionsSerialized that are represented as a
> > > string,
> > > > > but
> > > > > >> > > > contain
> > > > > >> > > > > > > composite structures in json. Could we flatten them
> > out
> > > > > >> directly in
> > > > > >> > > > the
> > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > Yes, now with Admin Client this looks a bit weird. My
> > > initial
> > > > > >> > > > motivation
> > > > > >> > > > > > was:
> > > > > >> > > > > > ReassignPartitionCommand accepts input in json, we
> want
> > to
> > > > > >> remain
> > > > > >> > > > tools'
> > > > > >> > > > > > interfaces unchanged, where possible.
> > > > > >> > > > > > If we port it to deserialized format, in CLI (/tools
> > > project)
> > > > > >> we will
> > > > > >> > > > > have
> > > > > >> > > > > > to add some
> > > > > >> > > > > > json library since /tools is written in java and we'll
> > > need to
> > > > > >> > > > > deserialize
> > > > > >> > > > > > json file
> > > > > >> > > > > > provided by a user. Can we quickly agree on what this
> > > library
> > > > > >> should
> > > > > >> > > be
> > > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > > >> > > > > >
> > > > > >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto
> > topic
> > > > > >> creation?
> > > > > >> > > > This
> > > > > >> > > > > > > will be a bit weird now that we have a separate
> topic
> > > > > >> creation api.
> > > > > >> > > > > Have
> > > > > >> > > > > > > you thought about how the new createTopicRequest and
> > > > > >> > > > > TopicMetadataRequest
> > > > > >> > > > > > > v1 will be used in the producer/consumer client, in
> > > addition
> > > > > >> to
> > > > > >> > > admin
> > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > >> TopicMetadataRequest
> > > > > >> > > from
> > > > > >> > > > > the
> > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > I agree, this strange logic should be fixed. I'm not
> > > confident
> > > > > >> in
> > > > > >> > > this
> > > > > >> > > > > > Kafka part so
> > > > > >> > > > > > correct me if I'm wrong, but it doesn't look like a
> hard
> > > thing
> > > > > >> to
> > > > > >> > > do, I
> > > > > >> > > > > > think we can
> > > > > >> > > > > > leverage AdminClient for that in Producer and
> > > unconditionally
> > > > > >> remove
> > > > > >> > > > > topic
> > > > > >> > > > > > creation from the TopicMetadataRequest_V1.
> > > > > >> > > > > >
> > > > > >> > > > > > 2. I think Jay meant getting rid of scala classes
> > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > >> HeartbeatResponseAndHeader. We
> > > > > >> > > did
> > > > > >> > > > > > that
> > > > > >> > > > > > > as a stop-gap thing when adding the new requests for
> > the
> > > > > >> consumers.
> > > > > >> > > > > > > However, the long term plan is to get rid of all
> those
> > > and
> > > > > >> just
> > > > > >> > > reuse
> > > > > >> > > > > the
> > > > > >> > > > > > > java request/response in the client. Since this KIP
> > > proposes
> > > > > >> to
> > > > > >> > > add a
> > > > > >> > > > > > > significant number of new requests, perhaps we
> should
> > > bite
> > > > > the
> > > > > >> > > bullet
> > > > > >> > > > > to
> > > > > >> > > > > > > clean up the existing scala requests first before
> > > adding new
> > > > > >> ones?
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > Yes, looks like I misunderstood the point of
> > > > > >> ...RequestAndHeader.
> > > > > >> > > > Okay, I
> > > > > >> > > > > > will
> > > > > >> > > > > > rework that. The only thing is that I don't see any
> > > example
> > > > > how
> > > > > >> it
> > > > > >> > > was
> > > > > >> > > > > done
> > > > > >> > > > > > for at
> > > > > >> > > > > > least one existing protocol message. Thus, as I
> > > understand, I
> > > > > >> have to
> > > > > >> > > > > think
> > > > > >> > > > > > how we
> > > > > >> > > > > > are going to do it.
> > > > > >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
> > > > > reasonable,
> > > > > >> but
> > > > > >> > > if
> > > > > >> > > > > it's
> > > > > >> > > > > > an *obligatory*
> > > > > >> > > > > > requirement to have Admin KIP done, I'm afraid this
> can
> > > be a
> > > > > >> serious
> > > > > >> > > > > > blocker for us.
> > > > > >> > > > > > There are 13 protocol messages and all that would
> > require
> > > not
> > > > > >> only
> > > > > >> > > unit
> > > > > >> > > > > > tests but quite
> > > > > >> > > > > > intensive manual testing, no? I'm afraid I'm not the
> > > right guy
> > > > > >> to
> > > > > >> > > cover
> > > > > >> > > > > > pretty much all
> > > > > >> > > > > > Kafka core internals :). Let me know your thoughts on
> > this
> > > > > >> item. Btw
> > > > > >> > > > > there
> > > > > >> > > > > > is a ticket to
> > > > > >> > > > > > follow-up this issue (
> > > > > >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > > >> > > > ).
> > > > > >> > > > > >
> > > > > >> > > > > > Thanks,
> > > > > >> > > > > > Andrii Biletskyi
> > > > > >> > > > > >
> > > > > >> > > > > >
> > > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <
> > > jun@confluent.io>
> > > > > >> wrote:
> > > > > >> > > > > >
> > > > > >> > > > > > > Andrii,
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > > A few more comments.
> > > > > >> > > > > > >
> > > > > >> > > > > > > 100. There are a few fields such as
> ReplicaAssignment,
> > > > > >> > > > > > > ReassignPartitionRequest,
> > > > > >> > > > > > > and PartitionsSerialized that are represented as a
> > > string,
> > > > > but
> > > > > >> > > > contain
> > > > > >> > > > > > > composite structures in json. Could we flatten them
> > out
> > > > > >> directly in
> > > > > >> > > > the
> > > > > >> > > > > > > protocol definition as arrays/records?
> > > > > >> > > > > > >
> > > > > >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto
> > > topic
> > > > > >> > > creation?
> > > > > >> > > > > This
> > > > > >> > > > > > > will be a bit weird now that we have a separate
> topic
> > > > > >> creation api.
> > > > > >> > > > > Have
> > > > > >> > > > > > > you thought about how the new createTopicRequest and
> > > > > >> > > > > TopicMetadataRequest
> > > > > >> > > > > > > v1 will be used in the producer/consumer client, in
> > > addition
> > > > > >> to
> > > > > >> > > admin
> > > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > > >> TopicMetadataRequest
> > > > > >> > > from
> > > > > >> > > > > the
> > > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > > >> > > > > > >
> > > > > >> > > > > > > 2. I think Jay meant getting rid of scala classes
> > > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > > >> HeartbeatResponseAndHeader. We
> > > > > >> > > did
> > > > > >> > > > > > that
> > > > > >> > > > > > > as a stop-gap thing when adding the new requests for
> > the
> > > > > >> consumers.
> > > > > >> > > > > > > However, the long term plan is to get rid of all
> those
> > > and
> > > > > >> just
> > > > > >> > > reuse
> > > > > >> > > > > the
> > > > > >> > > > > > > java request/response in the client. Since this KIP
> > > proposes
> > > > > >> to
> > > > > >> > > add a
> > > > > >> > > > > > > significant number of new requests, perhaps we
> should
> > > bite
> > > > > the
> > > > > >> > > bullet
> > > > > >> > > > > to
> > > > > >> > > > > > > clean up the existing scala requests first before
> > > adding new
> > > > > >> ones?
> > > > > >> > > > > > >
> > > > > >> > > > > > > Thanks,
> > > > > >> > > > > > >
> > > > > >> > > > > > > Jun
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > > > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >> > > > > > >
> > > > > >> > > > > > > > Hi,
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > As said above - I list again all comments from
> this
> > > thread
> > > > > >> so we
> > > > > >> > > > > > > > can see what's left and finalize all pending
> issues.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > Comments from Jay:
> > > > > >> > > > > > > > 1. This is much needed functionality, but there
> are
> > a
> > > lot
> > > > > >> of the
> > > > > >> > > so
> > > > > >> > > > > > let's
> > > > > >> > > > > > > > really think these protocols through. We really
> want
> > > to
> > > > > end
> > > > > >> up
> > > > > >> > > > with a
> > > > > >> > > > > > set
> > > > > >> > > > > > > > of well thought-out, orthoganol apis. For this
> > reason
> > > I
> > > > > >> think it
> > > > > >> > > is
> > > > > >> > > > > > > really
> > > > > >> > > > > > > > important to think through the end state even if
> > that
> > > > > >> includes
> > > > > >> > > APIs
> > > > > >> > > > > we
> > > > > >> > > > > > > > won't implement in the first phase.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: Definitely behind this. Would appreciate if
> there
> > > are
> > > > > >> concrete
> > > > > >> > > > > > > comments
> > > > > >> > > > > > > > how this can be improved.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 2. Let's please please please wait until we have
> > > switched
> > > > > >> the
> > > > > >> > > > server
> > > > > >> > > > > > over
> > > > > >> > > > > > > > to the new java protocol definitions. If we add
> > upteen
> > > > > more
> > > > > >> ad
> > > > > >> > > hoc
> > > > > >> > > > > > scala
> > > > > >> > > > > > > > objects that is just generating more work for the
> > > > > >> conversion we
> > > > > >> > > > know
> > > > > >> > > > > we
> > > > > >> > > > > > > > have to do.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: Fixed in the latest patch - removed scala
> > protocol
> > > > > >> classes.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 3. This proposal introduces a new type of optional
> > > > > >> parameter.
> > > > > >> > > This
> > > > > >> > > > is
> > > > > >> > > > > > > > inconsistent with everything else in the protocol
> > > where we
> > > > > >> use -1
> > > > > >> > > > or
> > > > > >> > > > > > some
> > > > > >> > > > > > > > other marker value. You could argue either way but
> > > let's
> > > > > >> stick
> > > > > >> > > with
> > > > > >> > > > > > that
> > > > > >> > > > > > > > for consistency. For clients that implemented the
> > > protocol
> > > > > >> in a
> > > > > >> > > > > better
> > > > > >> > > > > > > way
> > > > > >> > > > > > > > than our scala code these basic primitives are
> hard
> > to
> > > > > >> change.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf
> type
> > > and
> > > > > >> changed
> > > > > >> > > > > > protocol
> > > > > >> > > > > > > > accordingly.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> > > > > >> TopicMetadataRequest
> > > > > >> > > > > which
> > > > > >> > > > > > > has
> > > > > >> > > > > > > > brokers, topics, and partitions. I think we should
> > > rename
> > > > > >> that
> > > > > >> > > > > request
> > > > > >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest)
> and
> > > > > >> include the
> > > > > >> > > id
> > > > > >> > > > > of
> > > > > >> > > > > > > the
> > > > > >> > > > > > > > controller. Or are there other things we could add
> > > here?
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: I agree. Updated the KIP. Let's extends
> > > TopicMetadata
> > > > > to
> > > > > >> > > > version 2
> > > > > >> > > > > > and
> > > > > >> > > > > > > > include controller.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 5. We have a tendency to try to make a lot of
> > requests
> > > > > that
> > > > > >> can
> > > > > >> > > > only
> > > > > >> > > > > go
> > > > > >> > > > > > > to
> > > > > >> > > > > > > > particular nodes. This adds a lot of burden for
> > client
> > > > > >> > > > > implementations
> > > > > >> > > > > > > (it
> > > > > >> > > > > > > > sounds easy but each discovery can fail in many
> > parts
> > > so
> > > > > it
> > > > > >> ends
> > > > > >> > > up
> > > > > >> > > > > > > being a
> > > > > >> > > > > > > > full state machine to do right). I think we should
> > > > > consider
> > > > > >> > > making
> > > > > >> > > > > > admin
> > > > > >> > > > > > > > commands and ideally as many of the other apis as
> > > possible
> > > > > >> > > > available
> > > > > >> > > > > on
> > > > > >> > > > > > > all
> > > > > >> > > > > > > > brokers and just redirect to the controller on the
> > > broker
> > > > > >> side.
> > > > > >> > > > > Perhaps
> > > > > >> > > > > > > > there would be a general way to encapsulate this
> > > > > re-routing
> > > > > >> > > > behavior.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: It's a very interesting idea, but seems there
> are
> > > some
> > > > > >> > > concerns
> > > > > >> > > > > > about
> > > > > >> > > > > > > > this
> > > > > >> > > > > > > > feature (like performance considerations, how this
> > > will
> > > > > >> > > complicate
> > > > > >> > > > > > server
> > > > > >> > > > > > > > etc).
> > > > > >> > > > > > > > I believe this shouldn't be a blocker. If this
> > > feature is
> > > > > >> > > > implemented
> > > > > >> > > > > > at
> > > > > >> > > > > > > > some
> > > > > >> > > > > > > > point it won't affect Admin changes - at least no
> > > changes
> > > > > to
> > > > > >> > > public
> > > > > >> > > > > API
> > > > > >> > > > > > > > will be required.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 6. We should probably normalize the key value
> pairs
> > > used
> > > > > for
> > > > > >> > > > configs
> > > > > >> > > > > > > rather
> > > > > >> > > > > > > > than embedding a new formatting. So two strings
> > rather
> > > > > than
> > > > > >> one
> > > > > >> > > > with
> > > > > >> > > > > an
> > > > > >> > > > > > > > internal equals sign.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: Fixed in the latest patch - normalized configs
> > and
> > > > > >> changed
> > > > > >> > > > > protocol
> > > > > >> > > > > > > > accordingly.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 7. Is the postcondition of these APIs that the
> > > command has
> > > > > >> begun
> > > > > >> > > or
> > > > > >> > > > > > that
> > > > > >> > > > > > > > the command has been completed? It is a lot more
> > > usable if
> > > > > >> the
> > > > > >> > > > > command
> > > > > >> > > > > > > has
> > > > > >> > > > > > > > been completed so you know that if you create a
> > topic
> > > and
> > > > > >> then
> > > > > >> > > > > publish
> > > > > >> > > > > > to
> > > > > >> > > > > > > > it you won't get an exception about there being no
> > > such
> > > > > >> topic.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: For long running requests (like reassign
> > > partitions) -
> > > > > >> the
> > > > > >> > > post
> > > > > >> > > > > > > > condition is
> > > > > >> > > > > > > > command has begun - so we don't block the client.
> In
> > > case
> > > > > >> of your
> > > > > >> > > > > > > example -
> > > > > >> > > > > > > > topic commands, this will be refactored and topic
> > > commands
> > > > > >> will
> > > > > >> > > be
> > > > > >> > > > > > > executed
> > > > > >> > > > > > > > immediately, since the Controller will serve Admin
> > > > > requests
> > > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 8. Describe topic and list topics duplicate a lot
> of
> > > stuff
> > > > > >> in the
> > > > > >> > > > > > > metadata
> > > > > >> > > > > > > > request. Is there a reason to give back topics
> > marked
> > > for
> > > > > >> > > > deletion? I
> > > > > >> > > > > > > feel
> > > > > >> > > > > > > > like if we just make the post-condition of the
> > delete
> > > > > >> command be
> > > > > >> > > > that
> > > > > >> > > > > > the
> > > > > >> > > > > > > > topic is deleted that will get rid of the need for
> > > this
> > > > > >> right?
> > > > > >> > > And
> > > > > >> > > > it
> > > > > >> > > > > > > will
> > > > > >> > > > > > > > be much more intuitive.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: Fixed in the latest patch - removed topics
> marked
> > > for
> > > > > >> deletion
> > > > > >> > > > in
> > > > > >> > > > > > > > ListTopicsRequest.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 9. Should we consider batching these requests? We
> > have
> > > > > >> generally
> > > > > >> > > > > tried
> > > > > >> > > > > > to
> > > > > >> > > > > > > > allow multiple operations to be batched. My
> > suspicion
> > > is
> > > > > >> that
> > > > > >> > > > without
> > > > > >> > > > > > > this
> > > > > >> > > > > > > > we will get a lot of code that does something like
> > > > > >> > > > > > > >    for(topic: adminClient.listTopics())
> > > > > >> > > > > > > >       adminClient.describeTopic(topic)
> > > > > >> > > > > > > > this code will work great when you test on 5
> topics
> > > but
> > > > > not
> > > > > >> do as
> > > > > >> > > > > well
> > > > > >> > > > > > if
> > > > > >> > > > > > > > you have 50k.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: Updated the KIP - please check "Topic Admin
> > Schema"
> > > > > >> section.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 10. I think we should also discuss how we want to
> > > expose a
> > > > > >> > > > > programmatic
> > > > > >> > > > > > > JVM
> > > > > >> > > > > > > > client api for these operations. Currently people
> > > rely on
> > > > > >> > > > AdminUtils
> > > > > >> > > > > > > which
> > > > > >> > > > > > > > is totally sketchy. I think we probably need
> another
> > > > > client
> > > > > >> under
> > > > > >> > > > > > > clients/
> > > > > >> > > > > > > > that exposes administrative functionality. We will
> > > need
> > > > > >> this just
> > > > > >> > > > to
> > > > > >> > > > > > > > properly test the new apis, I suspect. We should
> > > figure
> > > > > out
> > > > > >> that
> > > > > >> > > > API.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: Updated the KIP - please check "Admin Client"
> > > section
> > > > > >> with an
> > > > > >> > > > > > initial
> > > > > >> > > > > > > > API proposal.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 11. The other information that would be really
> > useful
> > > to
> > > > > get
> > > > > >> > > would
> > > > > >> > > > be
> > > > > >> > > > > > > > information about partitions--how much data is in
> > the
> > > > > >> partition,
> > > > > >> > > > what
> > > > > >> > > > > > are
> > > > > >> > > > > > > > the segment offsets, what is the log-end offset
> > (i.e.
> > > last
> > > > > >> > > offset),
> > > > > >> > > > > > what
> > > > > >> > > > > > > is
> > > > > >> > > > > > > > the compaction point, etc. I think that done right
> > > this
> > > > > >> would be
> > > > > >> > > > the
> > > > > >> > > > > > > > successor to the very awkward OffsetRequest we
> have
> > > today.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the
> > latest
> > > > > >> patch. I
> > > > > >> > > > > believe
> > > > > >> > > > > > > > this should
> > > > > >> > > > > > > > be resolved in a separate KIP / jira ticket.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 12. Generally we can do good error handling
> without
> > > > > needing
> > > > > >> > > custom
> > > > > >> > > > > > > > server-side
> > > > > >> > > > > > > > messages. I.e. generally the client has the
> context
> > to
> > > > > know
> > > > > >> that
> > > > > >> > > if
> > > > > >> > > > > it
> > > > > >> > > > > > > got
> > > > > >> > > > > > > > an error that the topic doesn't exist to say
> "Topic
> > X
> > > > > >> doesn't
> > > > > >> > > > exist"
> > > > > >> > > > > > > rather
> > > > > >> > > > > > > > than "error code 14" (or whatever). Maybe there
> are
> > > > > specific
> > > > > >> > > cases
> > > > > >> > > > > > where
> > > > > >> > > > > > > > this is hard? If we want to add server-side error
> > > messages
> > > > > >> we
> > > > > >> > > > really
> > > > > >> > > > > do
> > > > > >> > > > > > > > need to do this in a consistent way across the
> > > protocol.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: Updated the KIP - please check "Protocol
> Errors"
> > > > > >> section. I
> > > > > >> > > > added
> > > > > >> > > > > > the
> > > > > >> > > > > > > > comprehensive, fine-grained list of error codes.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > Comments from Guozhang:
> > > > > >> > > > > > > > 13. Describe topic request: it would be great to
> go
> > > beyond
> > > > > >> just
> > > > > >> > > > > > batching
> > > > > >> > > > > > > on
> > > > > >> > > > > > > > topic name regex for this request. For example, a
> > very
> > > > > >> common use
> > > > > >> > > > > case
> > > > > >> > > > > > of
> > > > > >> > > > > > > > the topic command is to list all topics whose
> config
> > > A's
> > > > > >> value is
> > > > > >> > > > B.
> > > > > >> > > > > > With
> > > > > >> > > > > > > > topic name regex then we have to first retrieve
> > > __all__
> > > > > >> topics's
> > > > > >> > > > > > > > description info and then filter at the client
> end,
> > > which
> > > > > >> will
> > > > > >> > > be a
> > > > > >> > > > > > huge
> > > > > >> > > > > > > > burden on ZK.
> > > > > >> > > > > > > > AND
> > > > > >> > > > > > > > 14. Config K-Vs in create topic: this is related
> to
> > > the
> > > > > >> previous
> > > > > >> > > > > point;
> > > > > >> > > > > > > > maybe we can add another metadata K-V or just a
> > > metadata
> > > > > >> string
> > > > > >> > > > along
> > > > > >> > > > > > > side
> > > > > >> > > > > > > > with config K-V in create topic like we did for
> > offset
> > > > > >> commit
> > > > > >> > > > > request.
> > > > > >> > > > > > > This
> > > > > >> > > > > > > > field can be quite useful in storing information
> > like
> > > > > >> "owner" of
> > > > > >> > > > the
> > > > > >> > > > > > > topic
> > > > > >> > > > > > > > who issue the create command, etc, which is quite
> > > > > important
> > > > > >> for a
> > > > > >> > > > > > > > multi-tenant setting. Then in the describe topic
> > > request
> > > > > we
> > > > > >> can
> > > > > >> > > > also
> > > > > >> > > > > > > batch
> > > > > >> > > > > > > > on regex of the metadata field.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: As discussed it is very interesting but can be
> > > > > >> implemented
> > > > > >> > > later
> > > > > >> > > > > > after
> > > > > >> > > > > > > > we have some basic functionality there.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > 15. Today all the admin operations are async in
> the
> > > sense
> > > > > >> that
> > > > > >> > > > > command
> > > > > >> > > > > > > will
> > > > > >> > > > > > > > return once it is written in ZK, and that is why
> we
> > > need
> > > > > >> extra
> > > > > >> > > > > > > verification
> > > > > >> > > > > > > > like testUtil.waitForTopicCreated() / verify
> > partition
> > > > > >> > > reassignment
> > > > > >> > > > > > > > request, etc. With admin requests we could add a
> > flag
> > > to
> > > > > >> enable /
> > > > > >> > > > > > disable
> > > > > >> > > > > > > > synchronous requests; when it is turned on, the
> > > response
> > > > > >> will not
> > > > > >> > > > > > return
> > > > > >> > > > > > > > until the request has been completed. And for
> async
> > > > > >> requests we
> > > > > >> > > can
> > > > > >> > > > > > add a
> > > > > >> > > > > > > > "token" field in the response, and then only need
> a
> > > > > general
> > > > > >> > > "admin
> > > > > >> > > > > > > > verification request" with the given token to
> check
> > > if the
> > > > > >> async
> > > > > >> > > > > > request
> > > > > >> > > > > > > > has been completed.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: I see your point. My idea was to provide
> specific
> > > > > >> > > > Verify...Request
> > > > > >> > > > > > per
> > > > > >> > > > > > > > each
> > > > > >> > > > > > > > long running request, where needed. We can do it
> the
> > > way
> > > > > you
> > > > > >> > > > suggest.
> > > > > >> > > > > > The
> > > > > >> > > > > > > > only
> > > > > >> > > > > > > > concern is that introducing a token we again will
> > make
> > > > > >> schema
> > > > > >> > > > > > "dynamic".
> > > > > >> > > > > > > We
> > > > > >> > > > > > > > wanted
> > > > > >> > > > > > > > to do similar thing introducing single
> AdminRequest
> > > for
> > > > > all
> > > > > >> topic
> > > > > >> > > > > > > commands
> > > > > >> > > > > > > > but rejected
> > > > > >> > > > > > > > this idea because we wanted to have schema
> defined.
> > So
> > > > > this
> > > > > >> is
> > > > > >> > > > more a
> > > > > >> > > > > > > > choice between:
> > > > > >> > > > > > > > a) have fixed schema but introduce each time new
> > > > > >> Verify...Request
> > > > > >> > > > for
> > > > > >> > > > > > > > long-running requests
> > > > > >> > > > > > > > b) use one request for verification but generalize
> > it
> > > with
> > > > > >> token
> > > > > >> > > > > > > > I'm fine with whatever decision community come to.
> > > Just
> > > > > let
> > > > > >> me
> > > > > >> > > know
> > > > > >> > > > > > your
> > > > > >> > > > > > > > thoughts.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > Comment from Gwen:
> > > > > >> > > > > > > > 16. Specifically for ownership, I think the plan
> is
> > > to add
> > > > > >> ACL
> > > > > >> > > (it
> > > > > >> > > > > > sounds
> > > > > >> > > > > > > > like you are describing ACL) via an external
> system
> > > > > (Argus,
> > > > > >> > > > Sentry).
> > > > > >> > > > > > > > I remember KIP-11 described this, but I can't find
> > > the KIP
> > > > > >> any
> > > > > >> > > > > longer.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > A: Okay, no problem. Not sure though how we are
> > going
> > > to
> > > > > >> handle
> > > > > >> > > it.
> > > > > >> > > > > > Wait
> > > > > >> > > > > > > > which KIP
> > > > > >> > > > > > > > will be committed first and include changes to
> > > > > >> TopicMetadata from
> > > > > >> > > > the
> > > > > >> > > > > > > later
> > > > > >> > > > > > > > one?
> > > > > >> > > > > > > > Anyway, I added this note to "Open Questions"
> > section
> > > so
> > > > > we
> > > > > >> don't
> > > > > >> > > > > miss
> > > > > >> > > > > > > this
> > > > > >> > > > > > > > piece.
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > Thanks,
> > > > > >> > > > > > > > Andrii Biletskyi
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii
> Biletskyi <
> > > > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > > Hi all,
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > Today I uploaded the patch that covers some of
> the
> > > > > >> discussed
> > > > > >> > > and
> > > > > >> > > > > > agreed
> > > > > >> > > > > > > > > items:
> > > > > >> > > > > > > > > - removed MaybeOf optional type
> > > > > >> > > > > > > > > - switched to java protocol definitions
> > > > > >> > > > > > > > > - simplified messages (normalized configs,
> removed
> > > topic
> > > > > >> marked
> > > > > >> > > > for
> > > > > >> > > > > > > > > deletion)
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > I also updated the KIP-4 with respective changes
> > and
> > > > > >> wrote down
> > > > > >> > > > my
> > > > > >> > > > > > > > > proposal for
> > > > > >> > > > > > > > > pending items:
> > > > > >> > > > > > > > > - Batch Admin Operations -> updated Wire
> Protocol
> > > schema
> > > > > >> > > proposal
> > > > > >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
> > > > > >> > > > TopicMetadataRequest
> > > > > >> > > > > > > > > - Admin Client -> updated my initial proposal to
> > > reflect
> > > > > >> > > batching
> > > > > >> > > > > > > > > - Error codes -> proposed fine-grained error
> code
> > > > > instead
> > > > > >> of
> > > > > >> > > > > > > > > AdminRequestFailed
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > I will also send a separate email to cover all
> > > comments
> > > > > >> from
> > > > > >> > > this
> > > > > >> > > > > > > thread.
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > Thanks,
> > > > > >> > > > > > > > > Andrii Biletskyi
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> > > > > >> > > > > gshapira@cloudera.com
> > > > > >> > > > > > >
> > > > > >> > > > > > > > > wrote:
> > > > > >> > > > > > > > >
> > > > > >> > > > > > > > >> Found KIP-11 (
> > > > > >> > > > > > > > >>
> > > > > >> > > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >>
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > >> > > > > > > > >> )
> > > > > >> > > > > > > > >> It actually specifies changes to the Metadata
> > > protocol,
> > > > > >> so
> > > > > >> > > > making
> > > > > >> > > > > > sure
> > > > > >> > > > > > > > >> both KIPs are consistent in this regard will be
> > > good.
> > > > > >> > > > > > > > >>
> > > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira
> <
> > > > > >> > > > > > gshapira@cloudera.com
> > > > > >> > > > > > > >
> > > > > >> > > > > > > > >> wrote:
> > > > > >> > > > > > > > >> > Specifically for ownership, I think the plan
> is
> > > to
> > > > > add
> > > > > >> ACL
> > > > > >> > > (it
> > > > > >> > > > > > > sounds
> > > > > >> > > > > > > > >> > like you are describing ACL) via an external
> > > system
> > > > > >> (Argus,
> > > > > >> > > > > > Sentry).
> > > > > >> > > > > > > > >> > I remember KIP-11 described this, but I can't
> > > find
> > > > > the
> > > > > >> KIP
> > > > > >> > > any
> > > > > >> > > > > > > longer.
> > > > > >> > > > > > > > >> >
> > > > > >> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting
> > > > > >> information
> > > > > >> > > that
> > > > > >> > > > > > > already
> > > > > >> > > > > > > > >> > exists from Kafka brokers, not on adding
> > > information
> > > > > >> that
> > > > > >> > > > > perhaps
> > > > > >> > > > > > > > >> > should exist but doesn't yet?
> > > > > >> > > > > > > > >> >
> > > > > >> > > > > > > > >> > Gwen
> > > > > >> > > > > > > > >> >
> > > > > >> > > > > > > > >> >
> > > > > >> > > > > > > > >> >
> > > > > >> > > > > > > > >> >
> > > > > >> > > > > > > > >> >
> > > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang
> Wang
> > <
> > > > > >> > > > > > wangguoz@gmail.com>
> > > > > >> > > > > > > > >> wrote:
> > > > > >> > > > > > > > >> >> Folks,
> > > > > >> > > > > > > > >> >>
> > > > > >> > > > > > > > >> >> Just want to elaborate a bit more on the
> > > > > create-topic
> > > > > >> > > > metadata
> > > > > >> > > > > > and
> > > > > >> > > > > > > > >> batching
> > > > > >> > > > > > > > >> >> describe-topic based on config / metadata in
> > my
> > > > > >> previous
> > > > > >> > > > email
> > > > > >> > > > > as
> > > > > >> > > > > > > we
> > > > > >> > > > > > > > >> work
> > > > > >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to
> have
> > > some
> > > > > >> sort of
> > > > > >> > > > > topic
> > > > > >> > > > > > > > >> management
> > > > > >> > > > > > > > >> >> mechanisms, which I think is quite important
> > in
> > > a
> > > > > >> > > > multi-tenant
> > > > > >> > > > > /
> > > > > >> > > > > > > > cloud
> > > > > >> > > > > > > > >> >> architecture: today anyone can create topics
> > in
> > > a
> > > > > >> shared
> > > > > >> > > > Kafka
> > > > > >> > > > > > > > >> cluster, but
> > > > > >> > > > > > > > >> >> there is no concept or "ownership" of topics
> > > that
> > > > > are
> > > > > >> > > created
> > > > > >> > > > > by
> > > > > >> > > > > > > > >> different
> > > > > >> > > > > > > > >> >> users. For example, at LinkedIn we basically
> > > > > >> distinguish
> > > > > >> > > > topic
> > > > > >> > > > > > > owners
> > > > > >> > > > > > > > >> via
> > > > > >> > > > > > > > >> >> some casual topic name prefix, which is a
> bit
> > > > > awkward
> > > > > >> and
> > > > > >> > > > does
> > > > > >> > > > > > not
> > > > > >> > > > > > > > fly
> > > > > >> > > > > > > > >> as
> > > > > >> > > > > > > > >> >> we scale our customers. It would be great to
> > use
> > > > > >> > > > > describe-topics
> > > > > >> > > > > > > such
> > > > > >> > > > > > > > >> as:
> > > > > >> > > > > > > > >> >>
> > > > > >> > > > > > > > >> >> Describe all topics that is created by me.
> > > > > >> > > > > > > > >> >>
> > > > > >> > > > > > > > >> >> Describe all topics whose retention time is
> > > > > overriden
> > > > > >> to X.
> > > > > >> > > > > > > > >> >>
> > > > > >> > > > > > > > >> >> Describe all topics whose writable group
> > include
> > > > > user
> > > > > >> Y
> > > > > >> > > (this
> > > > > >> > > > > is
> > > > > >> > > > > > > > >> related to
> > > > > >> > > > > > > > >> >> authorization), etc..
> > > > > >> > > > > > > > >> >>
> > > > > >> > > > > > > > >> >> One possible way to achieve this is to add a
> > > > > metadata
> > > > > >> file
> > > > > >> > > in
> > > > > >> > > > > the
> > > > > >> > > > > > > > >> >> create-topic request, whose value will also
> be
> > > > > >> written ZK
> > > > > >> > > as
> > > > > >> > > > we
> > > > > >> > > > > > > > create
> > > > > >> > > > > > > > >> the
> > > > > >> > > > > > > > >> >> topic; then describe-topics can choose to
> > batch
> > > > > topics
> > > > > >> > > based
> > > > > >> > > > on
> > > > > >> > > > > > 1)
> > > > > >> > > > > > > > name
> > > > > >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata
> > > regex,
> > > > > etc.
> > > > > >> > > > > > > > >> >>
> > > > > >> > > > > > > > >> >> Thoughts?
> > > > > >> > > > > > > > >> >>
> > > > > >> > > > > > > > >> >> Guozhang
> > > > > >> > > > > > > > >> >>
> > > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang
> Wang
> > <
> > > > > >> > > > > > wangguoz@gmail.com>
> > > > > >> > > > > > > > >> wrote:
> > > > > >> > > > > > > > >> >>
> > > > > >> > > > > > > > >> >>> Thanks for the updated wiki. A few comments
> > > below:
> > > > > >> > > > > > > > >> >>>
> > > > > >> > > > > > > > >> >>> 1. Error description in response: I think
> if
> > > some
> > > > > >> > > errorCode
> > > > > >> > > > > > could
> > > > > >> > > > > > > > >> indicate
> > > > > >> > > > > > > > >> >>> several different error cases then we
> should
> > > really
> > > > > >> change
> > > > > >> > > > it
> > > > > >> > > > > to
> > > > > >> > > > > > > > >> multiple
> > > > > >> > > > > > > > >> >>> codes. In general the errorCode itself
> would
> > be
> > > > > >> precise
> > > > > >> > > and
> > > > > >> > > > > > > > >> sufficient for
> > > > > >> > > > > > > > >> >>> describing the server side errors.
> > > > > >> > > > > > > > >> >>>
> > > > > >> > > > > > > > >> >>> 2. Describe topic request: it would be
> great
> > > to go
> > > > > >> beyond
> > > > > >> > > > just
> > > > > >> > > > > > > > >> batching on
> > > > > >> > > > > > > > >> >>> topic name regex for this request. For
> > > example, a
> > > > > >> very
> > > > > >> > > > common
> > > > > >> > > > > > use
> > > > > >> > > > > > > > >> case of
> > > > > >> > > > > > > > >> >>> the topic command is to list all topics
> whose
> > > > > config
> > > > > >> A's
> > > > > >> > > > value
> > > > > >> > > > > > is
> > > > > >> > > > > > > B.
> > > > > >> > > > > > > > >> With
> > > > > >> > > > > > > > >> >>> topic name regex then we have to first
> > retrieve
> > > > > >> __all__
> > > > > >> > > > > topics's
> > > > > >> > > > > > > > >> >>> description info and then filter at the
> > client
> > > end,
> > > > > >> which
> > > > > >> > > > will
> > > > > >> > > > > > be
> > > > > >> > > > > > > a
> > > > > >> > > > > > > > >> huge
> > > > > >> > > > > > > > >> >>> burden on ZK.
> > > > > >> > > > > > > > >> >>>
> > > > > >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is
> > > related to
> > > > > >> the
> > > > > >> > > > > previous
> > > > > >> > > > > > > > point;
> > > > > >> > > > > > > > >> >>> maybe we can add another metadata K-V or
> > just a
> > > > > >> metadata
> > > > > >> > > > > string
> > > > > >> > > > > > > > along
> > > > > >> > > > > > > > >> side
> > > > > >> > > > > > > > >> >>> with config K-V in create topic like we did
> > for
> > > > > >> offset
> > > > > >> > > > commit
> > > > > >> > > > > > > > >> request. This
> > > > > >> > > > > > > > >> >>> field can be quite useful in storing
> > > information
> > > > > like
> > > > > >> > > > "owner"
> > > > > >> > > > > of
> > > > > >> > > > > > > the
> > > > > >> > > > > > > > >> topic
> > > > > >> > > > > > > > >> >>> who issue the create command, etc, which is
> > > quite
> > > > > >> > > important
> > > > > >> > > > > for
> > > > > >> > > > > > a
> > > > > >> > > > > > > > >> >>> multi-tenant setting. Then in the describe
> > > topic
> > > > > >> request
> > > > > >> > > we
> > > > > >> > > > > can
> > > > > >> > > > > > > also
> > > > > >> > > > > > > > >> batch
> > > > > >> > > > > > > > >> >>> on regex of the metadata field.
> > > > > >> > > > > > > > >> >>>
> > > > > >> > > > > > > > >> >>> 4. Today all the admin operations are async
> > in
> > > the
> > > > > >> sense
> > > > > >> > > > that
> > > > > >> > > > > > > > command
> > > > > >> > > > > > > > >> will
> > > > > >> > > > > > > > >> >>> return once it is written in ZK, and that
> is
> > > why we
> > > > > >> need
> > > > > >> > > > extra
> > > > > >> > > > > > > > >> verification
> > > > > >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() /
> verify
> > > > > >> partition
> > > > > >> > > > > > > reassignment
> > > > > >> > > > > > > > >> >>> request, etc. With admin requests we could
> > add
> > > a
> > > > > >> flag to
> > > > > >> > > > > enable
> > > > > >> > > > > > /
> > > > > >> > > > > > > > >> disable
> > > > > >> > > > > > > > >> >>> synchronous requests; when it is turned on,
> > the
> > > > > >> response
> > > > > >> > > > will
> > > > > >> > > > > > not
> > > > > >> > > > > > > > >> return
> > > > > >> > > > > > > > >> >>> until the request has been completed. And
> for
> > > async
> > > > > >> > > requests
> > > > > >> > > > > we
> > > > > >> > > > > > > can
> > > > > >> > > > > > > > >> add a
> > > > > >> > > > > > > > >> >>> "token" field in the response, and then
> only
> > > need a
> > > > > >> > > general
> > > > > >> > > > > > "admin
> > > > > >> > > > > > > > >> >>> verification request" with the given token
> to
> > > check
> > > > > >> if the
> > > > > >> > > > > async
> > > > > >> > > > > > > > >> request
> > > > > >> > > > > > > > >> >>> has been completed.
> > > > > >> > > > > > > > >> >>>
> > > > > >> > > > > > > > >> >>> 5. +1 for extending Metadata request to
> > include
> > > > > >> > > controller /
> > > > > >> > > > > > > > >> coordinator
> > > > > >> > > > > > > > >> >>> information, and then we can remove the
> > > > > >> ConsumerMetadata /
> > > > > >> > > > > > > > >> ClusterMetadata
> > > > > >> > > > > > > > >> >>> requests.
> > > > > >> > > > > > > > >> >>>
> > > > > >> > > > > > > > >> >>> Guozhang
> > > > > >> > > > > > > > >> >>>
> > > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel
> Koshy <
> > > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > > >> > > > > > > > >> wrote:
> > > > > >> > > > > > > > >> >>>
> > > > > >> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't
> > > think I
> > > > > >> will be
> > > > > >> > > > > able
> > > > > >> > > > > > to
> > > > > >> > > > > > > > >> make
> > > > > >> > > > > > > > >> >>>> it today, so if notes can be sent out
> > > afterward
> > > > > that
> > > > > >> > > would
> > > > > >> > > > be
> > > > > >> > > > > > > > great.
> > > > > >> > > > > > > > >> >>>>
> > > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800,
> > Gwen
> > > > > >> Shapira
> > > > > >> > > > wrote:
> > > > > >> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking
> > > forward
> > > > > >> to
> > > > > >> > > > > chatting
> > > > > >> > > > > > > with
> > > > > >> > > > > > > > >> >>>> everyone :)
> > > > > >> > > > > > > > >> >>>> >
> > > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe
> Stein
> > <
> > > > > >> > > > > > > joe.stein@stealth.ly>
> > > > > >> > > > > > > > >> wrote:
> > > > > >> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout
> > > invite
> > > > > to
> > > > > >> all
> > > > > >> > > > pmc,
> > > > > >> > > > > > > > >> committers
> > > > > >> > > > > > > > >> >>>> and
> > > > > >> > > > > > > > >> >>>> > > everyone I found working on a KIP. If
> I
> > > missed
> > > > > >> anyone
> > > > > >> > > > in
> > > > > >> > > > > > the
> > > > > >> > > > > > > > >> invite
> > > > > >> > > > > > > > >> >>>> please
> > > > > >> > > > > > > > >> >>>> > > let me know and can update it, np.
> > > > > >> > > > > > > > >> >>>> > >
> > > > > >> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm
> > > Eastern
> > > > > >> Time.
> > > > > >> > > > Maybe
> > > > > >> > > > > > we
> > > > > >> > > > > > > > can
> > > > > >> > > > > > > > >> get
> > > > > >> > > > > > > > >> >>>> INFRA
> > > > > >> > > > > > > > >> >>>> > > help to make a google account so we
> can
> > > manage
> > > > > >> > > better?
> > > > > >> > > > > > > > >> >>>> > >
> > > > > >> > > > > > > > >> >>>> > > To discuss
> > > > > >> > > > > > > > >> >>>> > >
> > > > > >> > > > > > > > >> >>>>
> > > > > >> > > > > > > > >>
> > > > > >> > > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >>
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > >> > > > > > > > >> >>>> > > in progress and related JIRA that are
> > > > > >> interdependent
> > > > > >> > > > and
> > > > > >> > > > > > > common
> > > > > >> > > > > > > > >> work.
> > > > > >> > > > > > > > >> >>>> > >
> > > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > >> > > > > > > > >> >>>> > >
> > > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay
> > > Kreps <
> > > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > > >> > > > > > > > >> >>>> wrote:
> > > > > >> > > > > > > > >> >>>> > >
> > > > > >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that
> will
> > > also
> > > > > >> record
> > > > > >> > > > and
> > > > > >> > > > > > make
> > > > > >> > > > > > > > the
> > > > > >> > > > > > > > >> >>>> sessions
> > > > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > > > >> > > > > > > > >> >>>> > >>
> > > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > > >> > > > > > > > >> >>>> > >>
> > > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM,
> Jeff
> > > > > Holoman
> > > > > >> <
> > > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > >> > > > > > > > >> >>>> > >>
> > > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > >> > > > > > > > >> >>>> > >> >
> > > > > >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for
> > > this
> > > > > >> purpose.
> > > > > >> > > We
> > > > > >> > > > > > could
> > > > > >> > > > > > > > >> record
> > > > > >> > > > > > > > >> >>>> the
> > > > > >> > > > > > > > >> >>>> > >> > sessions if there is interest and
> > > publish
> > > > > >> them
> > > > > >> > > out.
> > > > > >> > > > > > > > >> >>>> > >> >
> > > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > > >> > > > > > > > >> >>>> > >> >
> > > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > > >> > > > > > > > >> >>>> > >> >
> > > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM,
> Jay
> > > > > Kreps <
> > > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > > >> > > > > > > > >> >>>> wrote:
> > > > > >> > > > > > > > >> >>>> > >> >
> > > > > >> > > > > > > > >> >>>> > >> > > Let's try to get the technical
> > > hang-ups
> > > > > >> sorted
> > > > > >> > > > out,
> > > > > >> > > > > > > > though.
> > > > > >> > > > > > > > >> I
> > > > > >> > > > > > > > >> >>>> really
> > > > > >> > > > > > > > >> >>>> > >> > think
> > > > > >> > > > > > > > >> >>>> > >> > > there is some benefit to live
> > > discussion
> > > > > vs
> > > > > >> > > > > writing. I
> > > > > >> > > > > > > am
> > > > > >> > > > > > > > >> >>>> hopeful that
> > > > > >> > > > > > > > >> >>>> > >> if
> > > > > >> > > > > > > > >> >>>> > >> > > we post instructions and give
> > > ourselves a
> > > > > >> few
> > > > > >> > > > > attempts
> > > > > >> > > > > > > we
> > > > > >> > > > > > > > >> can
> > > > > >> > > > > > > > >> >>>> get it
> > > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work
> for
> > > > > >> me...any
> > > > > >> > > > > > objections?
> > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM,
> > Joe
> > > > > Stein
> > > > > >> <
> > > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > > >> > > > > > > > >> >>>> >
> > > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe
> like
> > > every
> > > > > >> > > Tuesday ~
> > > > > >> > > > > 1pm
> > > > > >> > > > > > > ET
> > > > > >> > > > > > > > /
> > > > > >> > > > > > > > >> 10am
> > > > > >> > > > > > > > >> >>>> PT
> > > > > >> > > > > > > > >> >>>> > >> ????
> > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but
> > > there
> > > > > is
> > > > > >> > > always
> > > > > >> > > > > some
> > > > > >> > > > > > > > >> issue or
> > > > > >> > > > > > > > >> >>>> > >> whatever
> > > > > >> > > > > > > > >> >>>> > >> > so
> > > > > >> > > > > > > > >> >>>> > >> > > > we know the apache irc channel
> > > works.
> > > > > We
> > > > > >> can
> > > > > >> > > > start
> > > > > >> > > > > > > there
> > > > > >> > > > > > > > >> and
> > > > > >> > > > > > > > >> >>>> see how
> > > > > >> > > > > > > > >> >>>> > >> it
> > > > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts
> too
> > > and
> > > > > >> > > associate
> > > > > >> > > > to
> > > > > >> > > > > > > > >> tickets if
> > > > > >> > > > > > > > >> >>>> need be
> > > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > > >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10
> AM,
> > > Jay
> > > > > >> Kreps <
> > > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a
> > Google
> > > > > >> Hangout to
> > > > > >> > > > chat
> > > > > >> > > > > > > about
> > > > > >> > > > > > > > >> this.
> > > > > >> > > > > > > > >> >>>> What
> > > > > >> > > > > > > > >> >>>> > >> > about
> > > > > >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
> > > > > further...I
> > > > > >> > > > actually
> > > > > >> > > > > > > think
> > > > > >> > > > > > > > it
> > > > > >> > > > > > > > >> >>>> would be
> > > > > >> > > > > > > > >> >>>> > >> > good
> > > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > > >> > > > > > > > >> >>>> > >> > > > > everyone spending a
> reasonable
> > > chunk
> > > > > of
> > > > > >> > > their
> > > > > >> > > > > week
> > > > > >> > > > > > > on
> > > > > >> > > > > > > > >> Kafka
> > > > > >> > > > > > > > >> >>>> stuff
> > > > > >> > > > > > > > >> >>>> > >> to
> > > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > > >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think
> we
> > > could
> > > > > >> use
> > > > > >> > > time
> > > > > >> > > > > to
> > > > > >> > > > > > > talk
> > > > > >> > > > > > > > >> >>>> through
> > > > > >> > > > > > > > >> >>>> > >> design
> > > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on
> top
> > of
> > > > > code
> > > > > >> > > > reviews,
> > > > > >> > > > > > talk
> > > > > >> > > > > > > > >> through
> > > > > >> > > > > > > > >> >>>> any
> > > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > We can make it publicly
> > > available so
> > > > > >> that
> > > > > >> > > any
> > > > > >> > > > > one
> > > > > >> > > > > > > can
> > > > > >> > > > > > > > >> follow
> > > > > >> > > > > > > > >> >>>> along
> > > > > >> > > > > > > > >> >>>> > >> > who
> > > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this?
> If
> > so
> > > > > I'll
> > > > > >> try
> > > > > >> > > to
> > > > > >> > > > > set
> > > > > >> > > > > > it
> > > > > >> > > > > > > > up
> > > > > >> > > > > > > > >> >>>> starting
> > > > > >> > > > > > > > >> >>>> > >> next
> > > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57
> > AM,
> > > > > Andrii
> > > > > >> > > > > Biletskyi
> > > > > >> > > > > > <
> > > > > >> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page,
> fixed
> > /
> > > > > >> aligned
> > > > > >> > > > > document
> > > > > >> > > > > > > > >> structure.
> > > > > >> > > > > > > > >> >>>> Also I
> > > > > >> > > > > > > > >> >>>> > >> > > added
> > > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > > > >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
> > > > > >> AdminClient so
> > > > > >> > > we
> > > > > >> > > > > have
> > > > > >> > > > > > > > >> something
> > > > > >> > > > > > > > >> >>>> to
> > > > > >> > > > > > > > >> >>>> > >> start
> > > > > >> > > > > > > > >> >>>> > >> > > > from
> > > > > >> > > > > > > > >> >>>> > >> > > > > > while
> > > > > >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > >
> > > > > >> > > > > > > > >> >>>> > >> > >
> > > > > >> > > > > > > > >> >>>> > >> >
> > > > > >> > > > > > > > >> >>>> > >>
> > > > > >> > > > > > > > >> >>>>
> > > > > >> > > > > > > > >>
> > > > > >> > > > > > > >
> > > > > >> > > > > > >
> > > > > >> > > > > >
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >>
> > > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > > > > >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at
> 9:01
> > > PM,
> > > > > >> Andrii
> > > > > >> > > > > > Biletskyi
> > > > > >> > > > > > > <
> > > > > >> > > > > > > > >> >>>> > >> > > > > >
> andrii.biletskyi@stealth.ly>
> > > > > wrote:
> > > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you
> are
> > > right,
> > > > > >> in
> > > > > >> > > most
> > > > > >> > > > > > cases
> > > > > >> > > > > > > > >> client
> > > > > >> > > > > > > > >> >>>> will
> > > > > >> > > > > > > > >> >>>> > >> have
> > > > > >> > > > > > > > >> >>>> > >> > > > enough
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > context to show
> descriptive
> > > error
> > > > > >> > > message.
> > > > > >> > > > > My
> > > > > >> > > > > > > > >> concern is
> > > > > >> > > > > > > > >> >>>> that
> > > > > >> > > > > > > > >> >>>> > >> we
> > > > > >> > > > > > > > >> >>>> > >> > > will
> > > > > >> > > > > > > > >> >>>> > >> > > > > > have
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > to
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error
> codes
> > > for
> > > > > >> each
> > > > > >> > > > > possible
> > > > > >> > > > > > > > >> error. Of
> > > > > >> > > > > > > > >> >>>> course,
> > > > > >> > > > > > > > >> >>>> > >> > we
> > > > > >> > > > > > > > >> >>>> > >> > > > > could
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > reuse
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
> > > > > >> > > > > > > UknownTopicOrPartitionCode,
> > > > > >> > > > > > > > >> but we
> > > > > >> > > > > > > > >> >>>> will
> > > > > >> > > > > > > > >> >>>> > >> > also
> > > > > >> > > > > > > > >> >>>> > >> > > > need
> > > > > >> > > > > > > > >> >>>> > >> > > > > > to
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> > > > > >> TopicAlreadyExistsCode,
> > > > > >> > > > > > > > >> >>>> TopicConfigInvalid (both
> > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > >> > > > > > > > >> >>>> > >> > > > > topic
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > name and config, and
> > probably
> > > > > user
> > > > > >> would
> > > > > >> > > > > like
> > > > > >> > > > > > to
> > > > > >> > > > > > > > >> know
> > > > > >> > > > > > > > >> >>>> what
> > > > > >> > > > > > > > >> >>>> > >> > exactly
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
> > > > > >> > > > > > > InvalidReplicaAssignment,
> > > > > >> > > > > > > > >> >>>> > >> InternalError
> > > > > >> > > > > > > > >> >>>> > >> > > > (e.g.
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
> > > > > TopicCommand,
> > > > > >> we
> > > > > >> > > will
> > > > > >> > > > > > also
> > > > > >> > > > > > > > >> need to
> > > > > >> > > > > > > > >> >>>> add
> > > > > >> > > > > > > > >> >>>> > >> > similar
> > > > > >> > > > > > > > >> >>>> > >> > > > > stuff
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > for
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> > > > > >> PreferredReplica. So
> > > > > >> > > > > we'll
> > > > > >> > > > > > > end
> > > > > >> > > > > > > > >> up
> > > > > >> > > > > > > > >> >>>> with a
> > > > > >> > > > > > > > >> >>>> > >> > large
> > > > > >> > > > > > > > >> >>>> > >> > > > list
> > > > > >> > > > > > > > >> >>>> > >> > > > > > of
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in
> > > Admin
> > > > > >> > > protocol.
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree
> > my
> > > > > >> proposal is
> > > > > >> > > > not
> > > > > >> > > > > > > > >> consistent
> > > > > >> > > > > > > > >> >>>> with
> > > > > >> > > > > > > > >> >>>> > >> > other
> > > > > >> > > > > > > > >> >>>> > >> > > > > cases.
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better
> > > solution
> > > > > >> or
> > > > > >> > > > > something
> > > > > >> > > > > > > > >> >>>> in-between.
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think
> it
> > > is a
> > > > > >> great
> > > > > >> > > > idea.
> > > > > >> > > > > > > This
> > > > > >> > > > > > > > >> way we
> > > > > >> > > > > > > > >> >>>> can
> > > > > >> > > > > > > > >> >>>> > >> move
> > > > > >> > > > > > > > >> >>>> > >> > > on
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > faster.
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on
> > > date/time
> > > > > so
> > > > > >> > > people
> > > > > >> > > > > can
> > > > > >> > > > > > > > join.
> > > > > >> > > > > > > > >> >>>> Will work
> > > > > >> > > > > > > > >> >>>> > >> > for
> > > > > >> > > > > > > > >> >>>> > >> > > me
> > > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > and
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime
> if
> > > > > agreed
> > > > > >> in
> > > > > >> > > > > advance.
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at
> > 7:09
> > > PM,
> > > > > >> Jay
> > > > > >> > > > Kreps <
> > > > > >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > > >> > > > > > > > >> >>>> > >> > > > > wrote:
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good
> > > error
> > > > > >> handling
> > > > > >> > > > > > without
> > > > > >> > > > > > > > >> needing
> > > > > >> > > > > > > > >> >>>> custom
> > > > > >> > > > > > > > >> >>>> > >> > > > > > server-side
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally
> > the
> > > > > >> client has
> > > > > >> > > > the
> > > > > >> > > > > > > > >> context to
> > > > > >> > > > > > > > >> >>>> know
> > > > > >> > > > > > > > >> >>>> > >> that
> > > > > >> > > > > > > > >> >>>> > >> > > if
> > > > > >> > > > > > > > >> >>>> > >> > > > it
> > > > > >> > > > > > > > >> >>>> > >> > > > > > got
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic
> > > doesn't
> > > > > >> exist
> > > > > >> > > to
> > > > > >> > > > > say
> > > > > >> > > > > > > > >> "Topic X
> > > > > >> > > > > > > > >> >>>> doesn't
> > > > > >> > > > > > > > >> >>>> > >> > > exist"
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> rather
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or
> > > > > >> whatever).
> > > > > >> > > Maybe
> > > > > >> > > > > > there
> > > > > >> > > > > > > > are
> > > > > >> > > > > > > > >> >>>> specific
> > > > > >> > > > > > > > >> >>>> > >> > cases
> > > > > >> > > > > > > > >> >>>> > >> > > > > where
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want
> > to
> > > add
> > > > > >> > > > server-side
> > > > > >> > > > > > > error
> > > > > >> > > > > > > > >> >>>> messages we
> > > > > >> > > > > > > > >> >>>> > >> > > really
> > > > > >> > > > > > > > >> >>>> > >> > > > > do
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a
> > > consistent
> > > > > >> way
> > > > > >> > > > across
> > > > > >> > > > > > the
> > > > > >> > > > > > > > >> protocol.
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of
> > open
> > > > > >> questions
> > > > > >> > > > here
> > > > > >> > > > > > > from
> > > > > >> > > > > > > > my
> > > > > >> > > > > > > > >> >>>> previous
> > > > > >> > > > > > > > >> >>>> > >> > > list. I
> > > > > >> > > > > > > > >> >>>> > >> > > > > > will
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few
> > > days for
> > > > > >> Strata
> > > > > >> > > > > > though.
> > > > > >> > > > > > > > >> Maybe
> > > > > >> > > > > > > > >> >>>> we could
> > > > > >> > > > > > > > >> >>>> > >> > do
> > > > > >> > > > > > > > >> >>>> > >> > > a
> > > > > >> > > > > > > > >> >>>> > >> > > > > > Google
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open
> > > issues
> > > > > >> some
> > > > > >> > > time
> > > > > >> > > > > > > towards
> > > > > >> > > > > > > > >> the
> > > > > >> > > > > > > > >> >>>> end of
> > > > > >> > > > > > > > >> >>>> > >> > next
> > > > > >> > > > > > > > >> >>>> > >> > > > week
> > > > > >> > > > > > > > >> >>>> > >> > > > > > for
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in
> this
> > > > > ticket?
> > > > > >> I
> > > > > >> > > have
> > > > > >> > > > a
> > > > > >> > > > > > > > feeling
> > > > > >> > > > > > > > >> that
> > > > > >> > > > > > > > >> >>>> might
> > > > > >> > > > > > > > >> >>>> > >> > > > progress
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster
> > than
> > > > > >> email--I
> > > > > >> > > > think
> > > > > >> > > > > we
> > > > > >> > > > > > > > >> could talk
> > > > > >> > > > > > > > >> >>>> > >> through
> > > > > >> > > > > > > > >> >>>> > >> > > > those
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up
> fairly
> > > > > >> quickly...
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at
> > > 7:27 AM,
> > > > > >> Andrii
> > > > > >> > > > > > > > Biletskyi <
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > andrii.biletskyi@stealth.ly
> > > >
> > > > > >> wrote:
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address
> > > some of
> > > > > >> the
> > > > > >> > > > issues
> > > > > >> > > > > > > which
> > > > > >> > > > > > > > >> were
> > > > > >> > > > > > > > >> >>>> > >> mentioned
> > > > > >> > > > > > > > >> >>>> > >> > > > > earlier
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> about
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format.
> One
> > of
> > > > > >> those was
> > > > > >> > > > > about
> > > > > >> > > > > > > > >> batching
> > > > > >> > > > > > > > >> >>>> > >> > operations.
> > > > > >> > > > > > > > >> >>>> > >> > > > What
> > > > > >> > > > > > > > >> >>>> > >> > > > > > if
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> we
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand
> > > approach
> > > > > >> and let
> > > > > >> > > > > people
> > > > > >> > > > > > > > >> specify
> > > > > >> > > > > > > > >> >>>> > >> topic-name
> > > > > >> > > > > > > > >> >>>> > >> > > by
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most
> of
> > > the
> > > > > use
> > > > > >> > > cases?
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what
> > > information
> > > > > >> should
> > > > > >> > > we
> > > > > >> > > > > > > > generally
> > > > > >> > > > > > > > >> >>>> provide in
> > > > > >> > > > > > > > >> >>>> > >> > > Admin
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin
> > > commands
> > > > > >> don't
> > > > > >> > > > imply
> > > > > >> > > > > > > they
> > > > > >> > > > > > > > >> will
> > > > > >> > > > > > > > >> >>>> be used
> > > > > >> > > > > > > > >> >>>> > >> > only
> > > > > >> > > > > > > > >> >>>> > >> > > > in
> > > > > >> > > > > > > > >> >>>> > >> > > > > > CLI
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > but,
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI
> is a
> > > very
> > > > > >> > > important
> > > > > >> > > > > > > client
> > > > > >> > > > > > > > >> of this
> > > > > >> > > > > > > > >> >>>> > >> > feature.
> > > > > >> > > > > > > > >> >>>> > >> > > In
> > > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > case,
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we
> would
> > > like
> > > > > to
> > > > > >> > > provide
> > > > > >> > > > > > users
> > > > > >> > > > > > > > >> with
> > > > > >> > > > > > > > >> >>>> rich
> > > > > >> > > > > > > > >> >>>> > >> > > experience
> > > > > >> > > > > > > > >> >>>> > >> > > > > in
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> terms
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > of
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > getting results /
> errors
> > > of
> > > > > the
> > > > > >> > > > executed
> > > > > >> > > > > > > > >> commands.
> > > > > >> > > > > > > > >> >>>> Usually
> > > > > >> > > > > > > > >> >>>> > >> we
> > > > > >> > > > > > > > >> >>>> > >> > > > supply
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> with
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses only
> > errorCode,
> > > > > which
> > > > > >> looks
> > > > > >> > > > > very
> > > > > >> > > > > > > > >> limiting,
> > > > > >> > > > > > > > >> >>>> in case
> > > > > >> > > > > > > > >> >>>> > >> > of
> > > > > >> > > > > > > > >> >>>> > >> > > > CLI
> > > > > >> > > > > > > > >> >>>> > >> > > > > we
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> may
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human
> > > readable
> > > > > >> error
> > > > > >> > > > > > > description.
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into
> account
> > > > > >> previous item
> > > > > >> > > > > about
> > > > > >> > > > > > > > >> batching,
> > > > > >> > > > > > > > >> >>>> what
> > > > > >> > > > > > > > >> >>>> > >> do
> > > > > >> > > > > > > > >> >>>> > >> > > you
> > > > > >> > > > > > > > >> >>>> > >> > > > > > think
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > about
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't
> > support
> > > > > >> regexp)
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest =>
> > > > > TopicName
> > > > > >> > > > > Partitions
> > > > > >> > > > > > > > >> Replicas
> > > > > >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse =>
> > > > > ErrorCode
> > > > > >> > > > > > > > ErrorDescription
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription =>
> > > string
> > > > > >> (empty
> > > > > >> > > if
> > > > > >> > > > > > > > >> successful)
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
> > > > > >> TopicNameRegexp
> > > > > >> > > >
> > > > > >
> > > > > > ...
> > > > > >
> > > > > > [Message clipped]
> > > > >
> > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Jun,

I might be wrong but didn't we agree we will let any broker from the
cluster handle *long-running* admin requests (at this time preferredReplica
and
reassignPartitions), via zk admin path. Thus CreateTopics etc should be
sent
only to the controller.

Thanks,
Andrii Biletskyi

On Wed, Mar 18, 2015 at 11:55 PM, Jun Rao <ju...@confluent.io> wrote:

> Joel, Andril,
>
> I think we agreed that those admin requests can be issued to any broker.
> Because of that, there doesn't seem to be a strong need to know the
> controller. So, perhaps we can proceed by not making any change to the
> format of TMR right now. When we start using create topic request in the
> producer, we will need a new version of TMR that doesn't trigger auto topic
> creation. But that can be done later.
>
> As a first cut implementation, I think the broker can just write to ZK
> directly for
> createToipic/alterTopic/reassignPartitions/preferredLeaderElection
> requests, instead of forwarding them to the controller. This will simplify
> the implementation on the broker side.
>
> Thanks,
>
> Jun
>
>
> On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <jj...@gmail.com> wrote:
>
> > (Thanks Andrii for the summary)
> >
> > For (1) yes we will circle back on that shortly after syncing up in
> > person. I think it is close to getting committed although development
> > for KAFKA-1927 can probably begin without it.
> >
> > There is one more item we covered at the hangout. i.e., whether we
> > want to add the coordinator to the topic metadata response or provide
> > a clearer ClusterMetadataRequest.
> >
> > There are two reasons I think we should try and avoid adding the
> > field:
> > - It is irrelevant to topic metadata
> > - If we finally do request rerouting in Kafka then the field would add
> >   little to no value. (It still helps to have a separate
> >   ClusterMetadataRequest to query for cluster-wide information such as
> >   'which broker is the controller?' as Joe mentioned.)
> >
> > I think it would be cleaner to have an explicit ClusterMetadataRequest
> > that you can send to any broker in order to obtain the controller (and
> > in the future possibly other cluster-wide information). I think the
> > main argument against doing this and instead adding it to the topic
> > metadata response was convenience - i.e., you don't have to discover
> > the controller in advance. However, I don't see much actual
> > benefit/convenience in this and in fact think it is a non-issue. Let
> > me know if I'm overlooking something here.
> >
> > As an example, say we need to initiate partition reassignment by
> > issuing the new ReassignPartitionsRequest to the controller (assume we
> > already have the desired manual partition assignment).  If we are to
> > augment topic metadata response then the flow be something like this :
> >
> > - Issue topic metadata request to any broker (and discover the
> >   controller
> > - Connect to controller if required (i.e., if the broker above !=
> >   controller)
> > - Issue the partition reassignment request to the controller.
> >
> > With an explicit cluster metadata request it would be:
> > - Issue cluster metadata request to any broker
> > - Connect to controller if required (i.e., if the broker above !=
> >   controller)
> > - Issue the partition reassignment request
> >
> > So it seems to add little practical value and bloats topic metadata
> > response with an irrelevant detail.
> >
> > The other angle to this is the following - is it a matter of naming?
> > Should we just rename topic metadata request/response to just
> > MetadataRequest/Response and add cluster metadata to it? By that same
> > token should we also allow querying for the consumer coordinator (and
> > in future transaction coordinator) as well? This leads to a bloated
> > request which isn't very appealing and altogether confusing.
> >
> > Thanks,
> >
> > Joel
> >
> > On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > > Andri,
> > >
> > > Thanks for the summary.
> > >
> > > 1. I just realized that in order to start working on KAFKA-1927, we
> will
> > > need to merge the changes to OffsetCommitRequest (from 0.8.2) to trunk.
> > > This is planned to be done as part of KAFKA-1634. So, we will need
> > Guozhang
> > > and Joel's help to wrap this up.
> > >
> > > 2. Thinking about this a bit more, if the semantic of those "write"
> > > requests is async (i.e., after the client gets a response, it just
> means
> > > that the operation is initiated, but not necessarily completed), we
> don't
> > > really need to forward the requests to the controller. Instead, the
> > > receiving broker can just write the operation to ZK as the admin
> command
> > > line tool previously does. This will simplify the implementation.
> > >
> > > 8. There is another implementation detail for describe topic. Ideally,
> we
> > > want to read the topic config from the broker cache, instead of
> > ZooKeeper.
> > > Currently, every broker reads the topic-level config for all topics.
> > > However, it ignores those for topics not hosted on itself. So, we may
> > need
> > > to change TopicConfigManager a bit so that it caches the configs for
> all
> > > topics.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Guys,
> > > >
> > > > Thanks for a great discussion!
> > > > Here are the actions points:
> > > >
> > > > 1. Q: Get rid of all scala requests objects, use java protocol
> > definitions.
> > > >     A: Gwen kindly took that (KAFKA-1927). It's important to speed up
> > > > review procedure
> > > >          there since this ticket blocks other important changes.
> > > >
> > > > 2. Q: Generic re-reroute facility vs client maintaining cluster
> state.
> > > >     A: Jay has added pseudo code to KAFKA-1912 - need to consider
> > whether
> > > > this will be
> > > >         easy to implement as a server-side feature (comments are
> > > > welcomed!).
> > > >
> > > > 3. Q: Controller field in wire protocol.
> > > >     A: This might be useful for clients, add this to
> > TopicMetadataResponse
> > > > (already in KIP).
> > > >
> > > > 4. Q: Decoupling topic creation from TMR.
> > > >     A: I will add proposed by Jun solution (using clientId for that)
> > to the
> > > > KIP.
> > > >
> > > > 5. Q: Bumping new versions of TMR vs grabbing all protocol changes in
> > one
> > > > version.
> > > >     A: It was decided to try to gather all changes to protocol
> (before
> > > > release).
> > > >         In case of TMR it worth checking: KAFKA-2020 and KIP-13
> > (quotas)
> > > >
> > > > 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
> > > >     A: Use jackson for that, /tools project is a separate jar so
> > shouldn't
> > > > be a big deal.
> > > >
> > > > 7.  Q: VerifyReassingPartitions vs generic status check command.
> > > >      A: For long-running requests like reassign partitions *progress*
> > check
> > > > request is useful,
> > > >          it makes sense to introduce it.
> > > >
> > > >  Please add, correct me if I missed something.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Joel,
> > > > >
> > > > > You are right, I removed ClusterMetadata because we have partially
> > > > > what we need in TopicMetadata. Also, as Jay pointed out earlier, we
> > > > > would like to have "orthogonal" API, but at the same time we need
> > > > > to be backward compatible.
> > > > >
> > > > > But I like your idea and even have some other arguments for this
> > option:
> > > > > There is also DescribeTopicRequest which was proposed in this KIP,
> > > > > it returns topic configs, partitions, replication factor plus
> > partition
> > > > > ISR, ASR,
> > > > > leader replica. The later part is really already there in
> > > > > TopicMetadataRequest.
> > > > > So again we'll have to add stuff to TMR, not to duplicate some info
> > in
> > > > > newly added requests. However, this way we'll end up with "monster"
> > > > > request which returns cluster metadata, topic replication and
> config
> > info
> > > > > plus partition replication data. Seems logical to split TMR to
> > > > > - ClusterMetadata (brokers + controller, maybe smth else)
> > > > > - TopicMetadata (topic info + partition details)
> > > > > But since current TMR is involved in lots of places (including
> > network
> > > > > client,
> > > > > as I understand) this might be very serious change and it probably
> > makes
> > > > > sense to stick with current approach.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > >
> > > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jj...@gmail.com>
> > wrote:
> > > > >
> > > > >> I may be missing some context but hopefully this will also be
> > covered
> > > > >> today: I thought the earlier proposal where there was an explicit
> > > > >> ClusterMetadata request was clearer and explicit. During the
> course
> > of
> > > > >> this thread I think the conclusion was that the main need was for
> > > > >> controller information and that can be rolled into the topic
> > metadata
> > > > >> response but that seems a bit irrelevant to topic metadata. FWIW I
> > > > >> think the full broker-list is also irrelevant to topic metadata,
> but
> > > > >> it is already there and in use. I think there is still room for an
> > > > >> explicit ClusterMetadata request since there may be other
> > > > >> cluster-level information that we may want to add over time (and
> > that
> > > > >> have nothing to do with topic metadata).
> > > > >>
> > > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
> > > > >> > Jun,
> > > > >> >
> > > > >> > 101. Okay, if you say that such use case is important. I also
> > think
> > > > >> > using clientId for these purposes is fine - if we already have
> > this
> > > > >> field
> > > > >> > as part of all Wire protocol messages, why not use that.
> > > > >> > I will update KIP-4 page if nobody has other ideas (which may
> > come up
> > > > >> > during the call today).
> > > > >> >
> > > > >> > 102.1 Agree, I'll update the KIP accordingly. I think we can add
> > new,
> > > > >> > fine-grained error codes if some error code received in specific
> > case
> > > > >> > won't give enough context to return a descriptive error message
> > for
> > > > >> user.
> > > > >> >
> > > > >> > Look forward to discussing all outstanding issues in detail
> today
> > > > during
> > > > >> > the call.
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Andrii Biletskyi
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > > > >> >
> > > > >> > > 101. There may be a use case where you only want the topics to
> > be
> > > > >> created
> > > > >> > > manually by admins. Currently, you can do that by disabling
> auto
> > > > topic
> > > > >> > > creation and issue topic creation from the TopicCommand. If we
> > > > >> disable auto
> > > > >> > > topic creation completely on the broker and don't have a way
> to
> > > > >> distinguish
> > > > >> > > between topic creation requests from the regular clients and
> the
> > > > >> admin, we
> > > > >> > > can't support manual topic creation any more. I was thinking
> > that
> > > > >> another
> > > > >> > > way of distinguishing the clients making the topic creation
> > requests
> > > > >> is
> > > > >> > > using clientId. For example, the admin tool can set it to
> > something
> > > > >> like
> > > > >> > > admin and the broker can treat that clientId specially.
> > > > >> > >
> > > > >> > > Also, there is a related discussion in KAFKA-2020. Currently,
> > we do
> > > > >> the
> > > > >> > > following in TopicMetadataResponse:
> > > > >> > >
> > > > >> > > 1. If leader is not available, we set the partition level
> error
> > code
> > > > >> to
> > > > >> > > LeaderNotAvailable.
> > > > >> > > 2. If a non-leader replica is not available, we take that
> > replica
> > > > out
> > > > >> of
> > > > >> > > the assigned replica list and isr in the response. As an
> > indication
> > > > >> for
> > > > >> > > doing that, we set the partition level error code to
> > > > >> ReplicaNotAvailable.
> > > > >> > >
> > > > >> > > This has a few problems. First, ReplicaNotAvailable probably
> > > > >> shouldn't be
> > > > >> > > an error, at least for the normal producer/consumer clients
> that
> > > > just
> > > > >> want
> > > > >> > > to find out the leader. Second, it can happen that both the
> > leader
> > > > and
> > > > >> > > another replica are not available at the same time. There is
> no
> > > > error
> > > > >> code
> > > > >> > > to indicate both. Third, even if a replica is not available,
> > it's
> > > > >> still
> > > > >> > > useful to return its replica id since some clients (e.g. admin
> > tool)
> > > > >> may
> > > > >> > > still make use of it.
> > > > >> > >
> > > > >> > > One way to address this issue is to always return the replica
> > id for
> > > > >> > > leader, assigned replicas, and isr regardless of whether the
> > > > >> corresponding
> > > > >> > > broker is live or not. Since we also return the list of live
> > > > brokers,
> > > > >> the
> > > > >> > > client can figure out whether a leader or a replica is live or
> > not
> > > > >> and act
> > > > >> > > accordingly. This way, we don't need to set the partition
> level
> > > > error
> > > > >> code
> > > > >> > > when the leader or a replica is not available. This doesn't
> > change
> > > > >> the wire
> > > > >> > > protocol, but does change the semantics. Since we are evolving
> > the
> > > > >> protocol
> > > > >> > > of TopicMetadataRequest here, we can potentially piggyback the
> > > > change.
> > > > >> > >
> > > > >> > > 102.1 For those types of errors due to invalid input,
> shouldn't
> > we
> > > > >> just
> > > > >> > > guard it at parameter validation time and throw
> > > > >> InvalidArgumentException
> > > > >> > > without even sending the request to the broker?
> > > > >> > >
> > > > >> > > Thanks,
> > > > >> > >
> > > > >> > > Jun
> > > > >> > >
> > > > >> > >
> > > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > >
> > > > >> > > > Jun,
> > > > >> > > >
> > > > >> > > > Answering your questions:
> > > > >> > > >
> > > > >> > > > 101. If I understand you correctly, you are saying future
> > producer
> > > > >> > > versions
> > > > >> > > > (which
> > > > >> > > > will be ported to TMR_V1) won't be able to automatically
> > create
> > > > >> topic (if
> > > > >> > > > we
> > > > >> > > > unconditionally remove topic creation from there). But we
> > need to
> > > > >> this
> > > > >> > > > preserve logic.
> > > > >> > > > Ok, about your proposal: I'm not a big fan too, when it
> comes
> > to
> > > > >> > > > differentiating
> > > > >> > > > clients directly in protocol schema. And also I'm not sure I
> > > > >> understand
> > > > >> > > at
> > > > >> > > > all why
> > > > >> > > > auto.create.topics.enable is a server side configuration.
> Can
> > we
> > > > >> > > deprecate
> > > > >> > > > this setting
> > > > >> > > > in future versions, add this setting to producer and based
> on
> > that
> > > > >> upon
> > > > >> > > > receiving
> > > > >> > > > UnknownTopic create topic explicitly by a separate producer
> > call
> > > > via
> > > > >> > > > adminClient?
> > > > >> > > >
> > > > >> > > > 102.1. Hm, yes. It's because we want to support batching and
> > at
> > > > the
> > > > >> same
> > > > >> > > > time we
> > > > >> > > > want to give descriptive error messages for clients. Since
> > > > >> AdminClient
> > > > >> > > > holds the context
> > > > >> > > > to construct such messages (e.g. AdminClient layer can know
> > that
> > > > >> > > > InvalidArgumentsCode
> > > > >> > > > means two cases: either invalid number - e.g. -1; or
> > > > >> replication-factor
> > > > >> > > was
> > > > >> > > > provided while
> > > > >> > > > partitions argument wasn't) - I wrapped responses in
> > Exceptions.
> > > > >> But I'm
> > > > >> > > > open to any
> > > > >> > > > other ideas, this was just initial version.
> > > > >> > > > 102.2. Yes, I agree. I'll change that to probably some other
> > dto.
> > > > >> > > >
> > > > >> > > > Thanks,
> > > > >> > > > Andrii Biletskyi
> > > > >> > > >
> > > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io>
> > > > wrote:
> > > > >> > > >
> > > > >> > > > > Andrii,
> > > > >> > > > >
> > > > >> > > > > 101. That's what I was thinking too, but it may not be
> that
> > > > >> simple. In
> > > > >> > > > > TopicMetadataRequest_V1,
> > > > >> > > > > we can let it not trigger auto topic creation. Then, in
> the
> > > > >> producer
> > > > >> > > > side,
> > > > >> > > > > if it gets an UnknownTopicException, it can explicitly
> > issue a
> > > > >> > > > > createTopicRequest for auto topic creation. On the
> consumer
> > > > side,
> > > > >> it
> > > > >> > > will
> > > > >> > > > > never issue createTopicRequest. This works when auto topic
> > > > >> creation is
> > > > >> > > > > enabled on the broker side. However, I am not sure how
> > things
> > > > >> will work
> > > > >> > > > > when auto topic creation is disabled on the broker side.
> In
> > this
> > > > >> case,
> > > > >> > > we
> > > > >> > > > > want to have a way to manually create a topic, potentially
> > > > through
> > > > >> > > admin
> > > > >> > > > > commands. However, then we need a way to distinguish
> > > > >> createTopicRequest
> > > > >> > > > > issued from the producer clients and the admin tools. May
> > be we
> > > > >> can
> > > > >> > > add a
> > > > >> > > > > new field in createTopicRequest and set it differently in
> > the
> > > > >> producer
> > > > >> > > > > client and the admin client. However, I am not sure if
> > that's
> > > > the
> > > > >> best
> > > > >> > > > > approach.
> > > > >> > > > >
> > > > >> > > > > 2. Yes, refactoring existing requests is a non-trivial
> > amount of
> > > > >> work.
> > > > >> > > I
> > > > >> > > > > posted some comments in KAFKA-1927. We will probably have
> > to fix
> > > > >> > > > KAFKA-1927
> > > > >> > > > > first, before adding the new logic in KAFKA-1694.
> > Otherwise, the
> > > > >> > > changes
> > > > >> > > > > will be too big.
> > > > >> > > > >
> > > > >> > > > > 102. About the AdminClient:
> > > > >> > > > > 102.1. It's a bit weird that we return exception in the
> > api. It
> > > > >> seems
> > > > >> > > > that
> > > > >> > > > > we should either return error code or throw an exception
> > when
> > > > >> getting
> > > > >> > > the
> > > > >> > > > > response state.
> > > > >> > > > > 102.2. We probably shouldn't explicitly use the request
> > object
> > > > in
> > > > >> the
> > > > >> > > > api.
> > > > >> > > > > Not every request evolution requires an api change.
> > > > >> > > > >
> > > > >> > > > > Thanks,
> > > > >> > > > >
> > > > >> > > > > Jun
> > > > >> > > > >
> > > > >> > > > >
> > > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > > > >
> > > > >> > > > > > Jun,
> > > > >> > > > > >
> > > > >> > > > > > Thanks for you comments. Answers inline:
> > > > >> > > > > >
> > > > >> > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > > >> > > > > > > ReassignPartitionRequest,
> > > > >> > > > > > > and PartitionsSerialized that are represented as a
> > string,
> > > > but
> > > > >> > > > contain
> > > > >> > > > > > > composite structures in json. Could we flatten them
> out
> > > > >> directly in
> > > > >> > > > the
> > > > >> > > > > > > protocol definition as arrays/records?
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > Yes, now with Admin Client this looks a bit weird. My
> > initial
> > > > >> > > > motivation
> > > > >> > > > > > was:
> > > > >> > > > > > ReassignPartitionCommand accepts input in json, we want
> to
> > > > >> remain
> > > > >> > > > tools'
> > > > >> > > > > > interfaces unchanged, where possible.
> > > > >> > > > > > If we port it to deserialized format, in CLI (/tools
> > project)
> > > > >> we will
> > > > >> > > > > have
> > > > >> > > > > > to add some
> > > > >> > > > > > json library since /tools is written in java and we'll
> > need to
> > > > >> > > > > deserialize
> > > > >> > > > > > json file
> > > > >> > > > > > provided by a user. Can we quickly agree on what this
> > library
> > > > >> should
> > > > >> > > be
> > > > >> > > > > > (Jackson, GSON, whatever)?
> > > > >> > > > > >
> > > > >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto
> topic
> > > > >> creation?
> > > > >> > > > This
> > > > >> > > > > > > will be a bit weird now that we have a separate topic
> > > > >> creation api.
> > > > >> > > > > Have
> > > > >> > > > > > > you thought about how the new createTopicRequest and
> > > > >> > > > > TopicMetadataRequest
> > > > >> > > > > > > v1 will be used in the producer/consumer client, in
> > addition
> > > > >> to
> > > > >> > > admin
> > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > >> TopicMetadataRequest
> > > > >> > > from
> > > > >> > > > > the
> > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > I agree, this strange logic should be fixed. I'm not
> > confident
> > > > >> in
> > > > >> > > this
> > > > >> > > > > > Kafka part so
> > > > >> > > > > > correct me if I'm wrong, but it doesn't look like a hard
> > thing
> > > > >> to
> > > > >> > > do, I
> > > > >> > > > > > think we can
> > > > >> > > > > > leverage AdminClient for that in Producer and
> > unconditionally
> > > > >> remove
> > > > >> > > > > topic
> > > > >> > > > > > creation from the TopicMetadataRequest_V1.
> > > > >> > > > > >
> > > > >> > > > > > 2. I think Jay meant getting rid of scala classes
> > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > >> HeartbeatResponseAndHeader. We
> > > > >> > > did
> > > > >> > > > > > that
> > > > >> > > > > > > as a stop-gap thing when adding the new requests for
> the
> > > > >> consumers.
> > > > >> > > > > > > However, the long term plan is to get rid of all those
> > and
> > > > >> just
> > > > >> > > reuse
> > > > >> > > > > the
> > > > >> > > > > > > java request/response in the client. Since this KIP
> > proposes
> > > > >> to
> > > > >> > > add a
> > > > >> > > > > > > significant number of new requests, perhaps we should
> > bite
> > > > the
> > > > >> > > bullet
> > > > >> > > > > to
> > > > >> > > > > > > clean up the existing scala requests first before
> > adding new
> > > > >> ones?
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > > > Yes, looks like I misunderstood the point of
> > > > >> ...RequestAndHeader.
> > > > >> > > > Okay, I
> > > > >> > > > > > will
> > > > >> > > > > > rework that. The only thing is that I don't see any
> > example
> > > > how
> > > > >> it
> > > > >> > > was
> > > > >> > > > > done
> > > > >> > > > > > for at
> > > > >> > > > > > least one existing protocol message. Thus, as I
> > understand, I
> > > > >> have to
> > > > >> > > > > think
> > > > >> > > > > > how we
> > > > >> > > > > > are going to do it.
> > > > >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
> > > > reasonable,
> > > > >> but
> > > > >> > > if
> > > > >> > > > > it's
> > > > >> > > > > > an *obligatory*
> > > > >> > > > > > requirement to have Admin KIP done, I'm afraid this can
> > be a
> > > > >> serious
> > > > >> > > > > > blocker for us.
> > > > >> > > > > > There are 13 protocol messages and all that would
> require
> > not
> > > > >> only
> > > > >> > > unit
> > > > >> > > > > > tests but quite
> > > > >> > > > > > intensive manual testing, no? I'm afraid I'm not the
> > right guy
> > > > >> to
> > > > >> > > cover
> > > > >> > > > > > pretty much all
> > > > >> > > > > > Kafka core internals :). Let me know your thoughts on
> this
> > > > >> item. Btw
> > > > >> > > > > there
> > > > >> > > > > > is a ticket to
> > > > >> > > > > > follow-up this issue (
> > > > >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > >> > > > ).
> > > > >> > > > > >
> > > > >> > > > > > Thanks,
> > > > >> > > > > > Andrii Biletskyi
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <
> > jun@confluent.io>
> > > > >> wrote:
> > > > >> > > > > >
> > > > >> > > > > > > Andrii,
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > A few more comments.
> > > > >> > > > > > >
> > > > >> > > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > > >> > > > > > > ReassignPartitionRequest,
> > > > >> > > > > > > and PartitionsSerialized that are represented as a
> > string,
> > > > but
> > > > >> > > > contain
> > > > >> > > > > > > composite structures in json. Could we flatten them
> out
> > > > >> directly in
> > > > >> > > > the
> > > > >> > > > > > > protocol definition as arrays/records?
> > > > >> > > > > > >
> > > > >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto
> > topic
> > > > >> > > creation?
> > > > >> > > > > This
> > > > >> > > > > > > will be a bit weird now that we have a separate topic
> > > > >> creation api.
> > > > >> > > > > Have
> > > > >> > > > > > > you thought about how the new createTopicRequest and
> > > > >> > > > > TopicMetadataRequest
> > > > >> > > > > > > v1 will be used in the producer/consumer client, in
> > addition
> > > > >> to
> > > > >> > > admin
> > > > >> > > > > > > tools? For example, ideally, we don't want
> > > > >> TopicMetadataRequest
> > > > >> > > from
> > > > >> > > > > the
> > > > >> > > > > > > consumer to trigger auto topic creation.
> > > > >> > > > > > >
> > > > >> > > > > > > 2. I think Jay meant getting rid of scala classes
> > > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > > >> HeartbeatResponseAndHeader. We
> > > > >> > > did
> > > > >> > > > > > that
> > > > >> > > > > > > as a stop-gap thing when adding the new requests for
> the
> > > > >> consumers.
> > > > >> > > > > > > However, the long term plan is to get rid of all those
> > and
> > > > >> just
> > > > >> > > reuse
> > > > >> > > > > the
> > > > >> > > > > > > java request/response in the client. Since this KIP
> > proposes
> > > > >> to
> > > > >> > > add a
> > > > >> > > > > > > significant number of new requests, perhaps we should
> > bite
> > > > the
> > > > >> > > bullet
> > > > >> > > > > to
> > > > >> > > > > > > clean up the existing scala requests first before
> > adding new
> > > > >> ones?
> > > > >> > > > > > >
> > > > >> > > > > > > Thanks,
> > > > >> > > > > > >
> > > > >> > > > > > > Jun
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > > > > > >
> > > > >> > > > > > > > Hi,
> > > > >> > > > > > > >
> > > > >> > > > > > > > As said above - I list again all comments from this
> > thread
> > > > >> so we
> > > > >> > > > > > > > can see what's left and finalize all pending issues.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Comments from Jay:
> > > > >> > > > > > > > 1. This is much needed functionality, but there are
> a
> > lot
> > > > >> of the
> > > > >> > > so
> > > > >> > > > > > let's
> > > > >> > > > > > > > really think these protocols through. We really want
> > to
> > > > end
> > > > >> up
> > > > >> > > > with a
> > > > >> > > > > > set
> > > > >> > > > > > > > of well thought-out, orthoganol apis. For this
> reason
> > I
> > > > >> think it
> > > > >> > > is
> > > > >> > > > > > > really
> > > > >> > > > > > > > important to think through the end state even if
> that
> > > > >> includes
> > > > >> > > APIs
> > > > >> > > > > we
> > > > >> > > > > > > > won't implement in the first phase.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Definitely behind this. Would appreciate if there
> > are
> > > > >> concrete
> > > > >> > > > > > > comments
> > > > >> > > > > > > > how this can be improved.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 2. Let's please please please wait until we have
> > switched
> > > > >> the
> > > > >> > > > server
> > > > >> > > > > > over
> > > > >> > > > > > > > to the new java protocol definitions. If we add
> upteen
> > > > more
> > > > >> ad
> > > > >> > > hoc
> > > > >> > > > > > scala
> > > > >> > > > > > > > objects that is just generating more work for the
> > > > >> conversion we
> > > > >> > > > know
> > > > >> > > > > we
> > > > >> > > > > > > > have to do.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Fixed in the latest patch - removed scala
> protocol
> > > > >> classes.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 3. This proposal introduces a new type of optional
> > > > >> parameter.
> > > > >> > > This
> > > > >> > > > is
> > > > >> > > > > > > > inconsistent with everything else in the protocol
> > where we
> > > > >> use -1
> > > > >> > > > or
> > > > >> > > > > > some
> > > > >> > > > > > > > other marker value. You could argue either way but
> > let's
> > > > >> stick
> > > > >> > > with
> > > > >> > > > > > that
> > > > >> > > > > > > > for consistency. For clients that implemented the
> > protocol
> > > > >> in a
> > > > >> > > > > better
> > > > >> > > > > > > way
> > > > >> > > > > > > > than our scala code these basic primitives are hard
> to
> > > > >> change.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf type
> > and
> > > > >> changed
> > > > >> > > > > > protocol
> > > > >> > > > > > > > accordingly.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> > > > >> TopicMetadataRequest
> > > > >> > > > > which
> > > > >> > > > > > > has
> > > > >> > > > > > > > brokers, topics, and partitions. I think we should
> > rename
> > > > >> that
> > > > >> > > > > request
> > > > >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest) and
> > > > >> include the
> > > > >> > > id
> > > > >> > > > > of
> > > > >> > > > > > > the
> > > > >> > > > > > > > controller. Or are there other things we could add
> > here?
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: I agree. Updated the KIP. Let's extends
> > TopicMetadata
> > > > to
> > > > >> > > > version 2
> > > > >> > > > > > and
> > > > >> > > > > > > > include controller.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 5. We have a tendency to try to make a lot of
> requests
> > > > that
> > > > >> can
> > > > >> > > > only
> > > > >> > > > > go
> > > > >> > > > > > > to
> > > > >> > > > > > > > particular nodes. This adds a lot of burden for
> client
> > > > >> > > > > implementations
> > > > >> > > > > > > (it
> > > > >> > > > > > > > sounds easy but each discovery can fail in many
> parts
> > so
> > > > it
> > > > >> ends
> > > > >> > > up
> > > > >> > > > > > > being a
> > > > >> > > > > > > > full state machine to do right). I think we should
> > > > consider
> > > > >> > > making
> > > > >> > > > > > admin
> > > > >> > > > > > > > commands and ideally as many of the other apis as
> > possible
> > > > >> > > > available
> > > > >> > > > > on
> > > > >> > > > > > > all
> > > > >> > > > > > > > brokers and just redirect to the controller on the
> > broker
> > > > >> side.
> > > > >> > > > > Perhaps
> > > > >> > > > > > > > there would be a general way to encapsulate this
> > > > re-routing
> > > > >> > > > behavior.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: It's a very interesting idea, but seems there are
> > some
> > > > >> > > concerns
> > > > >> > > > > > about
> > > > >> > > > > > > > this
> > > > >> > > > > > > > feature (like performance considerations, how this
> > will
> > > > >> > > complicate
> > > > >> > > > > > server
> > > > >> > > > > > > > etc).
> > > > >> > > > > > > > I believe this shouldn't be a blocker. If this
> > feature is
> > > > >> > > > implemented
> > > > >> > > > > > at
> > > > >> > > > > > > > some
> > > > >> > > > > > > > point it won't affect Admin changes - at least no
> > changes
> > > > to
> > > > >> > > public
> > > > >> > > > > API
> > > > >> > > > > > > > will be required.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 6. We should probably normalize the key value pairs
> > used
> > > > for
> > > > >> > > > configs
> > > > >> > > > > > > rather
> > > > >> > > > > > > > than embedding a new formatting. So two strings
> rather
> > > > than
> > > > >> one
> > > > >> > > > with
> > > > >> > > > > an
> > > > >> > > > > > > > internal equals sign.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Fixed in the latest patch - normalized configs
> and
> > > > >> changed
> > > > >> > > > > protocol
> > > > >> > > > > > > > accordingly.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 7. Is the postcondition of these APIs that the
> > command has
> > > > >> begun
> > > > >> > > or
> > > > >> > > > > > that
> > > > >> > > > > > > > the command has been completed? It is a lot more
> > usable if
> > > > >> the
> > > > >> > > > > command
> > > > >> > > > > > > has
> > > > >> > > > > > > > been completed so you know that if you create a
> topic
> > and
> > > > >> then
> > > > >> > > > > publish
> > > > >> > > > > > to
> > > > >> > > > > > > > it you won't get an exception about there being no
> > such
> > > > >> topic.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: For long running requests (like reassign
> > partitions) -
> > > > >> the
> > > > >> > > post
> > > > >> > > > > > > > condition is
> > > > >> > > > > > > > command has begun - so we don't block the client. In
> > case
> > > > >> of your
> > > > >> > > > > > > example -
> > > > >> > > > > > > > topic commands, this will be refactored and topic
> > commands
> > > > >> will
> > > > >> > > be
> > > > >> > > > > > > executed
> > > > >> > > > > > > > immediately, since the Controller will serve Admin
> > > > requests
> > > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > >> > > > > > > >
> > > > >> > > > > > > > 8. Describe topic and list topics duplicate a lot of
> > stuff
> > > > >> in the
> > > > >> > > > > > > metadata
> > > > >> > > > > > > > request. Is there a reason to give back topics
> marked
> > for
> > > > >> > > > deletion? I
> > > > >> > > > > > > feel
> > > > >> > > > > > > > like if we just make the post-condition of the
> delete
> > > > >> command be
> > > > >> > > > that
> > > > >> > > > > > the
> > > > >> > > > > > > > topic is deleted that will get rid of the need for
> > this
> > > > >> right?
> > > > >> > > And
> > > > >> > > > it
> > > > >> > > > > > > will
> > > > >> > > > > > > > be much more intuitive.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Fixed in the latest patch - removed topics marked
> > for
> > > > >> deletion
> > > > >> > > > in
> > > > >> > > > > > > > ListTopicsRequest.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 9. Should we consider batching these requests? We
> have
> > > > >> generally
> > > > >> > > > > tried
> > > > >> > > > > > to
> > > > >> > > > > > > > allow multiple operations to be batched. My
> suspicion
> > is
> > > > >> that
> > > > >> > > > without
> > > > >> > > > > > > this
> > > > >> > > > > > > > we will get a lot of code that does something like
> > > > >> > > > > > > >    for(topic: adminClient.listTopics())
> > > > >> > > > > > > >       adminClient.describeTopic(topic)
> > > > >> > > > > > > > this code will work great when you test on 5 topics
> > but
> > > > not
> > > > >> do as
> > > > >> > > > > well
> > > > >> > > > > > if
> > > > >> > > > > > > > you have 50k.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Updated the KIP - please check "Topic Admin
> Schema"
> > > > >> section.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 10. I think we should also discuss how we want to
> > expose a
> > > > >> > > > > programmatic
> > > > >> > > > > > > JVM
> > > > >> > > > > > > > client api for these operations. Currently people
> > rely on
> > > > >> > > > AdminUtils
> > > > >> > > > > > > which
> > > > >> > > > > > > > is totally sketchy. I think we probably need another
> > > > client
> > > > >> under
> > > > >> > > > > > > clients/
> > > > >> > > > > > > > that exposes administrative functionality. We will
> > need
> > > > >> this just
> > > > >> > > > to
> > > > >> > > > > > > > properly test the new apis, I suspect. We should
> > figure
> > > > out
> > > > >> that
> > > > >> > > > API.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Updated the KIP - please check "Admin Client"
> > section
> > > > >> with an
> > > > >> > > > > > initial
> > > > >> > > > > > > > API proposal.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 11. The other information that would be really
> useful
> > to
> > > > get
> > > > >> > > would
> > > > >> > > > be
> > > > >> > > > > > > > information about partitions--how much data is in
> the
> > > > >> partition,
> > > > >> > > > what
> > > > >> > > > > > are
> > > > >> > > > > > > > the segment offsets, what is the log-end offset
> (i.e.
> > last
> > > > >> > > offset),
> > > > >> > > > > > what
> > > > >> > > > > > > is
> > > > >> > > > > > > > the compaction point, etc. I think that done right
> > this
> > > > >> would be
> > > > >> > > > the
> > > > >> > > > > > > > successor to the very awkward OffsetRequest we have
> > today.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the
> latest
> > > > >> patch. I
> > > > >> > > > > believe
> > > > >> > > > > > > > this should
> > > > >> > > > > > > > be resolved in a separate KIP / jira ticket.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 12. Generally we can do good error handling without
> > > > needing
> > > > >> > > custom
> > > > >> > > > > > > > server-side
> > > > >> > > > > > > > messages. I.e. generally the client has the context
> to
> > > > know
> > > > >> that
> > > > >> > > if
> > > > >> > > > > it
> > > > >> > > > > > > got
> > > > >> > > > > > > > an error that the topic doesn't exist to say "Topic
> X
> > > > >> doesn't
> > > > >> > > > exist"
> > > > >> > > > > > > rather
> > > > >> > > > > > > > than "error code 14" (or whatever). Maybe there are
> > > > specific
> > > > >> > > cases
> > > > >> > > > > > where
> > > > >> > > > > > > > this is hard? If we want to add server-side error
> > messages
> > > > >> we
> > > > >> > > > really
> > > > >> > > > > do
> > > > >> > > > > > > > need to do this in a consistent way across the
> > protocol.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Updated the KIP - please check "Protocol Errors"
> > > > >> section. I
> > > > >> > > > added
> > > > >> > > > > > the
> > > > >> > > > > > > > comprehensive, fine-grained list of error codes.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Comments from Guozhang:
> > > > >> > > > > > > > 13. Describe topic request: it would be great to go
> > beyond
> > > > >> just
> > > > >> > > > > > batching
> > > > >> > > > > > > on
> > > > >> > > > > > > > topic name regex for this request. For example, a
> very
> > > > >> common use
> > > > >> > > > > case
> > > > >> > > > > > of
> > > > >> > > > > > > > the topic command is to list all topics whose config
> > A's
> > > > >> value is
> > > > >> > > > B.
> > > > >> > > > > > With
> > > > >> > > > > > > > topic name regex then we have to first retrieve
> > __all__
> > > > >> topics's
> > > > >> > > > > > > > description info and then filter at the client end,
> > which
> > > > >> will
> > > > >> > > be a
> > > > >> > > > > > huge
> > > > >> > > > > > > > burden on ZK.
> > > > >> > > > > > > > AND
> > > > >> > > > > > > > 14. Config K-Vs in create topic: this is related to
> > the
> > > > >> previous
> > > > >> > > > > point;
> > > > >> > > > > > > > maybe we can add another metadata K-V or just a
> > metadata
> > > > >> string
> > > > >> > > > along
> > > > >> > > > > > > side
> > > > >> > > > > > > > with config K-V in create topic like we did for
> offset
> > > > >> commit
> > > > >> > > > > request.
> > > > >> > > > > > > This
> > > > >> > > > > > > > field can be quite useful in storing information
> like
> > > > >> "owner" of
> > > > >> > > > the
> > > > >> > > > > > > topic
> > > > >> > > > > > > > who issue the create command, etc, which is quite
> > > > important
> > > > >> for a
> > > > >> > > > > > > > multi-tenant setting. Then in the describe topic
> > request
> > > > we
> > > > >> can
> > > > >> > > > also
> > > > >> > > > > > > batch
> > > > >> > > > > > > > on regex of the metadata field.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: As discussed it is very interesting but can be
> > > > >> implemented
> > > > >> > > later
> > > > >> > > > > > after
> > > > >> > > > > > > > we have some basic functionality there.
> > > > >> > > > > > > >
> > > > >> > > > > > > > 15. Today all the admin operations are async in the
> > sense
> > > > >> that
> > > > >> > > > > command
> > > > >> > > > > > > will
> > > > >> > > > > > > > return once it is written in ZK, and that is why we
> > need
> > > > >> extra
> > > > >> > > > > > > verification
> > > > >> > > > > > > > like testUtil.waitForTopicCreated() / verify
> partition
> > > > >> > > reassignment
> > > > >> > > > > > > > request, etc. With admin requests we could add a
> flag
> > to
> > > > >> enable /
> > > > >> > > > > > disable
> > > > >> > > > > > > > synchronous requests; when it is turned on, the
> > response
> > > > >> will not
> > > > >> > > > > > return
> > > > >> > > > > > > > until the request has been completed. And for async
> > > > >> requests we
> > > > >> > > can
> > > > >> > > > > > add a
> > > > >> > > > > > > > "token" field in the response, and then only need a
> > > > general
> > > > >> > > "admin
> > > > >> > > > > > > > verification request" with the given token to check
> > if the
> > > > >> async
> > > > >> > > > > > request
> > > > >> > > > > > > > has been completed.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: I see your point. My idea was to provide specific
> > > > >> > > > Verify...Request
> > > > >> > > > > > per
> > > > >> > > > > > > > each
> > > > >> > > > > > > > long running request, where needed. We can do it the
> > way
> > > > you
> > > > >> > > > suggest.
> > > > >> > > > > > The
> > > > >> > > > > > > > only
> > > > >> > > > > > > > concern is that introducing a token we again will
> make
> > > > >> schema
> > > > >> > > > > > "dynamic".
> > > > >> > > > > > > We
> > > > >> > > > > > > > wanted
> > > > >> > > > > > > > to do similar thing introducing single AdminRequest
> > for
> > > > all
> > > > >> topic
> > > > >> > > > > > > commands
> > > > >> > > > > > > > but rejected
> > > > >> > > > > > > > this idea because we wanted to have schema defined.
> So
> > > > this
> > > > >> is
> > > > >> > > > more a
> > > > >> > > > > > > > choice between:
> > > > >> > > > > > > > a) have fixed schema but introduce each time new
> > > > >> Verify...Request
> > > > >> > > > for
> > > > >> > > > > > > > long-running requests
> > > > >> > > > > > > > b) use one request for verification but generalize
> it
> > with
> > > > >> token
> > > > >> > > > > > > > I'm fine with whatever decision community come to.
> > Just
> > > > let
> > > > >> me
> > > > >> > > know
> > > > >> > > > > > your
> > > > >> > > > > > > > thoughts.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Comment from Gwen:
> > > > >> > > > > > > > 16. Specifically for ownership, I think the plan is
> > to add
> > > > >> ACL
> > > > >> > > (it
> > > > >> > > > > > sounds
> > > > >> > > > > > > > like you are describing ACL) via an external system
> > > > (Argus,
> > > > >> > > > Sentry).
> > > > >> > > > > > > > I remember KIP-11 described this, but I can't find
> > the KIP
> > > > >> any
> > > > >> > > > > longer.
> > > > >> > > > > > > >
> > > > >> > > > > > > > A: Okay, no problem. Not sure though how we are
> going
> > to
> > > > >> handle
> > > > >> > > it.
> > > > >> > > > > > Wait
> > > > >> > > > > > > > which KIP
> > > > >> > > > > > > > will be committed first and include changes to
> > > > >> TopicMetadata from
> > > > >> > > > the
> > > > >> > > > > > > later
> > > > >> > > > > > > > one?
> > > > >> > > > > > > > Anyway, I added this note to "Open Questions"
> section
> > so
> > > > we
> > > > >> don't
> > > > >> > > > > miss
> > > > >> > > > > > > this
> > > > >> > > > > > > > piece.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Thanks,
> > > > >> > > > > > > > Andrii Biletskyi
> > > > >> > > > > > > >
> > > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > > > > > > >
> > > > >> > > > > > > > > Hi all,
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > Today I uploaded the patch that covers some of the
> > > > >> discussed
> > > > >> > > and
> > > > >> > > > > > agreed
> > > > >> > > > > > > > > items:
> > > > >> > > > > > > > > - removed MaybeOf optional type
> > > > >> > > > > > > > > - switched to java protocol definitions
> > > > >> > > > > > > > > - simplified messages (normalized configs, removed
> > topic
> > > > >> marked
> > > > >> > > > for
> > > > >> > > > > > > > > deletion)
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > I also updated the KIP-4 with respective changes
> and
> > > > >> wrote down
> > > > >> > > > my
> > > > >> > > > > > > > > proposal for
> > > > >> > > > > > > > > pending items:
> > > > >> > > > > > > > > - Batch Admin Operations -> updated Wire Protocol
> > schema
> > > > >> > > proposal
> > > > >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
> > > > >> > > > TopicMetadataRequest
> > > > >> > > > > > > > > - Admin Client -> updated my initial proposal to
> > reflect
> > > > >> > > batching
> > > > >> > > > > > > > > - Error codes -> proposed fine-grained error code
> > > > instead
> > > > >> of
> > > > >> > > > > > > > > AdminRequestFailed
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > I will also send a separate email to cover all
> > comments
> > > > >> from
> > > > >> > > this
> > > > >> > > > > > > thread.
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > Thanks,
> > > > >> > > > > > > > > Andrii Biletskyi
> > > > >> > > > > > > > >
> > > > >> > > > > > > > >
> > > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> > > > >> > > > > gshapira@cloudera.com
> > > > >> > > > > > >
> > > > >> > > > > > > > > wrote:
> > > > >> > > > > > > > >
> > > > >> > > > > > > > >> Found KIP-11 (
> > > > >> > > > > > > > >>
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >>
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > >> > > > > > > > >> )
> > > > >> > > > > > > > >> It actually specifies changes to the Metadata
> > protocol,
> > > > >> so
> > > > >> > > > making
> > > > >> > > > > > sure
> > > > >> > > > > > > > >> both KIPs are consistent in this regard will be
> > good.
> > > > >> > > > > > > > >>
> > > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> > > > >> > > > > > gshapira@cloudera.com
> > > > >> > > > > > > >
> > > > >> > > > > > > > >> wrote:
> > > > >> > > > > > > > >> > Specifically for ownership, I think the plan is
> > to
> > > > add
> > > > >> ACL
> > > > >> > > (it
> > > > >> > > > > > > sounds
> > > > >> > > > > > > > >> > like you are describing ACL) via an external
> > system
> > > > >> (Argus,
> > > > >> > > > > > Sentry).
> > > > >> > > > > > > > >> > I remember KIP-11 described this, but I can't
> > find
> > > > the
> > > > >> KIP
> > > > >> > > any
> > > > >> > > > > > > longer.
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting
> > > > >> information
> > > > >> > > that
> > > > >> > > > > > > already
> > > > >> > > > > > > > >> > exists from Kafka brokers, not on adding
> > information
> > > > >> that
> > > > >> > > > > perhaps
> > > > >> > > > > > > > >> > should exist but doesn't yet?
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> > Gwen
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> >
> > > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang
> <
> > > > >> > > > > > wangguoz@gmail.com>
> > > > >> > > > > > > > >> wrote:
> > > > >> > > > > > > > >> >> Folks,
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Just want to elaborate a bit more on the
> > > > create-topic
> > > > >> > > > metadata
> > > > >> > > > > > and
> > > > >> > > > > > > > >> batching
> > > > >> > > > > > > > >> >> describe-topic based on config / metadata in
> my
> > > > >> previous
> > > > >> > > > email
> > > > >> > > > > as
> > > > >> > > > > > > we
> > > > >> > > > > > > > >> work
> > > > >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to have
> > some
> > > > >> sort of
> > > > >> > > > > topic
> > > > >> > > > > > > > >> management
> > > > >> > > > > > > > >> >> mechanisms, which I think is quite important
> in
> > a
> > > > >> > > > multi-tenant
> > > > >> > > > > /
> > > > >> > > > > > > > cloud
> > > > >> > > > > > > > >> >> architecture: today anyone can create topics
> in
> > a
> > > > >> shared
> > > > >> > > > Kafka
> > > > >> > > > > > > > >> cluster, but
> > > > >> > > > > > > > >> >> there is no concept or "ownership" of topics
> > that
> > > > are
> > > > >> > > created
> > > > >> > > > > by
> > > > >> > > > > > > > >> different
> > > > >> > > > > > > > >> >> users. For example, at LinkedIn we basically
> > > > >> distinguish
> > > > >> > > > topic
> > > > >> > > > > > > owners
> > > > >> > > > > > > > >> via
> > > > >> > > > > > > > >> >> some casual topic name prefix, which is a bit
> > > > awkward
> > > > >> and
> > > > >> > > > does
> > > > >> > > > > > not
> > > > >> > > > > > > > fly
> > > > >> > > > > > > > >> as
> > > > >> > > > > > > > >> >> we scale our customers. It would be great to
> use
> > > > >> > > > > describe-topics
> > > > >> > > > > > > such
> > > > >> > > > > > > > >> as:
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Describe all topics that is created by me.
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Describe all topics whose retention time is
> > > > overriden
> > > > >> to X.
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Describe all topics whose writable group
> include
> > > > user
> > > > >> Y
> > > > >> > > (this
> > > > >> > > > > is
> > > > >> > > > > > > > >> related to
> > > > >> > > > > > > > >> >> authorization), etc..
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> One possible way to achieve this is to add a
> > > > metadata
> > > > >> file
> > > > >> > > in
> > > > >> > > > > the
> > > > >> > > > > > > > >> >> create-topic request, whose value will also be
> > > > >> written ZK
> > > > >> > > as
> > > > >> > > > we
> > > > >> > > > > > > > create
> > > > >> > > > > > > > >> the
> > > > >> > > > > > > > >> >> topic; then describe-topics can choose to
> batch
> > > > topics
> > > > >> > > based
> > > > >> > > > on
> > > > >> > > > > > 1)
> > > > >> > > > > > > > name
> > > > >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata
> > regex,
> > > > etc.
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Thoughts?
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> Guozhang
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang
> <
> > > > >> > > > > > wangguoz@gmail.com>
> > > > >> > > > > > > > >> wrote:
> > > > >> > > > > > > > >> >>
> > > > >> > > > > > > > >> >>> Thanks for the updated wiki. A few comments
> > below:
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> 1. Error description in response: I think if
> > some
> > > > >> > > errorCode
> > > > >> > > > > > could
> > > > >> > > > > > > > >> indicate
> > > > >> > > > > > > > >> >>> several different error cases then we should
> > really
> > > > >> change
> > > > >> > > > it
> > > > >> > > > > to
> > > > >> > > > > > > > >> multiple
> > > > >> > > > > > > > >> >>> codes. In general the errorCode itself would
> be
> > > > >> precise
> > > > >> > > and
> > > > >> > > > > > > > >> sufficient for
> > > > >> > > > > > > > >> >>> describing the server side errors.
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> 2. Describe topic request: it would be great
> > to go
> > > > >> beyond
> > > > >> > > > just
> > > > >> > > > > > > > >> batching on
> > > > >> > > > > > > > >> >>> topic name regex for this request. For
> > example, a
> > > > >> very
> > > > >> > > > common
> > > > >> > > > > > use
> > > > >> > > > > > > > >> case of
> > > > >> > > > > > > > >> >>> the topic command is to list all topics whose
> > > > config
> > > > >> A's
> > > > >> > > > value
> > > > >> > > > > > is
> > > > >> > > > > > > B.
> > > > >> > > > > > > > >> With
> > > > >> > > > > > > > >> >>> topic name regex then we have to first
> retrieve
> > > > >> __all__
> > > > >> > > > > topics's
> > > > >> > > > > > > > >> >>> description info and then filter at the
> client
> > end,
> > > > >> which
> > > > >> > > > will
> > > > >> > > > > > be
> > > > >> > > > > > > a
> > > > >> > > > > > > > >> huge
> > > > >> > > > > > > > >> >>> burden on ZK.
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is
> > related to
> > > > >> the
> > > > >> > > > > previous
> > > > >> > > > > > > > point;
> > > > >> > > > > > > > >> >>> maybe we can add another metadata K-V or
> just a
> > > > >> metadata
> > > > >> > > > > string
> > > > >> > > > > > > > along
> > > > >> > > > > > > > >> side
> > > > >> > > > > > > > >> >>> with config K-V in create topic like we did
> for
> > > > >> offset
> > > > >> > > > commit
> > > > >> > > > > > > > >> request. This
> > > > >> > > > > > > > >> >>> field can be quite useful in storing
> > information
> > > > like
> > > > >> > > > "owner"
> > > > >> > > > > of
> > > > >> > > > > > > the
> > > > >> > > > > > > > >> topic
> > > > >> > > > > > > > >> >>> who issue the create command, etc, which is
> > quite
> > > > >> > > important
> > > > >> > > > > for
> > > > >> > > > > > a
> > > > >> > > > > > > > >> >>> multi-tenant setting. Then in the describe
> > topic
> > > > >> request
> > > > >> > > we
> > > > >> > > > > can
> > > > >> > > > > > > also
> > > > >> > > > > > > > >> batch
> > > > >> > > > > > > > >> >>> on regex of the metadata field.
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> 4. Today all the admin operations are async
> in
> > the
> > > > >> sense
> > > > >> > > > that
> > > > >> > > > > > > > command
> > > > >> > > > > > > > >> will
> > > > >> > > > > > > > >> >>> return once it is written in ZK, and that is
> > why we
> > > > >> need
> > > > >> > > > extra
> > > > >> > > > > > > > >> verification
> > > > >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify
> > > > >> partition
> > > > >> > > > > > > reassignment
> > > > >> > > > > > > > >> >>> request, etc. With admin requests we could
> add
> > a
> > > > >> flag to
> > > > >> > > > > enable
> > > > >> > > > > > /
> > > > >> > > > > > > > >> disable
> > > > >> > > > > > > > >> >>> synchronous requests; when it is turned on,
> the
> > > > >> response
> > > > >> > > > will
> > > > >> > > > > > not
> > > > >> > > > > > > > >> return
> > > > >> > > > > > > > >> >>> until the request has been completed. And for
> > async
> > > > >> > > requests
> > > > >> > > > > we
> > > > >> > > > > > > can
> > > > >> > > > > > > > >> add a
> > > > >> > > > > > > > >> >>> "token" field in the response, and then only
> > need a
> > > > >> > > general
> > > > >> > > > > > "admin
> > > > >> > > > > > > > >> >>> verification request" with the given token to
> > check
> > > > >> if the
> > > > >> > > > > async
> > > > >> > > > > > > > >> request
> > > > >> > > > > > > > >> >>> has been completed.
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> 5. +1 for extending Metadata request to
> include
> > > > >> > > controller /
> > > > >> > > > > > > > >> coordinator
> > > > >> > > > > > > > >> >>> information, and then we can remove the
> > > > >> ConsumerMetadata /
> > > > >> > > > > > > > >> ClusterMetadata
> > > > >> > > > > > > > >> >>> requests.
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> Guozhang
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> > > > >> > > > > > jjkoshy.w@gmail.com>
> > > > >> > > > > > > > >> wrote:
> > > > >> > > > > > > > >> >>>
> > > > >> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't
> > think I
> > > > >> will be
> > > > >> > > > > able
> > > > >> > > > > > to
> > > > >> > > > > > > > >> make
> > > > >> > > > > > > > >> >>>> it today, so if notes can be sent out
> > afterward
> > > > that
> > > > >> > > would
> > > > >> > > > be
> > > > >> > > > > > > > great.
> > > > >> > > > > > > > >> >>>>
> > > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800,
> Gwen
> > > > >> Shapira
> > > > >> > > > wrote:
> > > > >> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking
> > forward
> > > > >> to
> > > > >> > > > > chatting
> > > > >> > > > > > > with
> > > > >> > > > > > > > >> >>>> everyone :)
> > > > >> > > > > > > > >> >>>> >
> > > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein
> <
> > > > >> > > > > > > joe.stein@stealth.ly>
> > > > >> > > > > > > > >> wrote:
> > > > >> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout
> > invite
> > > > to
> > > > >> all
> > > > >> > > > pmc,
> > > > >> > > > > > > > >> committers
> > > > >> > > > > > > > >> >>>> and
> > > > >> > > > > > > > >> >>>> > > everyone I found working on a KIP. If I
> > missed
> > > > >> anyone
> > > > >> > > > in
> > > > >> > > > > > the
> > > > >> > > > > > > > >> invite
> > > > >> > > > > > > > >> >>>> please
> > > > >> > > > > > > > >> >>>> > > let me know and can update it, np.
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm
> > Eastern
> > > > >> Time.
> > > > >> > > > Maybe
> > > > >> > > > > > we
> > > > >> > > > > > > > can
> > > > >> > > > > > > > >> get
> > > > >> > > > > > > > >> >>>> INFRA
> > > > >> > > > > > > > >> >>>> > > help to make a google account so we can
> > manage
> > > > >> > > better?
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>> > > To discuss
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>>
> > > > >> > > > > > > > >>
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >>
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > >> > > > > > > > >> >>>> > > in progress and related JIRA that are
> > > > >> interdependent
> > > > >> > > > and
> > > > >> > > > > > > common
> > > > >> > > > > > > > >> work.
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay
> > Kreps <
> > > > >> > > > > > > > jay.kreps@gmail.com>
> > > > >> > > > > > > > >> >>>> wrote:
> > > > >> > > > > > > > >> >>>> > >
> > > > >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that will
> > also
> > > > >> record
> > > > >> > > > and
> > > > >> > > > > > make
> > > > >> > > > > > > > the
> > > > >> > > > > > > > >> >>>> sessions
> > > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > > >> > > > > > > > >> >>>> > >>
> > > > >> > > > > > > > >> >>>> > >> -Jay
> > > > >> > > > > > > > >> >>>> > >>
> > > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff
> > > > Holoman
> > > > >> <
> > > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > >> > > > > > > > >> >>>> > >>
> > > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for
> > this
> > > > >> purpose.
> > > > >> > > We
> > > > >> > > > > > could
> > > > >> > > > > > > > >> record
> > > > >> > > > > > > > >> >>>> the
> > > > >> > > > > > > > >> >>>> > >> > sessions if there is interest and
> > publish
> > > > >> them
> > > > >> > > out.
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >> > Thanks
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >> > Jeff
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay
> > > > Kreps <
> > > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > > >> > > > > > > > >> >>>> wrote:
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >> > > Let's try to get the technical
> > hang-ups
> > > > >> sorted
> > > > >> > > > out,
> > > > >> > > > > > > > though.
> > > > >> > > > > > > > >> I
> > > > >> > > > > > > > >> >>>> really
> > > > >> > > > > > > > >> >>>> > >> > think
> > > > >> > > > > > > > >> >>>> > >> > > there is some benefit to live
> > discussion
> > > > vs
> > > > >> > > > > writing. I
> > > > >> > > > > > > am
> > > > >> > > > > > > > >> >>>> hopeful that
> > > > >> > > > > > > > >> >>>> > >> if
> > > > >> > > > > > > > >> >>>> > >> > > we post instructions and give
> > ourselves a
> > > > >> few
> > > > >> > > > > attempts
> > > > >> > > > > > > we
> > > > >> > > > > > > > >> can
> > > > >> > > > > > > > >> >>>> get it
> > > > >> > > > > > > > >> >>>> > >> > > working.
> > > > >> > > > > > > > >> >>>> > >> > >
> > > > >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work for
> > > > >> me...any
> > > > >> > > > > > objections?
> > > > >> > > > > > > > >> >>>> > >> > >
> > > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > > >> > > > > > > > >> >>>> > >> > >
> > > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM,
> Joe
> > > > Stein
> > > > >> <
> > > > >> > > > > > > > >> joe.stein@stealth.ly
> > > > >> > > > > > > > >> >>>> >
> > > > >> > > > > > > > >> >>>> > >> wrote:
> > > > >> > > > > > > > >> >>>> > >> > >
> > > > >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe like
> > every
> > > > >> > > Tuesday ~
> > > > >> > > > > 1pm
> > > > >> > > > > > > ET
> > > > >> > > > > > > > /
> > > > >> > > > > > > > >> 10am
> > > > >> > > > > > > > >> >>>> PT
> > > > >> > > > > > > > >> >>>> > >> ????
> > > > >> > > > > > > > >> >>>> > >> > > >
> > > > >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but
> > there
> > > > is
> > > > >> > > always
> > > > >> > > > > some
> > > > >> > > > > > > > >> issue or
> > > > >> > > > > > > > >> >>>> > >> whatever
> > > > >> > > > > > > > >> >>>> > >> > so
> > > > >> > > > > > > > >> >>>> > >> > > > we know the apache irc channel
> > works.
> > > > We
> > > > >> can
> > > > >> > > > start
> > > > >> > > > > > > there
> > > > >> > > > > > > > >> and
> > > > >> > > > > > > > >> >>>> see how
> > > > >> > > > > > > > >> >>>> > >> it
> > > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too
> > and
> > > > >> > > associate
> > > > >> > > > to
> > > > >> > > > > > > > >> tickets if
> > > > >> > > > > > > > >> >>>> need be
> > > > >> > > > > > > > >> >>>> > >> > > makes
> > > > >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> > > > >> > > > > > > > >> >>>> > >> > > >
> > > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > >> > > > > > > > >> >>>> > >> > > >
> > > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM,
> > Jay
> > > > >> Kreps <
> > > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > >> > > > > > > > >> >>>> > >> > wrote:
> > > > >> > > > > > > > >> >>>> > >> > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a
> Google
> > > > >> Hangout to
> > > > >> > > > chat
> > > > >> > > > > > > about
> > > > >> > > > > > > > >> this.
> > > > >> > > > > > > > >> >>>> What
> > > > >> > > > > > > > >> >>>> > >> > about
> > > > >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
> > > > further...I
> > > > >> > > > actually
> > > > >> > > > > > > think
> > > > >> > > > > > > > it
> > > > >> > > > > > > > >> >>>> would be
> > > > >> > > > > > > > >> >>>> > >> > good
> > > > >> > > > > > > > >> >>>> > >> > > > for
> > > > >> > > > > > > > >> >>>> > >> > > > > everyone spending a reasonable
> > chunk
> > > > of
> > > > >> > > their
> > > > >> > > > > week
> > > > >> > > > > > > on
> > > > >> > > > > > > > >> Kafka
> > > > >> > > > > > > > >> >>>> stuff
> > > > >> > > > > > > > >> >>>> > >> to
> > > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > > >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think we
> > could
> > > > >> use
> > > > >> > > time
> > > > >> > > > > to
> > > > >> > > > > > > talk
> > > > >> > > > > > > > >> >>>> through
> > > > >> > > > > > > > >> >>>> > >> design
> > > > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top
> of
> > > > code
> > > > >> > > > reviews,
> > > > >> > > > > > talk
> > > > >> > > > > > > > >> through
> > > > >> > > > > > > > >> >>>> any
> > > > >> > > > > > > > >> >>>> > >> > tricky
> > > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > We can make it publicly
> > available so
> > > > >> that
> > > > >> > > any
> > > > >> > > > > one
> > > > >> > > > > > > can
> > > > >> > > > > > > > >> follow
> > > > >> > > > > > > > >> >>>> along
> > > > >> > > > > > > > >> >>>> > >> > who
> > > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this? If
> so
> > > > I'll
> > > > >> try
> > > > >> > > to
> > > > >> > > > > set
> > > > >> > > > > > it
> > > > >> > > > > > > > up
> > > > >> > > > > > > > >> >>>> starting
> > > > >> > > > > > > > >> >>>> > >> next
> > > > >> > > > > > > > >> >>>> > >> > > > week.
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57
> AM,
> > > > Andrii
> > > > >> > > > > Biletskyi
> > > > >> > > > > > <
> > > > >> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly>
> > wrote:
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed
> /
> > > > >> aligned
> > > > >> > > > > document
> > > > >> > > > > > > > >> structure.
> > > > >> > > > > > > > >> >>>> Also I
> > > > >> > > > > > > > >> >>>> > >> > > added
> > > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > > >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
> > > > >> AdminClient so
> > > > >> > > we
> > > > >> > > > > have
> > > > >> > > > > > > > >> something
> > > > >> > > > > > > > >> >>>> to
> > > > >> > > > > > > > >> >>>> > >> start
> > > > >> > > > > > > > >> >>>> > >> > > > from
> > > > >> > > > > > > > >> >>>> > >> > > > > > while
> > > > >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > >
> > > > >> > > > > > > > >> >>>> > >> > > >
> > > > >> > > > > > > > >> >>>> > >> > >
> > > > >> > > > > > > > >> >>>> > >> >
> > > > >> > > > > > > > >> >>>> > >>
> > > > >> > > > > > > > >> >>>>
> > > > >> > > > > > > > >>
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >>
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > > > >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01
> > PM,
> > > > >> Andrii
> > > > >> > > > > > Biletskyi
> > > > >> > > > > > > <
> > > > >> > > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly>
> > > > wrote:
> > > > >> > > > > > > > >> >>>> > >> > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you are
> > right,
> > > > >> in
> > > > >> > > most
> > > > >> > > > > > cases
> > > > >> > > > > > > > >> client
> > > > >> > > > > > > > >> >>>> will
> > > > >> > > > > > > > >> >>>> > >> have
> > > > >> > > > > > > > >> >>>> > >> > > > enough
> > > > >> > > > > > > > >> >>>> > >> > > > > > > context to show descriptive
> > error
> > > > >> > > message.
> > > > >> > > > > My
> > > > >> > > > > > > > >> concern is
> > > > >> > > > > > > > >> >>>> that
> > > > >> > > > > > > > >> >>>> > >> we
> > > > >> > > > > > > > >> >>>> > >> > > will
> > > > >> > > > > > > > >> >>>> > >> > > > > > have
> > > > >> > > > > > > > >> >>>> > >> > > > > > > to
> > > > >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error codes
> > for
> > > > >> each
> > > > >> > > > > possible
> > > > >> > > > > > > > >> error. Of
> > > > >> > > > > > > > >> >>>> course,
> > > > >> > > > > > > > >> >>>> > >> > we
> > > > >> > > > > > > > >> >>>> > >> > > > > could
> > > > >> > > > > > > > >> >>>> > >> > > > > > > reuse
> > > > >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
> > > > >> > > > > > > UknownTopicOrPartitionCode,
> > > > >> > > > > > > > >> but we
> > > > >> > > > > > > > >> >>>> will
> > > > >> > > > > > > > >> >>>> > >> > also
> > > > >> > > > > > > > >> >>>> > >> > > > need
> > > > >> > > > > > > > >> >>>> > >> > > > > > to
> > > > >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> > > > >> TopicAlreadyExistsCode,
> > > > >> > > > > > > > >> >>>> TopicConfigInvalid (both
> > > > >> > > > > > > > >> >>>> > >> > for
> > > > >> > > > > > > > >> >>>> > >> > > > > topic
> > > > >> > > > > > > > >> >>>> > >> > > > > > > name and config, and
> probably
> > > > user
> > > > >> would
> > > > >> > > > > like
> > > > >> > > > > > to
> > > > >> > > > > > > > >> know
> > > > >> > > > > > > > >> >>>> what
> > > > >> > > > > > > > >> >>>> > >> > exactly
> > > > >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
> > > > >> > > > > > > InvalidReplicaAssignment,
> > > > >> > > > > > > > >> >>>> > >> InternalError
> > > > >> > > > > > > > >> >>>> > >> > > > (e.g.
> > > > >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > > >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
> > > > TopicCommand,
> > > > >> we
> > > > >> > > will
> > > > >> > > > > > also
> > > > >> > > > > > > > >> need to
> > > > >> > > > > > > > >> >>>> add
> > > > >> > > > > > > > >> >>>> > >> > similar
> > > > >> > > > > > > > >> >>>> > >> > > > > stuff
> > > > >> > > > > > > > >> >>>> > >> > > > > > > for
> > > > >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> > > > >> PreferredReplica. So
> > > > >> > > > > we'll
> > > > >> > > > > > > end
> > > > >> > > > > > > > >> up
> > > > >> > > > > > > > >> >>>> with a
> > > > >> > > > > > > > >> >>>> > >> > large
> > > > >> > > > > > > > >> >>>> > >> > > > list
> > > > >> > > > > > > > >> >>>> > >> > > > > > of
> > > > >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in
> > Admin
> > > > >> > > protocol.
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree
> my
> > > > >> proposal is
> > > > >> > > > not
> > > > >> > > > > > > > >> consistent
> > > > >> > > > > > > > >> >>>> with
> > > > >> > > > > > > > >> >>>> > >> > other
> > > > >> > > > > > > > >> >>>> > >> > > > > cases.
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better
> > solution
> > > > >> or
> > > > >> > > > > something
> > > > >> > > > > > > > >> >>>> in-between.
> > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it
> > is a
> > > > >> great
> > > > >> > > > idea.
> > > > >> > > > > > > This
> > > > >> > > > > > > > >> way we
> > > > >> > > > > > > > >> >>>> can
> > > > >> > > > > > > > >> >>>> > >> move
> > > > >> > > > > > > > >> >>>> > >> > > on
> > > > >> > > > > > > > >> >>>> > >> > > > > > > faster.
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on
> > date/time
> > > > so
> > > > >> > > people
> > > > >> > > > > can
> > > > >> > > > > > > > join.
> > > > >> > > > > > > > >> >>>> Will work
> > > > >> > > > > > > > >> >>>> > >> > for
> > > > >> > > > > > > > >> >>>> > >> > > me
> > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > >> > > > > > > > >> >>>> > >> > > > > > > and
> > > > >> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime if
> > > > agreed
> > > > >> in
> > > > >> > > > > advance.
> > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > > >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at
> 7:09
> > PM,
> > > > >> Jay
> > > > >> > > > Kreps <
> > > > >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > >> > > > > > > > >> >>>> > >> > > > > wrote:
> > > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good
> > error
> > > > >> handling
> > > > >> > > > > > without
> > > > >> > > > > > > > >> needing
> > > > >> > > > > > > > >> >>>> custom
> > > > >> > > > > > > > >> >>>> > >> > > > > > server-side
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally
> the
> > > > >> client has
> > > > >> > > > the
> > > > >> > > > > > > > >> context to
> > > > >> > > > > > > > >> >>>> know
> > > > >> > > > > > > > >> >>>> > >> that
> > > > >> > > > > > > > >> >>>> > >> > > if
> > > > >> > > > > > > > >> >>>> > >> > > > it
> > > > >> > > > > > > > >> >>>> > >> > > > > > got
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic
> > doesn't
> > > > >> exist
> > > > >> > > to
> > > > >> > > > > say
> > > > >> > > > > > > > >> "Topic X
> > > > >> > > > > > > > >> >>>> doesn't
> > > > >> > > > > > > > >> >>>> > >> > > exist"
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> rather
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or
> > > > >> whatever).
> > > > >> > > Maybe
> > > > >> > > > > > there
> > > > >> > > > > > > > are
> > > > >> > > > > > > > >> >>>> specific
> > > > >> > > > > > > > >> >>>> > >> > cases
> > > > >> > > > > > > > >> >>>> > >> > > > > where
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want
> to
> > add
> > > > >> > > > server-side
> > > > >> > > > > > > error
> > > > >> > > > > > > > >> >>>> messages we
> > > > >> > > > > > > > >> >>>> > >> > > really
> > > > >> > > > > > > > >> >>>> > >> > > > > do
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a
> > consistent
> > > > >> way
> > > > >> > > > across
> > > > >> > > > > > the
> > > > >> > > > > > > > >> protocol.
> > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of
> open
> > > > >> questions
> > > > >> > > > here
> > > > >> > > > > > > from
> > > > >> > > > > > > > my
> > > > >> > > > > > > > >> >>>> previous
> > > > >> > > > > > > > >> >>>> > >> > > list. I
> > > > >> > > > > > > > >> >>>> > >> > > > > > will
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few
> > days for
> > > > >> Strata
> > > > >> > > > > > though.
> > > > >> > > > > > > > >> Maybe
> > > > >> > > > > > > > >> >>>> we could
> > > > >> > > > > > > > >> >>>> > >> > do
> > > > >> > > > > > > > >> >>>> > >> > > a
> > > > >> > > > > > > > >> >>>> > >> > > > > > Google
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open
> > issues
> > > > >> some
> > > > >> > > time
> > > > >> > > > > > > towards
> > > > >> > > > > > > > >> the
> > > > >> > > > > > > > >> >>>> end of
> > > > >> > > > > > > > >> >>>> > >> > next
> > > > >> > > > > > > > >> >>>> > >> > > > week
> > > > >> > > > > > > > >> >>>> > >> > > > > > for
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in this
> > > > ticket?
> > > > >> I
> > > > >> > > have
> > > > >> > > > a
> > > > >> > > > > > > > feeling
> > > > >> > > > > > > > >> that
> > > > >> > > > > > > > >> >>>> might
> > > > >> > > > > > > > >> >>>> > >> > > > progress
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster
> than
> > > > >> email--I
> > > > >> > > > think
> > > > >> > > > > we
> > > > >> > > > > > > > >> could talk
> > > > >> > > > > > > > >> >>>> > >> through
> > > > >> > > > > > > > >> >>>> > >> > > > those
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly
> > > > >> quickly...
> > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at
> > 7:27 AM,
> > > > >> Andrii
> > > > >> > > > > > > > Biletskyi <
> > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> andrii.biletskyi@stealth.ly
> > >
> > > > >> wrote:
> > > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address
> > some of
> > > > >> the
> > > > >> > > > issues
> > > > >> > > > > > > which
> > > > >> > > > > > > > >> were
> > > > >> > > > > > > > >> >>>> > >> mentioned
> > > > >> > > > > > > > >> >>>> > >> > > > > earlier
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> about
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One
> of
> > > > >> those was
> > > > >> > > > > about
> > > > >> > > > > > > > >> batching
> > > > >> > > > > > > > >> >>>> > >> > operations.
> > > > >> > > > > > > > >> >>>> > >> > > > What
> > > > >> > > > > > > > >> >>>> > >> > > > > > if
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> we
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand
> > approach
> > > > >> and let
> > > > >> > > > > people
> > > > >> > > > > > > > >> specify
> > > > >> > > > > > > > >> >>>> > >> topic-name
> > > > >> > > > > > > > >> >>>> > >> > > by
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most of
> > the
> > > > use
> > > > >> > > cases?
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what
> > information
> > > > >> should
> > > > >> > > we
> > > > >> > > > > > > > generally
> > > > >> > > > > > > > >> >>>> provide in
> > > > >> > > > > > > > >> >>>> > >> > > Admin
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin
> > commands
> > > > >> don't
> > > > >> > > > imply
> > > > >> > > > > > > they
> > > > >> > > > > > > > >> will
> > > > >> > > > > > > > >> >>>> be used
> > > > >> > > > > > > > >> >>>> > >> > only
> > > > >> > > > > > > > >> >>>> > >> > > > in
> > > > >> > > > > > > > >> >>>> > >> > > > > > CLI
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > but,
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a
> > very
> > > > >> > > important
> > > > >> > > > > > > client
> > > > >> > > > > > > > >> of this
> > > > >> > > > > > > > >> >>>> > >> > feature.
> > > > >> > > > > > > > >> >>>> > >> > > In
> > > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > case,
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would
> > like
> > > > to
> > > > >> > > provide
> > > > >> > > > > > users
> > > > >> > > > > > > > >> with
> > > > >> > > > > > > > >> >>>> rich
> > > > >> > > > > > > > >> >>>> > >> > > experience
> > > > >> > > > > > > > >> >>>> > >> > > > > in
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> terms
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > of
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > getting results / errors
> > of
> > > > the
> > > > >> > > > executed
> > > > >> > > > > > > > >> commands.
> > > > >> > > > > > > > >> >>>> Usually
> > > > >> > > > > > > > >> >>>> > >> we
> > > > >> > > > > > > > >> >>>> > >> > > > supply
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> with
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses only
> errorCode,
> > > > which
> > > > >> looks
> > > > >> > > > > very
> > > > >> > > > > > > > >> limiting,
> > > > >> > > > > > > > >> >>>> in case
> > > > >> > > > > > > > >> >>>> > >> > of
> > > > >> > > > > > > > >> >>>> > >> > > > CLI
> > > > >> > > > > > > > >> >>>> > >> > > > > we
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> may
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human
> > readable
> > > > >> error
> > > > >> > > > > > > description.
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into account
> > > > >> previous item
> > > > >> > > > > about
> > > > >> > > > > > > > >> batching,
> > > > >> > > > > > > > >> >>>> what
> > > > >> > > > > > > > >> >>>> > >> do
> > > > >> > > > > > > > >> >>>> > >> > > you
> > > > >> > > > > > > > >> >>>> > >> > > > > > think
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > about
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't
> support
> > > > >> regexp)
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest =>
> > > > TopicName
> > > > >> > > > > Partitions
> > > > >> > > > > > > > >> Replicas
> > > > >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse =>
> > > > ErrorCode
> > > > >> > > > > > > > ErrorDescription
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription =>
> > string
> > > > >> (empty
> > > > >> > > if
> > > > >> > > > > > > > >> successful)
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
> > > > >> TopicNameRegexp
> > > > >> > > >
> > > > >
> > > > > ...
> > > > >
> > > > > [Message clipped]
> > > >
> >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
Joel, Andril,

I think we agreed that those admin requests can be issued to any broker.
Because of that, there doesn't seem to be a strong need to know the
controller. So, perhaps we can proceed by not making any change to the
format of TMR right now. When we start using create topic request in the
producer, we will need a new version of TMR that doesn't trigger auto topic
creation. But that can be done later.

As a first cut implementation, I think the broker can just write to ZK
directly for
createToipic/alterTopic/reassignPartitions/preferredLeaderElection
requests, instead of forwarding them to the controller. This will simplify
the implementation on the broker side.

Thanks,

Jun


On Wed, Mar 18, 2015 at 11:58 AM, Joel Koshy <jj...@gmail.com> wrote:

> (Thanks Andrii for the summary)
>
> For (1) yes we will circle back on that shortly after syncing up in
> person. I think it is close to getting committed although development
> for KAFKA-1927 can probably begin without it.
>
> There is one more item we covered at the hangout. i.e., whether we
> want to add the coordinator to the topic metadata response or provide
> a clearer ClusterMetadataRequest.
>
> There are two reasons I think we should try and avoid adding the
> field:
> - It is irrelevant to topic metadata
> - If we finally do request rerouting in Kafka then the field would add
>   little to no value. (It still helps to have a separate
>   ClusterMetadataRequest to query for cluster-wide information such as
>   'which broker is the controller?' as Joe mentioned.)
>
> I think it would be cleaner to have an explicit ClusterMetadataRequest
> that you can send to any broker in order to obtain the controller (and
> in the future possibly other cluster-wide information). I think the
> main argument against doing this and instead adding it to the topic
> metadata response was convenience - i.e., you don't have to discover
> the controller in advance. However, I don't see much actual
> benefit/convenience in this and in fact think it is a non-issue. Let
> me know if I'm overlooking something here.
>
> As an example, say we need to initiate partition reassignment by
> issuing the new ReassignPartitionsRequest to the controller (assume we
> already have the desired manual partition assignment).  If we are to
> augment topic metadata response then the flow be something like this :
>
> - Issue topic metadata request to any broker (and discover the
>   controller
> - Connect to controller if required (i.e., if the broker above !=
>   controller)
> - Issue the partition reassignment request to the controller.
>
> With an explicit cluster metadata request it would be:
> - Issue cluster metadata request to any broker
> - Connect to controller if required (i.e., if the broker above !=
>   controller)
> - Issue the partition reassignment request
>
> So it seems to add little practical value and bloats topic metadata
> response with an irrelevant detail.
>
> The other angle to this is the following - is it a matter of naming?
> Should we just rename topic metadata request/response to just
> MetadataRequest/Response and add cluster metadata to it? By that same
> token should we also allow querying for the consumer coordinator (and
> in future transaction coordinator) as well? This leads to a bloated
> request which isn't very appealing and altogether confusing.
>
> Thanks,
>
> Joel
>
> On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> > Andri,
> >
> > Thanks for the summary.
> >
> > 1. I just realized that in order to start working on KAFKA-1927, we will
> > need to merge the changes to OffsetCommitRequest (from 0.8.2) to trunk.
> > This is planned to be done as part of KAFKA-1634. So, we will need
> Guozhang
> > and Joel's help to wrap this up.
> >
> > 2. Thinking about this a bit more, if the semantic of those "write"
> > requests is async (i.e., after the client gets a response, it just means
> > that the operation is initiated, but not necessarily completed), we don't
> > really need to forward the requests to the controller. Instead, the
> > receiving broker can just write the operation to ZK as the admin command
> > line tool previously does. This will simplify the implementation.
> >
> > 8. There is another implementation detail for describe topic. Ideally, we
> > want to read the topic config from the broker cache, instead of
> ZooKeeper.
> > Currently, every broker reads the topic-level config for all topics.
> > However, it ignores those for topics not hosted on itself. So, we may
> need
> > to change TopicConfigManager a bit so that it caches the configs for all
> > topics.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Guys,
> > >
> > > Thanks for a great discussion!
> > > Here are the actions points:
> > >
> > > 1. Q: Get rid of all scala requests objects, use java protocol
> definitions.
> > >     A: Gwen kindly took that (KAFKA-1927). It's important to speed up
> > > review procedure
> > >          there since this ticket blocks other important changes.
> > >
> > > 2. Q: Generic re-reroute facility vs client maintaining cluster state.
> > >     A: Jay has added pseudo code to KAFKA-1912 - need to consider
> whether
> > > this will be
> > >         easy to implement as a server-side feature (comments are
> > > welcomed!).
> > >
> > > 3. Q: Controller field in wire protocol.
> > >     A: This might be useful for clients, add this to
> TopicMetadataResponse
> > > (already in KIP).
> > >
> > > 4. Q: Decoupling topic creation from TMR.
> > >     A: I will add proposed by Jun solution (using clientId for that)
> to the
> > > KIP.
> > >
> > > 5. Q: Bumping new versions of TMR vs grabbing all protocol changes in
> one
> > > version.
> > >     A: It was decided to try to gather all changes to protocol (before
> > > release).
> > >         In case of TMR it worth checking: KAFKA-2020 and KIP-13
> (quotas)
> > >
> > > 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
> > >     A: Use jackson for that, /tools project is a separate jar so
> shouldn't
> > > be a big deal.
> > >
> > > 7.  Q: VerifyReassingPartitions vs generic status check command.
> > >      A: For long-running requests like reassign partitions *progress*
> check
> > > request is useful,
> > >          it makes sense to introduce it.
> > >
> > >  Please add, correct me if I missed something.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Joel,
> > > >
> > > > You are right, I removed ClusterMetadata because we have partially
> > > > what we need in TopicMetadata. Also, as Jay pointed out earlier, we
> > > > would like to have "orthogonal" API, but at the same time we need
> > > > to be backward compatible.
> > > >
> > > > But I like your idea and even have some other arguments for this
> option:
> > > > There is also DescribeTopicRequest which was proposed in this KIP,
> > > > it returns topic configs, partitions, replication factor plus
> partition
> > > > ISR, ASR,
> > > > leader replica. The later part is really already there in
> > > > TopicMetadataRequest.
> > > > So again we'll have to add stuff to TMR, not to duplicate some info
> in
> > > > newly added requests. However, this way we'll end up with "monster"
> > > > request which returns cluster metadata, topic replication and config
> info
> > > > plus partition replication data. Seems logical to split TMR to
> > > > - ClusterMetadata (brokers + controller, maybe smth else)
> > > > - TopicMetadata (topic info + partition details)
> > > > But since current TMR is involved in lots of places (including
> network
> > > > client,
> > > > as I understand) this might be very serious change and it probably
> makes
> > > > sense to stick with current approach.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > >
> > > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jj...@gmail.com>
> wrote:
> > > >
> > > >> I may be missing some context but hopefully this will also be
> covered
> > > >> today: I thought the earlier proposal where there was an explicit
> > > >> ClusterMetadata request was clearer and explicit. During the course
> of
> > > >> this thread I think the conclusion was that the main need was for
> > > >> controller information and that can be rolled into the topic
> metadata
> > > >> response but that seems a bit irrelevant to topic metadata. FWIW I
> > > >> think the full broker-list is also irrelevant to topic metadata, but
> > > >> it is already there and in use. I think there is still room for an
> > > >> explicit ClusterMetadata request since there may be other
> > > >> cluster-level information that we may want to add over time (and
> that
> > > >> have nothing to do with topic metadata).
> > > >>
> > > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
> > > >> > Jun,
> > > >> >
> > > >> > 101. Okay, if you say that such use case is important. I also
> think
> > > >> > using clientId for these purposes is fine - if we already have
> this
> > > >> field
> > > >> > as part of all Wire protocol messages, why not use that.
> > > >> > I will update KIP-4 page if nobody has other ideas (which may
> come up
> > > >> > during the call today).
> > > >> >
> > > >> > 102.1 Agree, I'll update the KIP accordingly. I think we can add
> new,
> > > >> > fine-grained error codes if some error code received in specific
> case
> > > >> > won't give enough context to return a descriptive error message
> for
> > > >> user.
> > > >> >
> > > >> > Look forward to discussing all outstanding issues in detail today
> > > during
> > > >> > the call.
> > > >> >
> > > >> > Thanks,
> > > >> > Andrii Biletskyi
> > > >> >
> > > >> >
> > > >> >
> > > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io>
> wrote:
> > > >> >
> > > >> > > 101. There may be a use case where you only want the topics to
> be
> > > >> created
> > > >> > > manually by admins. Currently, you can do that by disabling auto
> > > topic
> > > >> > > creation and issue topic creation from the TopicCommand. If we
> > > >> disable auto
> > > >> > > topic creation completely on the broker and don't have a way to
> > > >> distinguish
> > > >> > > between topic creation requests from the regular clients and the
> > > >> admin, we
> > > >> > > can't support manual topic creation any more. I was thinking
> that
> > > >> another
> > > >> > > way of distinguishing the clients making the topic creation
> requests
> > > >> is
> > > >> > > using clientId. For example, the admin tool can set it to
> something
> > > >> like
> > > >> > > admin and the broker can treat that clientId specially.
> > > >> > >
> > > >> > > Also, there is a related discussion in KAFKA-2020. Currently,
> we do
> > > >> the
> > > >> > > following in TopicMetadataResponse:
> > > >> > >
> > > >> > > 1. If leader is not available, we set the partition level error
> code
> > > >> to
> > > >> > > LeaderNotAvailable.
> > > >> > > 2. If a non-leader replica is not available, we take that
> replica
> > > out
> > > >> of
> > > >> > > the assigned replica list and isr in the response. As an
> indication
> > > >> for
> > > >> > > doing that, we set the partition level error code to
> > > >> ReplicaNotAvailable.
> > > >> > >
> > > >> > > This has a few problems. First, ReplicaNotAvailable probably
> > > >> shouldn't be
> > > >> > > an error, at least for the normal producer/consumer clients that
> > > just
> > > >> want
> > > >> > > to find out the leader. Second, it can happen that both the
> leader
> > > and
> > > >> > > another replica are not available at the same time. There is no
> > > error
> > > >> code
> > > >> > > to indicate both. Third, even if a replica is not available,
> it's
> > > >> still
> > > >> > > useful to return its replica id since some clients (e.g. admin
> tool)
> > > >> may
> > > >> > > still make use of it.
> > > >> > >
> > > >> > > One way to address this issue is to always return the replica
> id for
> > > >> > > leader, assigned replicas, and isr regardless of whether the
> > > >> corresponding
> > > >> > > broker is live or not. Since we also return the list of live
> > > brokers,
> > > >> the
> > > >> > > client can figure out whether a leader or a replica is live or
> not
> > > >> and act
> > > >> > > accordingly. This way, we don't need to set the partition level
> > > error
> > > >> code
> > > >> > > when the leader or a replica is not available. This doesn't
> change
> > > >> the wire
> > > >> > > protocol, but does change the semantics. Since we are evolving
> the
> > > >> protocol
> > > >> > > of TopicMetadataRequest here, we can potentially piggyback the
> > > change.
> > > >> > >
> > > >> > > 102.1 For those types of errors due to invalid input, shouldn't
> we
> > > >> just
> > > >> > > guard it at parameter validation time and throw
> > > >> InvalidArgumentException
> > > >> > > without even sending the request to the broker?
> > > >> > >
> > > >> > > Thanks,
> > > >> > >
> > > >> > > Jun
> > > >> > >
> > > >> > >
> > > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > > >> > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > >
> > > >> > > > Jun,
> > > >> > > >
> > > >> > > > Answering your questions:
> > > >> > > >
> > > >> > > > 101. If I understand you correctly, you are saying future
> producer
> > > >> > > versions
> > > >> > > > (which
> > > >> > > > will be ported to TMR_V1) won't be able to automatically
> create
> > > >> topic (if
> > > >> > > > we
> > > >> > > > unconditionally remove topic creation from there). But we
> need to
> > > >> this
> > > >> > > > preserve logic.
> > > >> > > > Ok, about your proposal: I'm not a big fan too, when it comes
> to
> > > >> > > > differentiating
> > > >> > > > clients directly in protocol schema. And also I'm not sure I
> > > >> understand
> > > >> > > at
> > > >> > > > all why
> > > >> > > > auto.create.topics.enable is a server side configuration. Can
> we
> > > >> > > deprecate
> > > >> > > > this setting
> > > >> > > > in future versions, add this setting to producer and based on
> that
> > > >> upon
> > > >> > > > receiving
> > > >> > > > UnknownTopic create topic explicitly by a separate producer
> call
> > > via
> > > >> > > > adminClient?
> > > >> > > >
> > > >> > > > 102.1. Hm, yes. It's because we want to support batching and
> at
> > > the
> > > >> same
> > > >> > > > time we
> > > >> > > > want to give descriptive error messages for clients. Since
> > > >> AdminClient
> > > >> > > > holds the context
> > > >> > > > to construct such messages (e.g. AdminClient layer can know
> that
> > > >> > > > InvalidArgumentsCode
> > > >> > > > means two cases: either invalid number - e.g. -1; or
> > > >> replication-factor
> > > >> > > was
> > > >> > > > provided while
> > > >> > > > partitions argument wasn't) - I wrapped responses in
> Exceptions.
> > > >> But I'm
> > > >> > > > open to any
> > > >> > > > other ideas, this was just initial version.
> > > >> > > > 102.2. Yes, I agree. I'll change that to probably some other
> dto.
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > > Andrii Biletskyi
> > > >> > > >
> > > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io>
> > > wrote:
> > > >> > > >
> > > >> > > > > Andrii,
> > > >> > > > >
> > > >> > > > > 101. That's what I was thinking too, but it may not be that
> > > >> simple. In
> > > >> > > > > TopicMetadataRequest_V1,
> > > >> > > > > we can let it not trigger auto topic creation. Then, in the
> > > >> producer
> > > >> > > > side,
> > > >> > > > > if it gets an UnknownTopicException, it can explicitly
> issue a
> > > >> > > > > createTopicRequest for auto topic creation. On the consumer
> > > side,
> > > >> it
> > > >> > > will
> > > >> > > > > never issue createTopicRequest. This works when auto topic
> > > >> creation is
> > > >> > > > > enabled on the broker side. However, I am not sure how
> things
> > > >> will work
> > > >> > > > > when auto topic creation is disabled on the broker side. In
> this
> > > >> case,
> > > >> > > we
> > > >> > > > > want to have a way to manually create a topic, potentially
> > > through
> > > >> > > admin
> > > >> > > > > commands. However, then we need a way to distinguish
> > > >> createTopicRequest
> > > >> > > > > issued from the producer clients and the admin tools. May
> be we
> > > >> can
> > > >> > > add a
> > > >> > > > > new field in createTopicRequest and set it differently in
> the
> > > >> producer
> > > >> > > > > client and the admin client. However, I am not sure if
> that's
> > > the
> > > >> best
> > > >> > > > > approach.
> > > >> > > > >
> > > >> > > > > 2. Yes, refactoring existing requests is a non-trivial
> amount of
> > > >> work.
> > > >> > > I
> > > >> > > > > posted some comments in KAFKA-1927. We will probably have
> to fix
> > > >> > > > KAFKA-1927
> > > >> > > > > first, before adding the new logic in KAFKA-1694.
> Otherwise, the
> > > >> > > changes
> > > >> > > > > will be too big.
> > > >> > > > >
> > > >> > > > > 102. About the AdminClient:
> > > >> > > > > 102.1. It's a bit weird that we return exception in the
> api. It
> > > >> seems
> > > >> > > > that
> > > >> > > > > we should either return error code or throw an exception
> when
> > > >> getting
> > > >> > > the
> > > >> > > > > response state.
> > > >> > > > > 102.2. We probably shouldn't explicitly use the request
> object
> > > in
> > > >> the
> > > >> > > > api.
> > > >> > > > > Not every request evolution requires an api change.
> > > >> > > > >
> > > >> > > > > Thanks,
> > > >> > > > >
> > > >> > > > > Jun
> > > >> > > > >
> > > >> > > > >
> > > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > > > >
> > > >> > > > > > Jun,
> > > >> > > > > >
> > > >> > > > > > Thanks for you comments. Answers inline:
> > > >> > > > > >
> > > >> > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > >> > > > > > > ReassignPartitionRequest,
> > > >> > > > > > > and PartitionsSerialized that are represented as a
> string,
> > > but
> > > >> > > > contain
> > > >> > > > > > > composite structures in json. Could we flatten them out
> > > >> directly in
> > > >> > > > the
> > > >> > > > > > > protocol definition as arrays/records?
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > Yes, now with Admin Client this looks a bit weird. My
> initial
> > > >> > > > motivation
> > > >> > > > > > was:
> > > >> > > > > > ReassignPartitionCommand accepts input in json, we want to
> > > >> remain
> > > >> > > > tools'
> > > >> > > > > > interfaces unchanged, where possible.
> > > >> > > > > > If we port it to deserialized format, in CLI (/tools
> project)
> > > >> we will
> > > >> > > > > have
> > > >> > > > > > to add some
> > > >> > > > > > json library since /tools is written in java and we'll
> need to
> > > >> > > > > deserialize
> > > >> > > > > > json file
> > > >> > > > > > provided by a user. Can we quickly agree on what this
> library
> > > >> should
> > > >> > > be
> > > >> > > > > > (Jackson, GSON, whatever)?
> > > >> > > > > >
> > > >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> > > >> creation?
> > > >> > > > This
> > > >> > > > > > > will be a bit weird now that we have a separate topic
> > > >> creation api.
> > > >> > > > > Have
> > > >> > > > > > > you thought about how the new createTopicRequest and
> > > >> > > > > TopicMetadataRequest
> > > >> > > > > > > v1 will be used in the producer/consumer client, in
> addition
> > > >> to
> > > >> > > admin
> > > >> > > > > > > tools? For example, ideally, we don't want
> > > >> TopicMetadataRequest
> > > >> > > from
> > > >> > > > > the
> > > >> > > > > > > consumer to trigger auto topic creation.
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > I agree, this strange logic should be fixed. I'm not
> confident
> > > >> in
> > > >> > > this
> > > >> > > > > > Kafka part so
> > > >> > > > > > correct me if I'm wrong, but it doesn't look like a hard
> thing
> > > >> to
> > > >> > > do, I
> > > >> > > > > > think we can
> > > >> > > > > > leverage AdminClient for that in Producer and
> unconditionally
> > > >> remove
> > > >> > > > > topic
> > > >> > > > > > creation from the TopicMetadataRequest_V1.
> > > >> > > > > >
> > > >> > > > > > 2. I think Jay meant getting rid of scala classes
> > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > >> HeartbeatResponseAndHeader. We
> > > >> > > did
> > > >> > > > > > that
> > > >> > > > > > > as a stop-gap thing when adding the new requests for the
> > > >> consumers.
> > > >> > > > > > > However, the long term plan is to get rid of all those
> and
> > > >> just
> > > >> > > reuse
> > > >> > > > > the
> > > >> > > > > > > java request/response in the client. Since this KIP
> proposes
> > > >> to
> > > >> > > add a
> > > >> > > > > > > significant number of new requests, perhaps we should
> bite
> > > the
> > > >> > > bullet
> > > >> > > > > to
> > > >> > > > > > > clean up the existing scala requests first before
> adding new
> > > >> ones?
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > > > Yes, looks like I misunderstood the point of
> > > >> ...RequestAndHeader.
> > > >> > > > Okay, I
> > > >> > > > > > will
> > > >> > > > > > rework that. The only thing is that I don't see any
> example
> > > how
> > > >> it
> > > >> > > was
> > > >> > > > > done
> > > >> > > > > > for at
> > > >> > > > > > least one existing protocol message. Thus, as I
> understand, I
> > > >> have to
> > > >> > > > > think
> > > >> > > > > > how we
> > > >> > > > > > are going to do it.
> > > >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
> > > reasonable,
> > > >> but
> > > >> > > if
> > > >> > > > > it's
> > > >> > > > > > an *obligatory*
> > > >> > > > > > requirement to have Admin KIP done, I'm afraid this can
> be a
> > > >> serious
> > > >> > > > > > blocker for us.
> > > >> > > > > > There are 13 protocol messages and all that would require
> not
> > > >> only
> > > >> > > unit
> > > >> > > > > > tests but quite
> > > >> > > > > > intensive manual testing, no? I'm afraid I'm not the
> right guy
> > > >> to
> > > >> > > cover
> > > >> > > > > > pretty much all
> > > >> > > > > > Kafka core internals :). Let me know your thoughts on this
> > > >> item. Btw
> > > >> > > > > there
> > > >> > > > > > is a ticket to
> > > >> > > > > > follow-up this issue (
> > > >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > >> > > > ).
> > > >> > > > > >
> > > >> > > > > > Thanks,
> > > >> > > > > > Andrii Biletskyi
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <
> jun@confluent.io>
> > > >> wrote:
> > > >> > > > > >
> > > >> > > > > > > Andrii,
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > A few more comments.
> > > >> > > > > > >
> > > >> > > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > >> > > > > > > ReassignPartitionRequest,
> > > >> > > > > > > and PartitionsSerialized that are represented as a
> string,
> > > but
> > > >> > > > contain
> > > >> > > > > > > composite structures in json. Could we flatten them out
> > > >> directly in
> > > >> > > > the
> > > >> > > > > > > protocol definition as arrays/records?
> > > >> > > > > > >
> > > >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto
> topic
> > > >> > > creation?
> > > >> > > > > This
> > > >> > > > > > > will be a bit weird now that we have a separate topic
> > > >> creation api.
> > > >> > > > > Have
> > > >> > > > > > > you thought about how the new createTopicRequest and
> > > >> > > > > TopicMetadataRequest
> > > >> > > > > > > v1 will be used in the producer/consumer client, in
> addition
> > > >> to
> > > >> > > admin
> > > >> > > > > > > tools? For example, ideally, we don't want
> > > >> TopicMetadataRequest
> > > >> > > from
> > > >> > > > > the
> > > >> > > > > > > consumer to trigger auto topic creation.
> > > >> > > > > > >
> > > >> > > > > > > 2. I think Jay meant getting rid of scala classes
> > > >> > > > > > > like HeartbeatRequestAndHeader and
> > > >> HeartbeatResponseAndHeader. We
> > > >> > > did
> > > >> > > > > > that
> > > >> > > > > > > as a stop-gap thing when adding the new requests for the
> > > >> consumers.
> > > >> > > > > > > However, the long term plan is to get rid of all those
> and
> > > >> just
> > > >> > > reuse
> > > >> > > > > the
> > > >> > > > > > > java request/response in the client. Since this KIP
> proposes
> > > >> to
> > > >> > > add a
> > > >> > > > > > > significant number of new requests, perhaps we should
> bite
> > > the
> > > >> > > bullet
> > > >> > > > > to
> > > >> > > > > > > clean up the existing scala requests first before
> adding new
> > > >> ones?
> > > >> > > > > > >
> > > >> > > > > > > Thanks,
> > > >> > > > > > >
> > > >> > > > > > > Jun
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > > > > > >
> > > >> > > > > > > > Hi,
> > > >> > > > > > > >
> > > >> > > > > > > > As said above - I list again all comments from this
> thread
> > > >> so we
> > > >> > > > > > > > can see what's left and finalize all pending issues.
> > > >> > > > > > > >
> > > >> > > > > > > > Comments from Jay:
> > > >> > > > > > > > 1. This is much needed functionality, but there are a
> lot
> > > >> of the
> > > >> > > so
> > > >> > > > > > let's
> > > >> > > > > > > > really think these protocols through. We really want
> to
> > > end
> > > >> up
> > > >> > > > with a
> > > >> > > > > > set
> > > >> > > > > > > > of well thought-out, orthoganol apis. For this reason
> I
> > > >> think it
> > > >> > > is
> > > >> > > > > > > really
> > > >> > > > > > > > important to think through the end state even if that
> > > >> includes
> > > >> > > APIs
> > > >> > > > > we
> > > >> > > > > > > > won't implement in the first phase.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Definitely behind this. Would appreciate if there
> are
> > > >> concrete
> > > >> > > > > > > comments
> > > >> > > > > > > > how this can be improved.
> > > >> > > > > > > >
> > > >> > > > > > > > 2. Let's please please please wait until we have
> switched
> > > >> the
> > > >> > > > server
> > > >> > > > > > over
> > > >> > > > > > > > to the new java protocol definitions. If we add upteen
> > > more
> > > >> ad
> > > >> > > hoc
> > > >> > > > > > scala
> > > >> > > > > > > > objects that is just generating more work for the
> > > >> conversion we
> > > >> > > > know
> > > >> > > > > we
> > > >> > > > > > > > have to do.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Fixed in the latest patch - removed scala protocol
> > > >> classes.
> > > >> > > > > > > >
> > > >> > > > > > > > 3. This proposal introduces a new type of optional
> > > >> parameter.
> > > >> > > This
> > > >> > > > is
> > > >> > > > > > > > inconsistent with everything else in the protocol
> where we
> > > >> use -1
> > > >> > > > or
> > > >> > > > > > some
> > > >> > > > > > > > other marker value. You could argue either way but
> let's
> > > >> stick
> > > >> > > with
> > > >> > > > > > that
> > > >> > > > > > > > for consistency. For clients that implemented the
> protocol
> > > >> in a
> > > >> > > > > better
> > > >> > > > > > > way
> > > >> > > > > > > > than our scala code these basic primitives are hard to
> > > >> change.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf type
> and
> > > >> changed
> > > >> > > > > > protocol
> > > >> > > > > > > > accordingly.
> > > >> > > > > > > >
> > > >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> > > >> TopicMetadataRequest
> > > >> > > > > which
> > > >> > > > > > > has
> > > >> > > > > > > > brokers, topics, and partitions. I think we should
> rename
> > > >> that
> > > >> > > > > request
> > > >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest) and
> > > >> include the
> > > >> > > id
> > > >> > > > > of
> > > >> > > > > > > the
> > > >> > > > > > > > controller. Or are there other things we could add
> here?
> > > >> > > > > > > >
> > > >> > > > > > > > A: I agree. Updated the KIP. Let's extends
> TopicMetadata
> > > to
> > > >> > > > version 2
> > > >> > > > > > and
> > > >> > > > > > > > include controller.
> > > >> > > > > > > >
> > > >> > > > > > > > 5. We have a tendency to try to make a lot of requests
> > > that
> > > >> can
> > > >> > > > only
> > > >> > > > > go
> > > >> > > > > > > to
> > > >> > > > > > > > particular nodes. This adds a lot of burden for client
> > > >> > > > > implementations
> > > >> > > > > > > (it
> > > >> > > > > > > > sounds easy but each discovery can fail in many parts
> so
> > > it
> > > >> ends
> > > >> > > up
> > > >> > > > > > > being a
> > > >> > > > > > > > full state machine to do right). I think we should
> > > consider
> > > >> > > making
> > > >> > > > > > admin
> > > >> > > > > > > > commands and ideally as many of the other apis as
> possible
> > > >> > > > available
> > > >> > > > > on
> > > >> > > > > > > all
> > > >> > > > > > > > brokers and just redirect to the controller on the
> broker
> > > >> side.
> > > >> > > > > Perhaps
> > > >> > > > > > > > there would be a general way to encapsulate this
> > > re-routing
> > > >> > > > behavior.
> > > >> > > > > > > >
> > > >> > > > > > > > A: It's a very interesting idea, but seems there are
> some
> > > >> > > concerns
> > > >> > > > > > about
> > > >> > > > > > > > this
> > > >> > > > > > > > feature (like performance considerations, how this
> will
> > > >> > > complicate
> > > >> > > > > > server
> > > >> > > > > > > > etc).
> > > >> > > > > > > > I believe this shouldn't be a blocker. If this
> feature is
> > > >> > > > implemented
> > > >> > > > > > at
> > > >> > > > > > > > some
> > > >> > > > > > > > point it won't affect Admin changes - at least no
> changes
> > > to
> > > >> > > public
> > > >> > > > > API
> > > >> > > > > > > > will be required.
> > > >> > > > > > > >
> > > >> > > > > > > > 6. We should probably normalize the key value pairs
> used
> > > for
> > > >> > > > configs
> > > >> > > > > > > rather
> > > >> > > > > > > > than embedding a new formatting. So two strings rather
> > > than
> > > >> one
> > > >> > > > with
> > > >> > > > > an
> > > >> > > > > > > > internal equals sign.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Fixed in the latest patch - normalized configs and
> > > >> changed
> > > >> > > > > protocol
> > > >> > > > > > > > accordingly.
> > > >> > > > > > > >
> > > >> > > > > > > > 7. Is the postcondition of these APIs that the
> command has
> > > >> begun
> > > >> > > or
> > > >> > > > > > that
> > > >> > > > > > > > the command has been completed? It is a lot more
> usable if
> > > >> the
> > > >> > > > > command
> > > >> > > > > > > has
> > > >> > > > > > > > been completed so you know that if you create a topic
> and
> > > >> then
> > > >> > > > > publish
> > > >> > > > > > to
> > > >> > > > > > > > it you won't get an exception about there being no
> such
> > > >> topic.
> > > >> > > > > > > >
> > > >> > > > > > > > A: For long running requests (like reassign
> partitions) -
> > > >> the
> > > >> > > post
> > > >> > > > > > > > condition is
> > > >> > > > > > > > command has begun - so we don't block the client. In
> case
> > > >> of your
> > > >> > > > > > > example -
> > > >> > > > > > > > topic commands, this will be refactored and topic
> commands
> > > >> will
> > > >> > > be
> > > >> > > > > > > executed
> > > >> > > > > > > > immediately, since the Controller will serve Admin
> > > requests
> > > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > > >> > > > > > > >
> > > >> > > > > > > > 8. Describe topic and list topics duplicate a lot of
> stuff
> > > >> in the
> > > >> > > > > > > metadata
> > > >> > > > > > > > request. Is there a reason to give back topics marked
> for
> > > >> > > > deletion? I
> > > >> > > > > > > feel
> > > >> > > > > > > > like if we just make the post-condition of the delete
> > > >> command be
> > > >> > > > that
> > > >> > > > > > the
> > > >> > > > > > > > topic is deleted that will get rid of the need for
> this
> > > >> right?
> > > >> > > And
> > > >> > > > it
> > > >> > > > > > > will
> > > >> > > > > > > > be much more intuitive.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Fixed in the latest patch - removed topics marked
> for
> > > >> deletion
> > > >> > > > in
> > > >> > > > > > > > ListTopicsRequest.
> > > >> > > > > > > >
> > > >> > > > > > > > 9. Should we consider batching these requests? We have
> > > >> generally
> > > >> > > > > tried
> > > >> > > > > > to
> > > >> > > > > > > > allow multiple operations to be batched. My suspicion
> is
> > > >> that
> > > >> > > > without
> > > >> > > > > > > this
> > > >> > > > > > > > we will get a lot of code that does something like
> > > >> > > > > > > >    for(topic: adminClient.listTopics())
> > > >> > > > > > > >       adminClient.describeTopic(topic)
> > > >> > > > > > > > this code will work great when you test on 5 topics
> but
> > > not
> > > >> do as
> > > >> > > > > well
> > > >> > > > > > if
> > > >> > > > > > > > you have 50k.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Updated the KIP - please check "Topic Admin Schema"
> > > >> section.
> > > >> > > > > > > >
> > > >> > > > > > > > 10. I think we should also discuss how we want to
> expose a
> > > >> > > > > programmatic
> > > >> > > > > > > JVM
> > > >> > > > > > > > client api for these operations. Currently people
> rely on
> > > >> > > > AdminUtils
> > > >> > > > > > > which
> > > >> > > > > > > > is totally sketchy. I think we probably need another
> > > client
> > > >> under
> > > >> > > > > > > clients/
> > > >> > > > > > > > that exposes administrative functionality. We will
> need
> > > >> this just
> > > >> > > > to
> > > >> > > > > > > > properly test the new apis, I suspect. We should
> figure
> > > out
> > > >> that
> > > >> > > > API.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Updated the KIP - please check "Admin Client"
> section
> > > >> with an
> > > >> > > > > > initial
> > > >> > > > > > > > API proposal.
> > > >> > > > > > > >
> > > >> > > > > > > > 11. The other information that would be really useful
> to
> > > get
> > > >> > > would
> > > >> > > > be
> > > >> > > > > > > > information about partitions--how much data is in the
> > > >> partition,
> > > >> > > > what
> > > >> > > > > > are
> > > >> > > > > > > > the segment offsets, what is the log-end offset (i.e.
> last
> > > >> > > offset),
> > > >> > > > > > what
> > > >> > > > > > > is
> > > >> > > > > > > > the compaction point, etc. I think that done right
> this
> > > >> would be
> > > >> > > > the
> > > >> > > > > > > > successor to the very awkward OffsetRequest we have
> today.
> > > >> > > > > > > >
> > > >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest
> > > >> patch. I
> > > >> > > > > believe
> > > >> > > > > > > > this should
> > > >> > > > > > > > be resolved in a separate KIP / jira ticket.
> > > >> > > > > > > >
> > > >> > > > > > > > 12. Generally we can do good error handling without
> > > needing
> > > >> > > custom
> > > >> > > > > > > > server-side
> > > >> > > > > > > > messages. I.e. generally the client has the context to
> > > know
> > > >> that
> > > >> > > if
> > > >> > > > > it
> > > >> > > > > > > got
> > > >> > > > > > > > an error that the topic doesn't exist to say "Topic X
> > > >> doesn't
> > > >> > > > exist"
> > > >> > > > > > > rather
> > > >> > > > > > > > than "error code 14" (or whatever). Maybe there are
> > > specific
> > > >> > > cases
> > > >> > > > > > where
> > > >> > > > > > > > this is hard? If we want to add server-side error
> messages
> > > >> we
> > > >> > > > really
> > > >> > > > > do
> > > >> > > > > > > > need to do this in a consistent way across the
> protocol.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Updated the KIP - please check "Protocol Errors"
> > > >> section. I
> > > >> > > > added
> > > >> > > > > > the
> > > >> > > > > > > > comprehensive, fine-grained list of error codes.
> > > >> > > > > > > >
> > > >> > > > > > > > Comments from Guozhang:
> > > >> > > > > > > > 13. Describe topic request: it would be great to go
> beyond
> > > >> just
> > > >> > > > > > batching
> > > >> > > > > > > on
> > > >> > > > > > > > topic name regex for this request. For example, a very
> > > >> common use
> > > >> > > > > case
> > > >> > > > > > of
> > > >> > > > > > > > the topic command is to list all topics whose config
> A's
> > > >> value is
> > > >> > > > B.
> > > >> > > > > > With
> > > >> > > > > > > > topic name regex then we have to first retrieve
> __all__
> > > >> topics's
> > > >> > > > > > > > description info and then filter at the client end,
> which
> > > >> will
> > > >> > > be a
> > > >> > > > > > huge
> > > >> > > > > > > > burden on ZK.
> > > >> > > > > > > > AND
> > > >> > > > > > > > 14. Config K-Vs in create topic: this is related to
> the
> > > >> previous
> > > >> > > > > point;
> > > >> > > > > > > > maybe we can add another metadata K-V or just a
> metadata
> > > >> string
> > > >> > > > along
> > > >> > > > > > > side
> > > >> > > > > > > > with config K-V in create topic like we did for offset
> > > >> commit
> > > >> > > > > request.
> > > >> > > > > > > This
> > > >> > > > > > > > field can be quite useful in storing information like
> > > >> "owner" of
> > > >> > > > the
> > > >> > > > > > > topic
> > > >> > > > > > > > who issue the create command, etc, which is quite
> > > important
> > > >> for a
> > > >> > > > > > > > multi-tenant setting. Then in the describe topic
> request
> > > we
> > > >> can
> > > >> > > > also
> > > >> > > > > > > batch
> > > >> > > > > > > > on regex of the metadata field.
> > > >> > > > > > > >
> > > >> > > > > > > > A: As discussed it is very interesting but can be
> > > >> implemented
> > > >> > > later
> > > >> > > > > > after
> > > >> > > > > > > > we have some basic functionality there.
> > > >> > > > > > > >
> > > >> > > > > > > > 15. Today all the admin operations are async in the
> sense
> > > >> that
> > > >> > > > > command
> > > >> > > > > > > will
> > > >> > > > > > > > return once it is written in ZK, and that is why we
> need
> > > >> extra
> > > >> > > > > > > verification
> > > >> > > > > > > > like testUtil.waitForTopicCreated() / verify partition
> > > >> > > reassignment
> > > >> > > > > > > > request, etc. With admin requests we could add a flag
> to
> > > >> enable /
> > > >> > > > > > disable
> > > >> > > > > > > > synchronous requests; when it is turned on, the
> response
> > > >> will not
> > > >> > > > > > return
> > > >> > > > > > > > until the request has been completed. And for async
> > > >> requests we
> > > >> > > can
> > > >> > > > > > add a
> > > >> > > > > > > > "token" field in the response, and then only need a
> > > general
> > > >> > > "admin
> > > >> > > > > > > > verification request" with the given token to check
> if the
> > > >> async
> > > >> > > > > > request
> > > >> > > > > > > > has been completed.
> > > >> > > > > > > >
> > > >> > > > > > > > A: I see your point. My idea was to provide specific
> > > >> > > > Verify...Request
> > > >> > > > > > per
> > > >> > > > > > > > each
> > > >> > > > > > > > long running request, where needed. We can do it the
> way
> > > you
> > > >> > > > suggest.
> > > >> > > > > > The
> > > >> > > > > > > > only
> > > >> > > > > > > > concern is that introducing a token we again will make
> > > >> schema
> > > >> > > > > > "dynamic".
> > > >> > > > > > > We
> > > >> > > > > > > > wanted
> > > >> > > > > > > > to do similar thing introducing single AdminRequest
> for
> > > all
> > > >> topic
> > > >> > > > > > > commands
> > > >> > > > > > > > but rejected
> > > >> > > > > > > > this idea because we wanted to have schema defined. So
> > > this
> > > >> is
> > > >> > > > more a
> > > >> > > > > > > > choice between:
> > > >> > > > > > > > a) have fixed schema but introduce each time new
> > > >> Verify...Request
> > > >> > > > for
> > > >> > > > > > > > long-running requests
> > > >> > > > > > > > b) use one request for verification but generalize it
> with
> > > >> token
> > > >> > > > > > > > I'm fine with whatever decision community come to.
> Just
> > > let
> > > >> me
> > > >> > > know
> > > >> > > > > > your
> > > >> > > > > > > > thoughts.
> > > >> > > > > > > >
> > > >> > > > > > > > Comment from Gwen:
> > > >> > > > > > > > 16. Specifically for ownership, I think the plan is
> to add
> > > >> ACL
> > > >> > > (it
> > > >> > > > > > sounds
> > > >> > > > > > > > like you are describing ACL) via an external system
> > > (Argus,
> > > >> > > > Sentry).
> > > >> > > > > > > > I remember KIP-11 described this, but I can't find
> the KIP
> > > >> any
> > > >> > > > > longer.
> > > >> > > > > > > >
> > > >> > > > > > > > A: Okay, no problem. Not sure though how we are going
> to
> > > >> handle
> > > >> > > it.
> > > >> > > > > > Wait
> > > >> > > > > > > > which KIP
> > > >> > > > > > > > will be committed first and include changes to
> > > >> TopicMetadata from
> > > >> > > > the
> > > >> > > > > > > later
> > > >> > > > > > > > one?
> > > >> > > > > > > > Anyway, I added this note to "Open Questions" section
> so
> > > we
> > > >> don't
> > > >> > > > > miss
> > > >> > > > > > > this
> > > >> > > > > > > > piece.
> > > >> > > > > > > >
> > > >> > > > > > > > Thanks,
> > > >> > > > > > > > Andrii Biletskyi
> > > >> > > > > > > >
> > > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > > > > > > >
> > > >> > > > > > > > > Hi all,
> > > >> > > > > > > > >
> > > >> > > > > > > > > Today I uploaded the patch that covers some of the
> > > >> discussed
> > > >> > > and
> > > >> > > > > > agreed
> > > >> > > > > > > > > items:
> > > >> > > > > > > > > - removed MaybeOf optional type
> > > >> > > > > > > > > - switched to java protocol definitions
> > > >> > > > > > > > > - simplified messages (normalized configs, removed
> topic
> > > >> marked
> > > >> > > > for
> > > >> > > > > > > > > deletion)
> > > >> > > > > > > > >
> > > >> > > > > > > > > I also updated the KIP-4 with respective changes and
> > > >> wrote down
> > > >> > > > my
> > > >> > > > > > > > > proposal for
> > > >> > > > > > > > > pending items:
> > > >> > > > > > > > > - Batch Admin Operations -> updated Wire Protocol
> schema
> > > >> > > proposal
> > > >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
> > > >> > > > TopicMetadataRequest
> > > >> > > > > > > > > - Admin Client -> updated my initial proposal to
> reflect
> > > >> > > batching
> > > >> > > > > > > > > - Error codes -> proposed fine-grained error code
> > > instead
> > > >> of
> > > >> > > > > > > > > AdminRequestFailed
> > > >> > > > > > > > >
> > > >> > > > > > > > > I will also send a separate email to cover all
> comments
> > > >> from
> > > >> > > this
> > > >> > > > > > > thread.
> > > >> > > > > > > > >
> > > >> > > > > > > > > Thanks,
> > > >> > > > > > > > > Andrii Biletskyi
> > > >> > > > > > > > >
> > > >> > > > > > > > >
> > > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> > > >> > > > > gshapira@cloudera.com
> > > >> > > > > > >
> > > >> > > > > > > > > wrote:
> > > >> > > > > > > > >
> > > >> > > > > > > > >> Found KIP-11 (
> > > >> > > > > > > > >>
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > >> > > > > > > > >> )
> > > >> > > > > > > > >> It actually specifies changes to the Metadata
> protocol,
> > > >> so
> > > >> > > > making
> > > >> > > > > > sure
> > > >> > > > > > > > >> both KIPs are consistent in this regard will be
> good.
> > > >> > > > > > > > >>
> > > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> > > >> > > > > > gshapira@cloudera.com
> > > >> > > > > > > >
> > > >> > > > > > > > >> wrote:
> > > >> > > > > > > > >> > Specifically for ownership, I think the plan is
> to
> > > add
> > > >> ACL
> > > >> > > (it
> > > >> > > > > > > sounds
> > > >> > > > > > > > >> > like you are describing ACL) via an external
> system
> > > >> (Argus,
> > > >> > > > > > Sentry).
> > > >> > > > > > > > >> > I remember KIP-11 described this, but I can't
> find
> > > the
> > > >> KIP
> > > >> > > any
> > > >> > > > > > > longer.
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting
> > > >> information
> > > >> > > that
> > > >> > > > > > > already
> > > >> > > > > > > > >> > exists from Kafka brokers, not on adding
> information
> > > >> that
> > > >> > > > > perhaps
> > > >> > > > > > > > >> > should exist but doesn't yet?
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> > Gwen
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> >
> > > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> > > >> > > > > > wangguoz@gmail.com>
> > > >> > > > > > > > >> wrote:
> > > >> > > > > > > > >> >> Folks,
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Just want to elaborate a bit more on the
> > > create-topic
> > > >> > > > metadata
> > > >> > > > > > and
> > > >> > > > > > > > >> batching
> > > >> > > > > > > > >> >> describe-topic based on config / metadata in my
> > > >> previous
> > > >> > > > email
> > > >> > > > > as
> > > >> > > > > > > we
> > > >> > > > > > > > >> work
> > > >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to have
> some
> > > >> sort of
> > > >> > > > > topic
> > > >> > > > > > > > >> management
> > > >> > > > > > > > >> >> mechanisms, which I think is quite important in
> a
> > > >> > > > multi-tenant
> > > >> > > > > /
> > > >> > > > > > > > cloud
> > > >> > > > > > > > >> >> architecture: today anyone can create topics in
> a
> > > >> shared
> > > >> > > > Kafka
> > > >> > > > > > > > >> cluster, but
> > > >> > > > > > > > >> >> there is no concept or "ownership" of topics
> that
> > > are
> > > >> > > created
> > > >> > > > > by
> > > >> > > > > > > > >> different
> > > >> > > > > > > > >> >> users. For example, at LinkedIn we basically
> > > >> distinguish
> > > >> > > > topic
> > > >> > > > > > > owners
> > > >> > > > > > > > >> via
> > > >> > > > > > > > >> >> some casual topic name prefix, which is a bit
> > > awkward
> > > >> and
> > > >> > > > does
> > > >> > > > > > not
> > > >> > > > > > > > fly
> > > >> > > > > > > > >> as
> > > >> > > > > > > > >> >> we scale our customers. It would be great to use
> > > >> > > > > describe-topics
> > > >> > > > > > > such
> > > >> > > > > > > > >> as:
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Describe all topics that is created by me.
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Describe all topics whose retention time is
> > > overriden
> > > >> to X.
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Describe all topics whose writable group include
> > > user
> > > >> Y
> > > >> > > (this
> > > >> > > > > is
> > > >> > > > > > > > >> related to
> > > >> > > > > > > > >> >> authorization), etc..
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> One possible way to achieve this is to add a
> > > metadata
> > > >> file
> > > >> > > in
> > > >> > > > > the
> > > >> > > > > > > > >> >> create-topic request, whose value will also be
> > > >> written ZK
> > > >> > > as
> > > >> > > > we
> > > >> > > > > > > > create
> > > >> > > > > > > > >> the
> > > >> > > > > > > > >> >> topic; then describe-topics can choose to batch
> > > topics
> > > >> > > based
> > > >> > > > on
> > > >> > > > > > 1)
> > > >> > > > > > > > name
> > > >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata
> regex,
> > > etc.
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Thoughts?
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> Guozhang
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> > > >> > > > > > wangguoz@gmail.com>
> > > >> > > > > > > > >> wrote:
> > > >> > > > > > > > >> >>
> > > >> > > > > > > > >> >>> Thanks for the updated wiki. A few comments
> below:
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> 1. Error description in response: I think if
> some
> > > >> > > errorCode
> > > >> > > > > > could
> > > >> > > > > > > > >> indicate
> > > >> > > > > > > > >> >>> several different error cases then we should
> really
> > > >> change
> > > >> > > > it
> > > >> > > > > to
> > > >> > > > > > > > >> multiple
> > > >> > > > > > > > >> >>> codes. In general the errorCode itself would be
> > > >> precise
> > > >> > > and
> > > >> > > > > > > > >> sufficient for
> > > >> > > > > > > > >> >>> describing the server side errors.
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> 2. Describe topic request: it would be great
> to go
> > > >> beyond
> > > >> > > > just
> > > >> > > > > > > > >> batching on
> > > >> > > > > > > > >> >>> topic name regex for this request. For
> example, a
> > > >> very
> > > >> > > > common
> > > >> > > > > > use
> > > >> > > > > > > > >> case of
> > > >> > > > > > > > >> >>> the topic command is to list all topics whose
> > > config
> > > >> A's
> > > >> > > > value
> > > >> > > > > > is
> > > >> > > > > > > B.
> > > >> > > > > > > > >> With
> > > >> > > > > > > > >> >>> topic name regex then we have to first retrieve
> > > >> __all__
> > > >> > > > > topics's
> > > >> > > > > > > > >> >>> description info and then filter at the client
> end,
> > > >> which
> > > >> > > > will
> > > >> > > > > > be
> > > >> > > > > > > a
> > > >> > > > > > > > >> huge
> > > >> > > > > > > > >> >>> burden on ZK.
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is
> related to
> > > >> the
> > > >> > > > > previous
> > > >> > > > > > > > point;
> > > >> > > > > > > > >> >>> maybe we can add another metadata K-V or just a
> > > >> metadata
> > > >> > > > > string
> > > >> > > > > > > > along
> > > >> > > > > > > > >> side
> > > >> > > > > > > > >> >>> with config K-V in create topic like we did for
> > > >> offset
> > > >> > > > commit
> > > >> > > > > > > > >> request. This
> > > >> > > > > > > > >> >>> field can be quite useful in storing
> information
> > > like
> > > >> > > > "owner"
> > > >> > > > > of
> > > >> > > > > > > the
> > > >> > > > > > > > >> topic
> > > >> > > > > > > > >> >>> who issue the create command, etc, which is
> quite
> > > >> > > important
> > > >> > > > > for
> > > >> > > > > > a
> > > >> > > > > > > > >> >>> multi-tenant setting. Then in the describe
> topic
> > > >> request
> > > >> > > we
> > > >> > > > > can
> > > >> > > > > > > also
> > > >> > > > > > > > >> batch
> > > >> > > > > > > > >> >>> on regex of the metadata field.
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> 4. Today all the admin operations are async in
> the
> > > >> sense
> > > >> > > > that
> > > >> > > > > > > > command
> > > >> > > > > > > > >> will
> > > >> > > > > > > > >> >>> return once it is written in ZK, and that is
> why we
> > > >> need
> > > >> > > > extra
> > > >> > > > > > > > >> verification
> > > >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify
> > > >> partition
> > > >> > > > > > > reassignment
> > > >> > > > > > > > >> >>> request, etc. With admin requests we could add
> a
> > > >> flag to
> > > >> > > > > enable
> > > >> > > > > > /
> > > >> > > > > > > > >> disable
> > > >> > > > > > > > >> >>> synchronous requests; when it is turned on, the
> > > >> response
> > > >> > > > will
> > > >> > > > > > not
> > > >> > > > > > > > >> return
> > > >> > > > > > > > >> >>> until the request has been completed. And for
> async
> > > >> > > requests
> > > >> > > > > we
> > > >> > > > > > > can
> > > >> > > > > > > > >> add a
> > > >> > > > > > > > >> >>> "token" field in the response, and then only
> need a
> > > >> > > general
> > > >> > > > > > "admin
> > > >> > > > > > > > >> >>> verification request" with the given token to
> check
> > > >> if the
> > > >> > > > > async
> > > >> > > > > > > > >> request
> > > >> > > > > > > > >> >>> has been completed.
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> 5. +1 for extending Metadata request to include
> > > >> > > controller /
> > > >> > > > > > > > >> coordinator
> > > >> > > > > > > > >> >>> information, and then we can remove the
> > > >> ConsumerMetadata /
> > > >> > > > > > > > >> ClusterMetadata
> > > >> > > > > > > > >> >>> requests.
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> Guozhang
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> > > >> > > > > > jjkoshy.w@gmail.com>
> > > >> > > > > > > > >> wrote:
> > > >> > > > > > > > >> >>>
> > > >> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't
> think I
> > > >> will be
> > > >> > > > > able
> > > >> > > > > > to
> > > >> > > > > > > > >> make
> > > >> > > > > > > > >> >>>> it today, so if notes can be sent out
> afterward
> > > that
> > > >> > > would
> > > >> > > > be
> > > >> > > > > > > > great.
> > > >> > > > > > > > >> >>>>
> > > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen
> > > >> Shapira
> > > >> > > > wrote:
> > > >> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking
> forward
> > > >> to
> > > >> > > > > chatting
> > > >> > > > > > > with
> > > >> > > > > > > > >> >>>> everyone :)
> > > >> > > > > > > > >> >>>> >
> > > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> > > >> > > > > > > joe.stein@stealth.ly>
> > > >> > > > > > > > >> wrote:
> > > >> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout
> invite
> > > to
> > > >> all
> > > >> > > > pmc,
> > > >> > > > > > > > >> committers
> > > >> > > > > > > > >> >>>> and
> > > >> > > > > > > > >> >>>> > > everyone I found working on a KIP. If I
> missed
> > > >> anyone
> > > >> > > > in
> > > >> > > > > > the
> > > >> > > > > > > > >> invite
> > > >> > > > > > > > >> >>>> please
> > > >> > > > > > > > >> >>>> > > let me know and can update it, np.
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm
> Eastern
> > > >> Time.
> > > >> > > > Maybe
> > > >> > > > > > we
> > > >> > > > > > > > can
> > > >> > > > > > > > >> get
> > > >> > > > > > > > >> >>>> INFRA
> > > >> > > > > > > > >> >>>> > > help to make a google account so we can
> manage
> > > >> > > better?
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>> > > To discuss
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>>
> > > >> > > > > > > > >>
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > >> > > > > > > > >> >>>> > > in progress and related JIRA that are
> > > >> interdependent
> > > >> > > > and
> > > >> > > > > > > common
> > > >> > > > > > > > >> work.
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay
> Kreps <
> > > >> > > > > > > > jay.kreps@gmail.com>
> > > >> > > > > > > > >> >>>> wrote:
> > > >> > > > > > > > >> >>>> > >
> > > >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that will
> also
> > > >> record
> > > >> > > > and
> > > >> > > > > > make
> > > >> > > > > > > > the
> > > >> > > > > > > > >> >>>> sessions
> > > >> > > > > > > > >> >>>> > >> available on youtube.
> > > >> > > > > > > > >> >>>> > >>
> > > >> > > > > > > > >> >>>> > >> -Jay
> > > >> > > > > > > > >> >>>> > >>
> > > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff
> > > Holoman
> > > >> <
> > > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > >> > > > > > > > >> >>>> > >> wrote:
> > > >> > > > > > > > >> >>>> > >>
> > > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for
> this
> > > >> purpose.
> > > >> > > We
> > > >> > > > > > could
> > > >> > > > > > > > >> record
> > > >> > > > > > > > >> >>>> the
> > > >> > > > > > > > >> >>>> > >> > sessions if there is interest and
> publish
> > > >> them
> > > >> > > out.
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >> > Thanks
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >> > Jeff
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay
> > > Kreps <
> > > >> > > > > > > > >> jay.kreps@gmail.com>
> > > >> > > > > > > > >> >>>> wrote:
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >> > > Let's try to get the technical
> hang-ups
> > > >> sorted
> > > >> > > > out,
> > > >> > > > > > > > though.
> > > >> > > > > > > > >> I
> > > >> > > > > > > > >> >>>> really
> > > >> > > > > > > > >> >>>> > >> > think
> > > >> > > > > > > > >> >>>> > >> > > there is some benefit to live
> discussion
> > > vs
> > > >> > > > > writing. I
> > > >> > > > > > > am
> > > >> > > > > > > > >> >>>> hopeful that
> > > >> > > > > > > > >> >>>> > >> if
> > > >> > > > > > > > >> >>>> > >> > > we post instructions and give
> ourselves a
> > > >> few
> > > >> > > > > attempts
> > > >> > > > > > > we
> > > >> > > > > > > > >> can
> > > >> > > > > > > > >> >>>> get it
> > > >> > > > > > > > >> >>>> > >> > > working.
> > > >> > > > > > > > >> >>>> > >> > >
> > > >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work for
> > > >> me...any
> > > >> > > > > > objections?
> > > >> > > > > > > > >> >>>> > >> > >
> > > >> > > > > > > > >> >>>> > >> > > -Jay
> > > >> > > > > > > > >> >>>> > >> > >
> > > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe
> > > Stein
> > > >> <
> > > >> > > > > > > > >> joe.stein@stealth.ly
> > > >> > > > > > > > >> >>>> >
> > > >> > > > > > > > >> >>>> > >> wrote:
> > > >> > > > > > > > >> >>>> > >> > >
> > > >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe like
> every
> > > >> > > Tuesday ~
> > > >> > > > > 1pm
> > > >> > > > > > > ET
> > > >> > > > > > > > /
> > > >> > > > > > > > >> 10am
> > > >> > > > > > > > >> >>>> PT
> > > >> > > > > > > > >> >>>> > >> ????
> > > >> > > > > > > > >> >>>> > >> > > >
> > > >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but
> there
> > > is
> > > >> > > always
> > > >> > > > > some
> > > >> > > > > > > > >> issue or
> > > >> > > > > > > > >> >>>> > >> whatever
> > > >> > > > > > > > >> >>>> > >> > so
> > > >> > > > > > > > >> >>>> > >> > > > we know the apache irc channel
> works.
> > > We
> > > >> can
> > > >> > > > start
> > > >> > > > > > > there
> > > >> > > > > > > > >> and
> > > >> > > > > > > > >> >>>> see how
> > > >> > > > > > > > >> >>>> > >> it
> > > >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too
> and
> > > >> > > associate
> > > >> > > > to
> > > >> > > > > > > > >> tickets if
> > > >> > > > > > > > >> >>>> need be
> > > >> > > > > > > > >> >>>> > >> > > makes
> > > >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> > > >> > > > > > > > >> >>>> > >> > > >
> > > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > >> > > > > > > > >> >>>> > >> > > >
> > > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM,
> Jay
> > > >> Kreps <
> > > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > >> > > > > > > > >> >>>> > >> > wrote:
> > > >> > > > > > > > >> >>>> > >> > > >
> > > >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a Google
> > > >> Hangout to
> > > >> > > > chat
> > > >> > > > > > > about
> > > >> > > > > > > > >> this.
> > > >> > > > > > > > >> >>>> What
> > > >> > > > > > > > >> >>>> > >> > about
> > > >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
> > > further...I
> > > >> > > > actually
> > > >> > > > > > > think
> > > >> > > > > > > > it
> > > >> > > > > > > > >> >>>> would be
> > > >> > > > > > > > >> >>>> > >> > good
> > > >> > > > > > > > >> >>>> > >> > > > for
> > > >> > > > > > > > >> >>>> > >> > > > > everyone spending a reasonable
> chunk
> > > of
> > > >> > > their
> > > >> > > > > week
> > > >> > > > > > > on
> > > >> > > > > > > > >> Kafka
> > > >> > > > > > > > >> >>>> stuff
> > > >> > > > > > > > >> >>>> > >> to
> > > >> > > > > > > > >> >>>> > >> > > > maybe
> > > >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think we
> could
> > > >> use
> > > >> > > time
> > > >> > > > > to
> > > >> > > > > > > talk
> > > >> > > > > > > > >> >>>> through
> > > >> > > > > > > > >> >>>> > >> design
> > > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of
> > > code
> > > >> > > > reviews,
> > > >> > > > > > talk
> > > >> > > > > > > > >> through
> > > >> > > > > > > > >> >>>> any
> > > >> > > > > > > > >> >>>> > >> > tricky
> > > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > We can make it publicly
> available so
> > > >> that
> > > >> > > any
> > > >> > > > > one
> > > >> > > > > > > can
> > > >> > > > > > > > >> follow
> > > >> > > > > > > > >> >>>> along
> > > >> > > > > > > > >> >>>> > >> > who
> > > >> > > > > > > > >> >>>> > >> > > > > likes.
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this? If so
> > > I'll
> > > >> try
> > > >> > > to
> > > >> > > > > set
> > > >> > > > > > it
> > > >> > > > > > > > up
> > > >> > > > > > > > >> >>>> starting
> > > >> > > > > > > > >> >>>> > >> next
> > > >> > > > > > > > >> >>>> > >> > > > week.
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM,
> > > Andrii
> > > >> > > > > Biletskyi
> > > >> > > > > > <
> > > >> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly>
> wrote:
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed /
> > > >> aligned
> > > >> > > > > document
> > > >> > > > > > > > >> structure.
> > > >> > > > > > > > >> >>>> Also I
> > > >> > > > > > > > >> >>>> > >> > > added
> > > >> > > > > > > > >> >>>> > >> > > > > > some
> > > >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
> > > >> AdminClient so
> > > >> > > we
> > > >> > > > > have
> > > >> > > > > > > > >> something
> > > >> > > > > > > > >> >>>> to
> > > >> > > > > > > > >> >>>> > >> start
> > > >> > > > > > > > >> >>>> > >> > > > from
> > > >> > > > > > > > >> >>>> > >> > > > > > while
> > > >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > >
> > > >> > > > > > > > >> >>>> > >> > > >
> > > >> > > > > > > > >> >>>> > >> > >
> > > >> > > > > > > > >> >>>> > >> >
> > > >> > > > > > > > >> >>>> > >>
> > > >> > > > > > > > >> >>>>
> > > >> > > > > > > > >>
> > > >> > > > > > > >
> > > >> > > > > > >
> > > >> > > > > >
> > > >> > > > >
> > > >> > > >
> > > >> > >
> > > >>
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > > >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01
> PM,
> > > >> Andrii
> > > >> > > > > > Biletskyi
> > > >> > > > > > > <
> > > >> > > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > >> > > > > > > > >> >>>> > >> > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you are
> right,
> > > >> in
> > > >> > > most
> > > >> > > > > > cases
> > > >> > > > > > > > >> client
> > > >> > > > > > > > >> >>>> will
> > > >> > > > > > > > >> >>>> > >> have
> > > >> > > > > > > > >> >>>> > >> > > > enough
> > > >> > > > > > > > >> >>>> > >> > > > > > > context to show descriptive
> error
> > > >> > > message.
> > > >> > > > > My
> > > >> > > > > > > > >> concern is
> > > >> > > > > > > > >> >>>> that
> > > >> > > > > > > > >> >>>> > >> we
> > > >> > > > > > > > >> >>>> > >> > > will
> > > >> > > > > > > > >> >>>> > >> > > > > > have
> > > >> > > > > > > > >> >>>> > >> > > > > > > to
> > > >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error codes
> for
> > > >> each
> > > >> > > > > possible
> > > >> > > > > > > > >> error. Of
> > > >> > > > > > > > >> >>>> course,
> > > >> > > > > > > > >> >>>> > >> > we
> > > >> > > > > > > > >> >>>> > >> > > > > could
> > > >> > > > > > > > >> >>>> > >> > > > > > > reuse
> > > >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
> > > >> > > > > > > UknownTopicOrPartitionCode,
> > > >> > > > > > > > >> but we
> > > >> > > > > > > > >> >>>> will
> > > >> > > > > > > > >> >>>> > >> > also
> > > >> > > > > > > > >> >>>> > >> > > > need
> > > >> > > > > > > > >> >>>> > >> > > > > > to
> > > >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> > > >> TopicAlreadyExistsCode,
> > > >> > > > > > > > >> >>>> TopicConfigInvalid (both
> > > >> > > > > > > > >> >>>> > >> > for
> > > >> > > > > > > > >> >>>> > >> > > > > topic
> > > >> > > > > > > > >> >>>> > >> > > > > > > name and config, and probably
> > > user
> > > >> would
> > > >> > > > > like
> > > >> > > > > > to
> > > >> > > > > > > > >> know
> > > >> > > > > > > > >> >>>> what
> > > >> > > > > > > > >> >>>> > >> > exactly
> > > >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
> > > >> > > > > > > InvalidReplicaAssignment,
> > > >> > > > > > > > >> >>>> > >> InternalError
> > > >> > > > > > > > >> >>>> > >> > > > (e.g.
> > > >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
> > > TopicCommand,
> > > >> we
> > > >> > > will
> > > >> > > > > > also
> > > >> > > > > > > > >> need to
> > > >> > > > > > > > >> >>>> add
> > > >> > > > > > > > >> >>>> > >> > similar
> > > >> > > > > > > > >> >>>> > >> > > > > stuff
> > > >> > > > > > > > >> >>>> > >> > > > > > > for
> > > >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> > > >> PreferredReplica. So
> > > >> > > > > we'll
> > > >> > > > > > > end
> > > >> > > > > > > > >> up
> > > >> > > > > > > > >> >>>> with a
> > > >> > > > > > > > >> >>>> > >> > large
> > > >> > > > > > > > >> >>>> > >> > > > list
> > > >> > > > > > > > >> >>>> > >> > > > > > of
> > > >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in
> Admin
> > > >> > > protocol.
> > > >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree my
> > > >> proposal is
> > > >> > > > not
> > > >> > > > > > > > >> consistent
> > > >> > > > > > > > >> >>>> with
> > > >> > > > > > > > >> >>>> > >> > other
> > > >> > > > > > > > >> >>>> > >> > > > > cases.
> > > >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better
> solution
> > > >> or
> > > >> > > > > something
> > > >> > > > > > > > >> >>>> in-between.
> > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it
> is a
> > > >> great
> > > >> > > > idea.
> > > >> > > > > > > This
> > > >> > > > > > > > >> way we
> > > >> > > > > > > > >> >>>> can
> > > >> > > > > > > > >> >>>> > >> move
> > > >> > > > > > > > >> >>>> > >> > > on
> > > >> > > > > > > > >> >>>> > >> > > > > > > faster.
> > > >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on
> date/time
> > > so
> > > >> > > people
> > > >> > > > > can
> > > >> > > > > > > > join.
> > > >> > > > > > > > >> >>>> Will work
> > > >> > > > > > > > >> >>>> > >> > for
> > > >> > > > > > > > >> >>>> > >> > > me
> > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > >> > > > > > > > >> >>>> > >> > > > > > > and
> > > >> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime if
> > > agreed
> > > >> in
> > > >> > > > > advance.
> > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09
> PM,
> > > >> Jay
> > > >> > > > Kreps <
> > > >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > >> > > > > > > > >> >>>> > >> > > > > wrote:
> > > >> > > > > > > > >> >>>> > >> > > > > > >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good
> error
> > > >> handling
> > > >> > > > > > without
> > > >> > > > > > > > >> needing
> > > >> > > > > > > > >> >>>> custom
> > > >> > > > > > > > >> >>>> > >> > > > > > server-side
> > > >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the
> > > >> client has
> > > >> > > > the
> > > >> > > > > > > > >> context to
> > > >> > > > > > > > >> >>>> know
> > > >> > > > > > > > >> >>>> > >> that
> > > >> > > > > > > > >> >>>> > >> > > if
> > > >> > > > > > > > >> >>>> > >> > > > it
> > > >> > > > > > > > >> >>>> > >> > > > > > got
> > > >> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic
> doesn't
> > > >> exist
> > > >> > > to
> > > >> > > > > say
> > > >> > > > > > > > >> "Topic X
> > > >> > > > > > > > >> >>>> doesn't
> > > >> > > > > > > > >> >>>> > >> > > exist"
> > > >> > > > > > > > >> >>>> > >> > > > > > >> rather
> > > >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or
> > > >> whatever).
> > > >> > > Maybe
> > > >> > > > > > there
> > > >> > > > > > > > are
> > > >> > > > > > > > >> >>>> specific
> > > >> > > > > > > > >> >>>> > >> > cases
> > > >> > > > > > > > >> >>>> > >> > > > > where
> > > >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to
> add
> > > >> > > > server-side
> > > >> > > > > > > error
> > > >> > > > > > > > >> >>>> messages we
> > > >> > > > > > > > >> >>>> > >> > > really
> > > >> > > > > > > > >> >>>> > >> > > > > do
> > > >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a
> consistent
> > > >> way
> > > >> > > > across
> > > >> > > > > > the
> > > >> > > > > > > > >> protocol.
> > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open
> > > >> questions
> > > >> > > > here
> > > >> > > > > > > from
> > > >> > > > > > > > my
> > > >> > > > > > > > >> >>>> previous
> > > >> > > > > > > > >> >>>> > >> > > list. I
> > > >> > > > > > > > >> >>>> > >> > > > > > will
> > > >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few
> days for
> > > >> Strata
> > > >> > > > > > though.
> > > >> > > > > > > > >> Maybe
> > > >> > > > > > > > >> >>>> we could
> > > >> > > > > > > > >> >>>> > >> > do
> > > >> > > > > > > > >> >>>> > >> > > a
> > > >> > > > > > > > >> >>>> > >> > > > > > Google
> > > >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open
> issues
> > > >> some
> > > >> > > time
> > > >> > > > > > > towards
> > > >> > > > > > > > >> the
> > > >> > > > > > > > >> >>>> end of
> > > >> > > > > > > > >> >>>> > >> > next
> > > >> > > > > > > > >> >>>> > >> > > > week
> > > >> > > > > > > > >> >>>> > >> > > > > > for
> > > >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in this
> > > ticket?
> > > >> I
> > > >> > > have
> > > >> > > > a
> > > >> > > > > > > > feeling
> > > >> > > > > > > > >> that
> > > >> > > > > > > > >> >>>> might
> > > >> > > > > > > > >> >>>> > >> > > > progress
> > > >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster than
> > > >> email--I
> > > >> > > > think
> > > >> > > > > we
> > > >> > > > > > > > >> could talk
> > > >> > > > > > > > >> >>>> > >> through
> > > >> > > > > > > > >> >>>> > >> > > > those
> > > >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly
> > > >> quickly...
> > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at
> 7:27 AM,
> > > >> Andrii
> > > >> > > > > > > > Biletskyi <
> > > >> > > > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly
> >
> > > >> wrote:
> > > >> > > > > > > > >> >>>> > >> > > > > > >>
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address
> some of
> > > >> the
> > > >> > > > issues
> > > >> > > > > > > which
> > > >> > > > > > > > >> were
> > > >> > > > > > > > >> >>>> > >> mentioned
> > > >> > > > > > > > >> >>>> > >> > > > > earlier
> > > >> > > > > > > > >> >>>> > >> > > > > > >> about
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of
> > > >> those was
> > > >> > > > > about
> > > >> > > > > > > > >> batching
> > > >> > > > > > > > >> >>>> > >> > operations.
> > > >> > > > > > > > >> >>>> > >> > > > What
> > > >> > > > > > > > >> >>>> > >> > > > > > if
> > > >> > > > > > > > >> >>>> > >> > > > > > >> we
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand
> approach
> > > >> and let
> > > >> > > > > people
> > > >> > > > > > > > >> specify
> > > >> > > > > > > > >> >>>> > >> topic-name
> > > >> > > > > > > > >> >>>> > >> > > by
> > > >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most of
> the
> > > use
> > > >> > > cases?
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what
> information
> > > >> should
> > > >> > > we
> > > >> > > > > > > > generally
> > > >> > > > > > > > >> >>>> provide in
> > > >> > > > > > > > >> >>>> > >> > > Admin
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin
> commands
> > > >> don't
> > > >> > > > imply
> > > >> > > > > > > they
> > > >> > > > > > > > >> will
> > > >> > > > > > > > >> >>>> be used
> > > >> > > > > > > > >> >>>> > >> > only
> > > >> > > > > > > > >> >>>> > >> > > > in
> > > >> > > > > > > > >> >>>> > >> > > > > > CLI
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > but,
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a
> very
> > > >> > > important
> > > >> > > > > > > client
> > > >> > > > > > > > >> of this
> > > >> > > > > > > > >> >>>> > >> > feature.
> > > >> > > > > > > > >> >>>> > >> > > In
> > > >> > > > > > > > >> >>>> > >> > > > > > this
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > case,
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would
> like
> > > to
> > > >> > > provide
> > > >> > > > > > users
> > > >> > > > > > > > >> with
> > > >> > > > > > > > >> >>>> rich
> > > >> > > > > > > > >> >>>> > >> > > experience
> > > >> > > > > > > > >> >>>> > >> > > > > in
> > > >> > > > > > > > >> >>>> > >> > > > > > >> terms
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > of
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > getting results / errors
> of
> > > the
> > > >> > > > executed
> > > >> > > > > > > > >> commands.
> > > >> > > > > > > > >> >>>> Usually
> > > >> > > > > > > > >> >>>> > >> we
> > > >> > > > > > > > >> >>>> > >> > > > supply
> > > >> > > > > > > > >> >>>> > >> > > > > > >> with
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > responses only errorCode,
> > > which
> > > >> looks
> > > >> > > > > very
> > > >> > > > > > > > >> limiting,
> > > >> > > > > > > > >> >>>> in case
> > > >> > > > > > > > >> >>>> > >> > of
> > > >> > > > > > > > >> >>>> > >> > > > CLI
> > > >> > > > > > > > >> >>>> > >> > > > > we
> > > >> > > > > > > > >> >>>> > >> > > > > > >> may
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human
> readable
> > > >> error
> > > >> > > > > > > description.
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into account
> > > >> previous item
> > > >> > > > > about
> > > >> > > > > > > > >> batching,
> > > >> > > > > > > > >> >>>> what
> > > >> > > > > > > > >> >>>> > >> do
> > > >> > > > > > > > >> >>>> > >> > > you
> > > >> > > > > > > > >> >>>> > >> > > > > > think
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > about
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support
> > > >> regexp)
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest =>
> > > TopicName
> > > >> > > > > Partitions
> > > >> > > > > > > > >> Replicas
> > > >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse =>
> > > ErrorCode
> > > >> > > > > > > > ErrorDescription
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription =>
> string
> > > >> (empty
> > > >> > > if
> > > >> > > > > > > > >> successful)
> > > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > > >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
> > > >> TopicNameRegexp
> > > >> > > >
> > > >
> > > > ...
> > > >
> > > > [Message clipped]
> > >
>
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Joel Koshy <jj...@gmail.com>.
(Thanks Andrii for the summary)

For (1) yes we will circle back on that shortly after syncing up in
person. I think it is close to getting committed although development
for KAFKA-1927 can probably begin without it.

There is one more item we covered at the hangout. i.e., whether we
want to add the coordinator to the topic metadata response or provide
a clearer ClusterMetadataRequest.

There are two reasons I think we should try and avoid adding the
field:
- It is irrelevant to topic metadata
- If we finally do request rerouting in Kafka then the field would add
  little to no value. (It still helps to have a separate
  ClusterMetadataRequest to query for cluster-wide information such as
  'which broker is the controller?' as Joe mentioned.)

I think it would be cleaner to have an explicit ClusterMetadataRequest
that you can send to any broker in order to obtain the controller (and
in the future possibly other cluster-wide information). I think the
main argument against doing this and instead adding it to the topic
metadata response was convenience - i.e., you don't have to discover
the controller in advance. However, I don't see much actual
benefit/convenience in this and in fact think it is a non-issue. Let
me know if I'm overlooking something here.

As an example, say we need to initiate partition reassignment by
issuing the new ReassignPartitionsRequest to the controller (assume we
already have the desired manual partition assignment).  If we are to
augment topic metadata response then the flow be something like this :

- Issue topic metadata request to any broker (and discover the
  controller
- Connect to controller if required (i.e., if the broker above !=
  controller)
- Issue the partition reassignment request to the controller.

With an explicit cluster metadata request it would be:
- Issue cluster metadata request to any broker
- Connect to controller if required (i.e., if the broker above !=
  controller)
- Issue the partition reassignment request

So it seems to add little practical value and bloats topic metadata
response with an irrelevant detail.

The other angle to this is the following - is it a matter of naming?
Should we just rename topic metadata request/response to just
MetadataRequest/Response and add cluster metadata to it? By that same
token should we also allow querying for the consumer coordinator (and
in future transaction coordinator) as well? This leads to a bloated
request which isn't very appealing and altogether confusing.

Thanks,

Joel

On Wed, Mar 18, 2015 at 09:34:12AM -0700, Jun Rao wrote:
> Andri,
> 
> Thanks for the summary.
> 
> 1. I just realized that in order to start working on KAFKA-1927, we will
> need to merge the changes to OffsetCommitRequest (from 0.8.2) to trunk.
> This is planned to be done as part of KAFKA-1634. So, we will need Guozhang
> and Joel's help to wrap this up.
> 
> 2. Thinking about this a bit more, if the semantic of those "write"
> requests is async (i.e., after the client gets a response, it just means
> that the operation is initiated, but not necessarily completed), we don't
> really need to forward the requests to the controller. Instead, the
> receiving broker can just write the operation to ZK as the admin command
> line tool previously does. This will simplify the implementation.
> 
> 8. There is another implementation detail for describe topic. Ideally, we
> want to read the topic config from the broker cache, instead of ZooKeeper.
> Currently, every broker reads the topic-level config for all topics.
> However, it ignores those for topics not hosted on itself. So, we may need
> to change TopicConfigManager a bit so that it caches the configs for all
> topics.
> 
> Thanks,
> 
> Jun
> 
> 
> On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
> 
> > Guys,
> >
> > Thanks for a great discussion!
> > Here are the actions points:
> >
> > 1. Q: Get rid of all scala requests objects, use java protocol definitions.
> >     A: Gwen kindly took that (KAFKA-1927). It's important to speed up
> > review procedure
> >          there since this ticket blocks other important changes.
> >
> > 2. Q: Generic re-reroute facility vs client maintaining cluster state.
> >     A: Jay has added pseudo code to KAFKA-1912 - need to consider whether
> > this will be
> >         easy to implement as a server-side feature (comments are
> > welcomed!).
> >
> > 3. Q: Controller field in wire protocol.
> >     A: This might be useful for clients, add this to TopicMetadataResponse
> > (already in KIP).
> >
> > 4. Q: Decoupling topic creation from TMR.
> >     A: I will add proposed by Jun solution (using clientId for that) to the
> > KIP.
> >
> > 5. Q: Bumping new versions of TMR vs grabbing all protocol changes in one
> > version.
> >     A: It was decided to try to gather all changes to protocol (before
> > release).
> >         In case of TMR it worth checking: KAFKA-2020 and KIP-13 (quotas)
> >
> > 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
> >     A: Use jackson for that, /tools project is a separate jar so shouldn't
> > be a big deal.
> >
> > 7.  Q: VerifyReassingPartitions vs generic status check command.
> >      A: For long-running requests like reassign partitions *progress* check
> > request is useful,
> >          it makes sense to introduce it.
> >
> >  Please add, correct me if I missed something.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> > On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Joel,
> > >
> > > You are right, I removed ClusterMetadata because we have partially
> > > what we need in TopicMetadata. Also, as Jay pointed out earlier, we
> > > would like to have "orthogonal" API, but at the same time we need
> > > to be backward compatible.
> > >
> > > But I like your idea and even have some other arguments for this option:
> > > There is also DescribeTopicRequest which was proposed in this KIP,
> > > it returns topic configs, partitions, replication factor plus partition
> > > ISR, ASR,
> > > leader replica. The later part is really already there in
> > > TopicMetadataRequest.
> > > So again we'll have to add stuff to TMR, not to duplicate some info in
> > > newly added requests. However, this way we'll end up with "monster"
> > > request which returns cluster metadata, topic replication and config info
> > > plus partition replication data. Seems logical to split TMR to
> > > - ClusterMetadata (brokers + controller, maybe smth else)
> > > - TopicMetadata (topic info + partition details)
> > > But since current TMR is involved in lots of places (including network
> > > client,
> > > as I understand) this might be very serious change and it probably makes
> > > sense to stick with current approach.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > >
> > > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jj...@gmail.com> wrote:
> > >
> > >> I may be missing some context but hopefully this will also be covered
> > >> today: I thought the earlier proposal where there was an explicit
> > >> ClusterMetadata request was clearer and explicit. During the course of
> > >> this thread I think the conclusion was that the main need was for
> > >> controller information and that can be rolled into the topic metadata
> > >> response but that seems a bit irrelevant to topic metadata. FWIW I
> > >> think the full broker-list is also irrelevant to topic metadata, but
> > >> it is already there and in use. I think there is still room for an
> > >> explicit ClusterMetadata request since there may be other
> > >> cluster-level information that we may want to add over time (and that
> > >> have nothing to do with topic metadata).
> > >>
> > >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
> > >> > Jun,
> > >> >
> > >> > 101. Okay, if you say that such use case is important. I also think
> > >> > using clientId for these purposes is fine - if we already have this
> > >> field
> > >> > as part of all Wire protocol messages, why not use that.
> > >> > I will update KIP-4 page if nobody has other ideas (which may come up
> > >> > during the call today).
> > >> >
> > >> > 102.1 Agree, I'll update the KIP accordingly. I think we can add new,
> > >> > fine-grained error codes if some error code received in specific case
> > >> > won't give enough context to return a descriptive error message for
> > >> user.
> > >> >
> > >> > Look forward to discussing all outstanding issues in detail today
> > during
> > >> > the call.
> > >> >
> > >> > Thanks,
> > >> > Andrii Biletskyi
> > >> >
> > >> >
> > >> >
> > >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io> wrote:
> > >> >
> > >> > > 101. There may be a use case where you only want the topics to be
> > >> created
> > >> > > manually by admins. Currently, you can do that by disabling auto
> > topic
> > >> > > creation and issue topic creation from the TopicCommand. If we
> > >> disable auto
> > >> > > topic creation completely on the broker and don't have a way to
> > >> distinguish
> > >> > > between topic creation requests from the regular clients and the
> > >> admin, we
> > >> > > can't support manual topic creation any more. I was thinking that
> > >> another
> > >> > > way of distinguishing the clients making the topic creation requests
> > >> is
> > >> > > using clientId. For example, the admin tool can set it to something
> > >> like
> > >> > > admin and the broker can treat that clientId specially.
> > >> > >
> > >> > > Also, there is a related discussion in KAFKA-2020. Currently, we do
> > >> the
> > >> > > following in TopicMetadataResponse:
> > >> > >
> > >> > > 1. If leader is not available, we set the partition level error code
> > >> to
> > >> > > LeaderNotAvailable.
> > >> > > 2. If a non-leader replica is not available, we take that replica
> > out
> > >> of
> > >> > > the assigned replica list and isr in the response. As an indication
> > >> for
> > >> > > doing that, we set the partition level error code to
> > >> ReplicaNotAvailable.
> > >> > >
> > >> > > This has a few problems. First, ReplicaNotAvailable probably
> > >> shouldn't be
> > >> > > an error, at least for the normal producer/consumer clients that
> > just
> > >> want
> > >> > > to find out the leader. Second, it can happen that both the leader
> > and
> > >> > > another replica are not available at the same time. There is no
> > error
> > >> code
> > >> > > to indicate both. Third, even if a replica is not available, it's
> > >> still
> > >> > > useful to return its replica id since some clients (e.g. admin tool)
> > >> may
> > >> > > still make use of it.
> > >> > >
> > >> > > One way to address this issue is to always return the replica id for
> > >> > > leader, assigned replicas, and isr regardless of whether the
> > >> corresponding
> > >> > > broker is live or not. Since we also return the list of live
> > brokers,
> > >> the
> > >> > > client can figure out whether a leader or a replica is live or not
> > >> and act
> > >> > > accordingly. This way, we don't need to set the partition level
> > error
> > >> code
> > >> > > when the leader or a replica is not available. This doesn't change
> > >> the wire
> > >> > > protocol, but does change the semantics. Since we are evolving the
> > >> protocol
> > >> > > of TopicMetadataRequest here, we can potentially piggyback the
> > change.
> > >> > >
> > >> > > 102.1 For those types of errors due to invalid input, shouldn't we
> > >> just
> > >> > > guard it at parameter validation time and throw
> > >> InvalidArgumentException
> > >> > > without even sending the request to the broker?
> > >> > >
> > >> > > Thanks,
> > >> > >
> > >> > > Jun
> > >> > >
> > >> > >
> > >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > >> > > andrii.biletskyi@stealth.ly> wrote:
> > >> > >
> > >> > > > Jun,
> > >> > > >
> > >> > > > Answering your questions:
> > >> > > >
> > >> > > > 101. If I understand you correctly, you are saying future producer
> > >> > > versions
> > >> > > > (which
> > >> > > > will be ported to TMR_V1) won't be able to automatically create
> > >> topic (if
> > >> > > > we
> > >> > > > unconditionally remove topic creation from there). But we need to
> > >> this
> > >> > > > preserve logic.
> > >> > > > Ok, about your proposal: I'm not a big fan too, when it comes to
> > >> > > > differentiating
> > >> > > > clients directly in protocol schema. And also I'm not sure I
> > >> understand
> > >> > > at
> > >> > > > all why
> > >> > > > auto.create.topics.enable is a server side configuration. Can we
> > >> > > deprecate
> > >> > > > this setting
> > >> > > > in future versions, add this setting to producer and based on that
> > >> upon
> > >> > > > receiving
> > >> > > > UnknownTopic create topic explicitly by a separate producer call
> > via
> > >> > > > adminClient?
> > >> > > >
> > >> > > > 102.1. Hm, yes. It's because we want to support batching and at
> > the
> > >> same
> > >> > > > time we
> > >> > > > want to give descriptive error messages for clients. Since
> > >> AdminClient
> > >> > > > holds the context
> > >> > > > to construct such messages (e.g. AdminClient layer can know that
> > >> > > > InvalidArgumentsCode
> > >> > > > means two cases: either invalid number - e.g. -1; or
> > >> replication-factor
> > >> > > was
> > >> > > > provided while
> > >> > > > partitions argument wasn't) - I wrapped responses in Exceptions.
> > >> But I'm
> > >> > > > open to any
> > >> > > > other ideas, this was just initial version.
> > >> > > > 102.2. Yes, I agree. I'll change that to probably some other dto.
> > >> > > >
> > >> > > > Thanks,
> > >> > > > Andrii Biletskyi
> > >> > > >
> > >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io>
> > wrote:
> > >> > > >
> > >> > > > > Andrii,
> > >> > > > >
> > >> > > > > 101. That's what I was thinking too, but it may not be that
> > >> simple. In
> > >> > > > > TopicMetadataRequest_V1,
> > >> > > > > we can let it not trigger auto topic creation. Then, in the
> > >> producer
> > >> > > > side,
> > >> > > > > if it gets an UnknownTopicException, it can explicitly issue a
> > >> > > > > createTopicRequest for auto topic creation. On the consumer
> > side,
> > >> it
> > >> > > will
> > >> > > > > never issue createTopicRequest. This works when auto topic
> > >> creation is
> > >> > > > > enabled on the broker side. However, I am not sure how things
> > >> will work
> > >> > > > > when auto topic creation is disabled on the broker side. In this
> > >> case,
> > >> > > we
> > >> > > > > want to have a way to manually create a topic, potentially
> > through
> > >> > > admin
> > >> > > > > commands. However, then we need a way to distinguish
> > >> createTopicRequest
> > >> > > > > issued from the producer clients and the admin tools. May be we
> > >> can
> > >> > > add a
> > >> > > > > new field in createTopicRequest and set it differently in the
> > >> producer
> > >> > > > > client and the admin client. However, I am not sure if that's
> > the
> > >> best
> > >> > > > > approach.
> > >> > > > >
> > >> > > > > 2. Yes, refactoring existing requests is a non-trivial amount of
> > >> work.
> > >> > > I
> > >> > > > > posted some comments in KAFKA-1927. We will probably have to fix
> > >> > > > KAFKA-1927
> > >> > > > > first, before adding the new logic in KAFKA-1694. Otherwise, the
> > >> > > changes
> > >> > > > > will be too big.
> > >> > > > >
> > >> > > > > 102. About the AdminClient:
> > >> > > > > 102.1. It's a bit weird that we return exception in the api. It
> > >> seems
> > >> > > > that
> > >> > > > > we should either return error code or throw an exception when
> > >> getting
> > >> > > the
> > >> > > > > response state.
> > >> > > > > 102.2. We probably shouldn't explicitly use the request object
> > in
> > >> the
> > >> > > > api.
> > >> > > > > Not every request evolution requires an api change.
> > >> > > > >
> > >> > > > > Thanks,
> > >> > > > >
> > >> > > > > Jun
> > >> > > > >
> > >> > > > >
> > >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > >
> > >> > > > > > Jun,
> > >> > > > > >
> > >> > > > > > Thanks for you comments. Answers inline:
> > >> > > > > >
> > >> > > > > > 100. There are a few fields such as ReplicaAssignment,
> > >> > > > > > > ReassignPartitionRequest,
> > >> > > > > > > and PartitionsSerialized that are represented as a string,
> > but
> > >> > > > contain
> > >> > > > > > > composite structures in json. Could we flatten them out
> > >> directly in
> > >> > > > the
> > >> > > > > > > protocol definition as arrays/records?
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > Yes, now with Admin Client this looks a bit weird. My initial
> > >> > > > motivation
> > >> > > > > > was:
> > >> > > > > > ReassignPartitionCommand accepts input in json, we want to
> > >> remain
> > >> > > > tools'
> > >> > > > > > interfaces unchanged, where possible.
> > >> > > > > > If we port it to deserialized format, in CLI (/tools project)
> > >> we will
> > >> > > > > have
> > >> > > > > > to add some
> > >> > > > > > json library since /tools is written in java and we'll need to
> > >> > > > > deserialize
> > >> > > > > > json file
> > >> > > > > > provided by a user. Can we quickly agree on what this library
> > >> should
> > >> > > be
> > >> > > > > > (Jackson, GSON, whatever)?
> > >> > > > > >
> > >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> > >> creation?
> > >> > > > This
> > >> > > > > > > will be a bit weird now that we have a separate topic
> > >> creation api.
> > >> > > > > Have
> > >> > > > > > > you thought about how the new createTopicRequest and
> > >> > > > > TopicMetadataRequest
> > >> > > > > > > v1 will be used in the producer/consumer client, in addition
> > >> to
> > >> > > admin
> > >> > > > > > > tools? For example, ideally, we don't want
> > >> TopicMetadataRequest
> > >> > > from
> > >> > > > > the
> > >> > > > > > > consumer to trigger auto topic creation.
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > I agree, this strange logic should be fixed. I'm not confident
> > >> in
> > >> > > this
> > >> > > > > > Kafka part so
> > >> > > > > > correct me if I'm wrong, but it doesn't look like a hard thing
> > >> to
> > >> > > do, I
> > >> > > > > > think we can
> > >> > > > > > leverage AdminClient for that in Producer and unconditionally
> > >> remove
> > >> > > > > topic
> > >> > > > > > creation from the TopicMetadataRequest_V1.
> > >> > > > > >
> > >> > > > > > 2. I think Jay meant getting rid of scala classes
> > >> > > > > > > like HeartbeatRequestAndHeader and
> > >> HeartbeatResponseAndHeader. We
> > >> > > did
> > >> > > > > > that
> > >> > > > > > > as a stop-gap thing when adding the new requests for the
> > >> consumers.
> > >> > > > > > > However, the long term plan is to get rid of all those and
> > >> just
> > >> > > reuse
> > >> > > > > the
> > >> > > > > > > java request/response in the client. Since this KIP proposes
> > >> to
> > >> > > add a
> > >> > > > > > > significant number of new requests, perhaps we should bite
> > the
> > >> > > bullet
> > >> > > > > to
> > >> > > > > > > clean up the existing scala requests first before adding new
> > >> ones?
> > >> > > > > > >
> > >> > > > > >
> > >> > > > > > Yes, looks like I misunderstood the point of
> > >> ...RequestAndHeader.
> > >> > > > Okay, I
> > >> > > > > > will
> > >> > > > > > rework that. The only thing is that I don't see any example
> > how
> > >> it
> > >> > > was
> > >> > > > > done
> > >> > > > > > for at
> > >> > > > > > least one existing protocol message. Thus, as I understand, I
> > >> have to
> > >> > > > > think
> > >> > > > > > how we
> > >> > > > > > are going to do it.
> > >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
> > reasonable,
> > >> but
> > >> > > if
> > >> > > > > it's
> > >> > > > > > an *obligatory*
> > >> > > > > > requirement to have Admin KIP done, I'm afraid this can be a
> > >> serious
> > >> > > > > > blocker for us.
> > >> > > > > > There are 13 protocol messages and all that would require not
> > >> only
> > >> > > unit
> > >> > > > > > tests but quite
> > >> > > > > > intensive manual testing, no? I'm afraid I'm not the right guy
> > >> to
> > >> > > cover
> > >> > > > > > pretty much all
> > >> > > > > > Kafka core internals :). Let me know your thoughts on this
> > >> item. Btw
> > >> > > > > there
> > >> > > > > > is a ticket to
> > >> > > > > > follow-up this issue (
> > >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > >> > > > ).
> > >> > > > > >
> > >> > > > > > Thanks,
> > >> > > > > > Andrii Biletskyi
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <ju...@confluent.io>
> > >> wrote:
> > >> > > > > >
> > >> > > > > > > Andrii,
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > A few more comments.
> > >> > > > > > >
> > >> > > > > > > 100. There are a few fields such as ReplicaAssignment,
> > >> > > > > > > ReassignPartitionRequest,
> > >> > > > > > > and PartitionsSerialized that are represented as a string,
> > but
> > >> > > > contain
> > >> > > > > > > composite structures in json. Could we flatten them out
> > >> directly in
> > >> > > > the
> > >> > > > > > > protocol definition as arrays/records?
> > >> > > > > > >
> > >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> > >> > > creation?
> > >> > > > > This
> > >> > > > > > > will be a bit weird now that we have a separate topic
> > >> creation api.
> > >> > > > > Have
> > >> > > > > > > you thought about how the new createTopicRequest and
> > >> > > > > TopicMetadataRequest
> > >> > > > > > > v1 will be used in the producer/consumer client, in addition
> > >> to
> > >> > > admin
> > >> > > > > > > tools? For example, ideally, we don't want
> > >> TopicMetadataRequest
> > >> > > from
> > >> > > > > the
> > >> > > > > > > consumer to trigger auto topic creation.
> > >> > > > > > >
> > >> > > > > > > 2. I think Jay meant getting rid of scala classes
> > >> > > > > > > like HeartbeatRequestAndHeader and
> > >> HeartbeatResponseAndHeader. We
> > >> > > did
> > >> > > > > > that
> > >> > > > > > > as a stop-gap thing when adding the new requests for the
> > >> consumers.
> > >> > > > > > > However, the long term plan is to get rid of all those and
> > >> just
> > >> > > reuse
> > >> > > > > the
> > >> > > > > > > java request/response in the client. Since this KIP proposes
> > >> to
> > >> > > add a
> > >> > > > > > > significant number of new requests, perhaps we should bite
> > the
> > >> > > bullet
> > >> > > > > to
> > >> > > > > > > clean up the existing scala requests first before adding new
> > >> ones?
> > >> > > > > > >
> > >> > > > > > > Thanks,
> > >> > > > > > >
> > >> > > > > > > Jun
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > >
> > >> > > > > > > > Hi,
> > >> > > > > > > >
> > >> > > > > > > > As said above - I list again all comments from this thread
> > >> so we
> > >> > > > > > > > can see what's left and finalize all pending issues.
> > >> > > > > > > >
> > >> > > > > > > > Comments from Jay:
> > >> > > > > > > > 1. This is much needed functionality, but there are a lot
> > >> of the
> > >> > > so
> > >> > > > > > let's
> > >> > > > > > > > really think these protocols through. We really want to
> > end
> > >> up
> > >> > > > with a
> > >> > > > > > set
> > >> > > > > > > > of well thought-out, orthoganol apis. For this reason I
> > >> think it
> > >> > > is
> > >> > > > > > > really
> > >> > > > > > > > important to think through the end state even if that
> > >> includes
> > >> > > APIs
> > >> > > > > we
> > >> > > > > > > > won't implement in the first phase.
> > >> > > > > > > >
> > >> > > > > > > > A: Definitely behind this. Would appreciate if there are
> > >> concrete
> > >> > > > > > > comments
> > >> > > > > > > > how this can be improved.
> > >> > > > > > > >
> > >> > > > > > > > 2. Let's please please please wait until we have switched
> > >> the
> > >> > > > server
> > >> > > > > > over
> > >> > > > > > > > to the new java protocol definitions. If we add upteen
> > more
> > >> ad
> > >> > > hoc
> > >> > > > > > scala
> > >> > > > > > > > objects that is just generating more work for the
> > >> conversion we
> > >> > > > know
> > >> > > > > we
> > >> > > > > > > > have to do.
> > >> > > > > > > >
> > >> > > > > > > > A: Fixed in the latest patch - removed scala protocol
> > >> classes.
> > >> > > > > > > >
> > >> > > > > > > > 3. This proposal introduces a new type of optional
> > >> parameter.
> > >> > > This
> > >> > > > is
> > >> > > > > > > > inconsistent with everything else in the protocol where we
> > >> use -1
> > >> > > > or
> > >> > > > > > some
> > >> > > > > > > > other marker value. You could argue either way but let's
> > >> stick
> > >> > > with
> > >> > > > > > that
> > >> > > > > > > > for consistency. For clients that implemented the protocol
> > >> in a
> > >> > > > > better
> > >> > > > > > > way
> > >> > > > > > > > than our scala code these basic primitives are hard to
> > >> change.
> > >> > > > > > > >
> > >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf type and
> > >> changed
> > >> > > > > > protocol
> > >> > > > > > > > accordingly.
> > >> > > > > > > >
> > >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> > >> TopicMetadataRequest
> > >> > > > > which
> > >> > > > > > > has
> > >> > > > > > > > brokers, topics, and partitions. I think we should rename
> > >> that
> > >> > > > > request
> > >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest) and
> > >> include the
> > >> > > id
> > >> > > > > of
> > >> > > > > > > the
> > >> > > > > > > > controller. Or are there other things we could add here?
> > >> > > > > > > >
> > >> > > > > > > > A: I agree. Updated the KIP. Let's extends TopicMetadata
> > to
> > >> > > > version 2
> > >> > > > > > and
> > >> > > > > > > > include controller.
> > >> > > > > > > >
> > >> > > > > > > > 5. We have a tendency to try to make a lot of requests
> > that
> > >> can
> > >> > > > only
> > >> > > > > go
> > >> > > > > > > to
> > >> > > > > > > > particular nodes. This adds a lot of burden for client
> > >> > > > > implementations
> > >> > > > > > > (it
> > >> > > > > > > > sounds easy but each discovery can fail in many parts so
> > it
> > >> ends
> > >> > > up
> > >> > > > > > > being a
> > >> > > > > > > > full state machine to do right). I think we should
> > consider
> > >> > > making
> > >> > > > > > admin
> > >> > > > > > > > commands and ideally as many of the other apis as possible
> > >> > > > available
> > >> > > > > on
> > >> > > > > > > all
> > >> > > > > > > > brokers and just redirect to the controller on the broker
> > >> side.
> > >> > > > > Perhaps
> > >> > > > > > > > there would be a general way to encapsulate this
> > re-routing
> > >> > > > behavior.
> > >> > > > > > > >
> > >> > > > > > > > A: It's a very interesting idea, but seems there are some
> > >> > > concerns
> > >> > > > > > about
> > >> > > > > > > > this
> > >> > > > > > > > feature (like performance considerations, how this will
> > >> > > complicate
> > >> > > > > > server
> > >> > > > > > > > etc).
> > >> > > > > > > > I believe this shouldn't be a blocker. If this feature is
> > >> > > > implemented
> > >> > > > > > at
> > >> > > > > > > > some
> > >> > > > > > > > point it won't affect Admin changes - at least no changes
> > to
> > >> > > public
> > >> > > > > API
> > >> > > > > > > > will be required.
> > >> > > > > > > >
> > >> > > > > > > > 6. We should probably normalize the key value pairs used
> > for
> > >> > > > configs
> > >> > > > > > > rather
> > >> > > > > > > > than embedding a new formatting. So two strings rather
> > than
> > >> one
> > >> > > > with
> > >> > > > > an
> > >> > > > > > > > internal equals sign.
> > >> > > > > > > >
> > >> > > > > > > > A: Fixed in the latest patch - normalized configs and
> > >> changed
> > >> > > > > protocol
> > >> > > > > > > > accordingly.
> > >> > > > > > > >
> > >> > > > > > > > 7. Is the postcondition of these APIs that the command has
> > >> begun
> > >> > > or
> > >> > > > > > that
> > >> > > > > > > > the command has been completed? It is a lot more usable if
> > >> the
> > >> > > > > command
> > >> > > > > > > has
> > >> > > > > > > > been completed so you know that if you create a topic and
> > >> then
> > >> > > > > publish
> > >> > > > > > to
> > >> > > > > > > > it you won't get an exception about there being no such
> > >> topic.
> > >> > > > > > > >
> > >> > > > > > > > A: For long running requests (like reassign partitions) -
> > >> the
> > >> > > post
> > >> > > > > > > > condition is
> > >> > > > > > > > command has begun - so we don't block the client. In case
> > >> of your
> > >> > > > > > > example -
> > >> > > > > > > > topic commands, this will be refactored and topic commands
> > >> will
> > >> > > be
> > >> > > > > > > executed
> > >> > > > > > > > immediately, since the Controller will serve Admin
> > requests
> > >> > > > > > > > (follow-up ticket KAFKA-1777).
> > >> > > > > > > >
> > >> > > > > > > > 8. Describe topic and list topics duplicate a lot of stuff
> > >> in the
> > >> > > > > > > metadata
> > >> > > > > > > > request. Is there a reason to give back topics marked for
> > >> > > > deletion? I
> > >> > > > > > > feel
> > >> > > > > > > > like if we just make the post-condition of the delete
> > >> command be
> > >> > > > that
> > >> > > > > > the
> > >> > > > > > > > topic is deleted that will get rid of the need for this
> > >> right?
> > >> > > And
> > >> > > > it
> > >> > > > > > > will
> > >> > > > > > > > be much more intuitive.
> > >> > > > > > > >
> > >> > > > > > > > A: Fixed in the latest patch - removed topics marked for
> > >> deletion
> > >> > > > in
> > >> > > > > > > > ListTopicsRequest.
> > >> > > > > > > >
> > >> > > > > > > > 9. Should we consider batching these requests? We have
> > >> generally
> > >> > > > > tried
> > >> > > > > > to
> > >> > > > > > > > allow multiple operations to be batched. My suspicion is
> > >> that
> > >> > > > without
> > >> > > > > > > this
> > >> > > > > > > > we will get a lot of code that does something like
> > >> > > > > > > >    for(topic: adminClient.listTopics())
> > >> > > > > > > >       adminClient.describeTopic(topic)
> > >> > > > > > > > this code will work great when you test on 5 topics but
> > not
> > >> do as
> > >> > > > > well
> > >> > > > > > if
> > >> > > > > > > > you have 50k.
> > >> > > > > > > >
> > >> > > > > > > > A: Updated the KIP - please check "Topic Admin Schema"
> > >> section.
> > >> > > > > > > >
> > >> > > > > > > > 10. I think we should also discuss how we want to expose a
> > >> > > > > programmatic
> > >> > > > > > > JVM
> > >> > > > > > > > client api for these operations. Currently people rely on
> > >> > > > AdminUtils
> > >> > > > > > > which
> > >> > > > > > > > is totally sketchy. I think we probably need another
> > client
> > >> under
> > >> > > > > > > clients/
> > >> > > > > > > > that exposes administrative functionality. We will need
> > >> this just
> > >> > > > to
> > >> > > > > > > > properly test the new apis, I suspect. We should figure
> > out
> > >> that
> > >> > > > API.
> > >> > > > > > > >
> > >> > > > > > > > A: Updated the KIP - please check "Admin Client" section
> > >> with an
> > >> > > > > > initial
> > >> > > > > > > > API proposal.
> > >> > > > > > > >
> > >> > > > > > > > 11. The other information that would be really useful to
> > get
> > >> > > would
> > >> > > > be
> > >> > > > > > > > information about partitions--how much data is in the
> > >> partition,
> > >> > > > what
> > >> > > > > > are
> > >> > > > > > > > the segment offsets, what is the log-end offset (i.e. last
> > >> > > offset),
> > >> > > > > > what
> > >> > > > > > > is
> > >> > > > > > > > the compaction point, etc. I think that done right this
> > >> would be
> > >> > > > the
> > >> > > > > > > > successor to the very awkward OffsetRequest we have today.
> > >> > > > > > > >
> > >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest
> > >> patch. I
> > >> > > > > believe
> > >> > > > > > > > this should
> > >> > > > > > > > be resolved in a separate KIP / jira ticket.
> > >> > > > > > > >
> > >> > > > > > > > 12. Generally we can do good error handling without
> > needing
> > >> > > custom
> > >> > > > > > > > server-side
> > >> > > > > > > > messages. I.e. generally the client has the context to
> > know
> > >> that
> > >> > > if
> > >> > > > > it
> > >> > > > > > > got
> > >> > > > > > > > an error that the topic doesn't exist to say "Topic X
> > >> doesn't
> > >> > > > exist"
> > >> > > > > > > rather
> > >> > > > > > > > than "error code 14" (or whatever). Maybe there are
> > specific
> > >> > > cases
> > >> > > > > > where
> > >> > > > > > > > this is hard? If we want to add server-side error messages
> > >> we
> > >> > > > really
> > >> > > > > do
> > >> > > > > > > > need to do this in a consistent way across the protocol.
> > >> > > > > > > >
> > >> > > > > > > > A: Updated the KIP - please check "Protocol Errors"
> > >> section. I
> > >> > > > added
> > >> > > > > > the
> > >> > > > > > > > comprehensive, fine-grained list of error codes.
> > >> > > > > > > >
> > >> > > > > > > > Comments from Guozhang:
> > >> > > > > > > > 13. Describe topic request: it would be great to go beyond
> > >> just
> > >> > > > > > batching
> > >> > > > > > > on
> > >> > > > > > > > topic name regex for this request. For example, a very
> > >> common use
> > >> > > > > case
> > >> > > > > > of
> > >> > > > > > > > the topic command is to list all topics whose config A's
> > >> value is
> > >> > > > B.
> > >> > > > > > With
> > >> > > > > > > > topic name regex then we have to first retrieve __all__
> > >> topics's
> > >> > > > > > > > description info and then filter at the client end, which
> > >> will
> > >> > > be a
> > >> > > > > > huge
> > >> > > > > > > > burden on ZK.
> > >> > > > > > > > AND
> > >> > > > > > > > 14. Config K-Vs in create topic: this is related to the
> > >> previous
> > >> > > > > point;
> > >> > > > > > > > maybe we can add another metadata K-V or just a metadata
> > >> string
> > >> > > > along
> > >> > > > > > > side
> > >> > > > > > > > with config K-V in create topic like we did for offset
> > >> commit
> > >> > > > > request.
> > >> > > > > > > This
> > >> > > > > > > > field can be quite useful in storing information like
> > >> "owner" of
> > >> > > > the
> > >> > > > > > > topic
> > >> > > > > > > > who issue the create command, etc, which is quite
> > important
> > >> for a
> > >> > > > > > > > multi-tenant setting. Then in the describe topic request
> > we
> > >> can
> > >> > > > also
> > >> > > > > > > batch
> > >> > > > > > > > on regex of the metadata field.
> > >> > > > > > > >
> > >> > > > > > > > A: As discussed it is very interesting but can be
> > >> implemented
> > >> > > later
> > >> > > > > > after
> > >> > > > > > > > we have some basic functionality there.
> > >> > > > > > > >
> > >> > > > > > > > 15. Today all the admin operations are async in the sense
> > >> that
> > >> > > > > command
> > >> > > > > > > will
> > >> > > > > > > > return once it is written in ZK, and that is why we need
> > >> extra
> > >> > > > > > > verification
> > >> > > > > > > > like testUtil.waitForTopicCreated() / verify partition
> > >> > > reassignment
> > >> > > > > > > > request, etc. With admin requests we could add a flag to
> > >> enable /
> > >> > > > > > disable
> > >> > > > > > > > synchronous requests; when it is turned on, the response
> > >> will not
> > >> > > > > > return
> > >> > > > > > > > until the request has been completed. And for async
> > >> requests we
> > >> > > can
> > >> > > > > > add a
> > >> > > > > > > > "token" field in the response, and then only need a
> > general
> > >> > > "admin
> > >> > > > > > > > verification request" with the given token to check if the
> > >> async
> > >> > > > > > request
> > >> > > > > > > > has been completed.
> > >> > > > > > > >
> > >> > > > > > > > A: I see your point. My idea was to provide specific
> > >> > > > Verify...Request
> > >> > > > > > per
> > >> > > > > > > > each
> > >> > > > > > > > long running request, where needed. We can do it the way
> > you
> > >> > > > suggest.
> > >> > > > > > The
> > >> > > > > > > > only
> > >> > > > > > > > concern is that introducing a token we again will make
> > >> schema
> > >> > > > > > "dynamic".
> > >> > > > > > > We
> > >> > > > > > > > wanted
> > >> > > > > > > > to do similar thing introducing single AdminRequest for
> > all
> > >> topic
> > >> > > > > > > commands
> > >> > > > > > > > but rejected
> > >> > > > > > > > this idea because we wanted to have schema defined. So
> > this
> > >> is
> > >> > > > more a
> > >> > > > > > > > choice between:
> > >> > > > > > > > a) have fixed schema but introduce each time new
> > >> Verify...Request
> > >> > > > for
> > >> > > > > > > > long-running requests
> > >> > > > > > > > b) use one request for verification but generalize it with
> > >> token
> > >> > > > > > > > I'm fine with whatever decision community come to. Just
> > let
> > >> me
> > >> > > know
> > >> > > > > > your
> > >> > > > > > > > thoughts.
> > >> > > > > > > >
> > >> > > > > > > > Comment from Gwen:
> > >> > > > > > > > 16. Specifically for ownership, I think the plan is to add
> > >> ACL
> > >> > > (it
> > >> > > > > > sounds
> > >> > > > > > > > like you are describing ACL) via an external system
> > (Argus,
> > >> > > > Sentry).
> > >> > > > > > > > I remember KIP-11 described this, but I can't find the KIP
> > >> any
> > >> > > > > longer.
> > >> > > > > > > >
> > >> > > > > > > > A: Okay, no problem. Not sure though how we are going to
> > >> handle
> > >> > > it.
> > >> > > > > > Wait
> > >> > > > > > > > which KIP
> > >> > > > > > > > will be committed first and include changes to
> > >> TopicMetadata from
> > >> > > > the
> > >> > > > > > > later
> > >> > > > > > > > one?
> > >> > > > > > > > Anyway, I added this note to "Open Questions" section so
> > we
> > >> don't
> > >> > > > > miss
> > >> > > > > > > this
> > >> > > > > > > > piece.
> > >> > > > > > > >
> > >> > > > > > > > Thanks,
> > >> > > > > > > > Andrii Biletskyi
> > >> > > > > > > >
> > >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > > >
> > >> > > > > > > > > Hi all,
> > >> > > > > > > > >
> > >> > > > > > > > > Today I uploaded the patch that covers some of the
> > >> discussed
> > >> > > and
> > >> > > > > > agreed
> > >> > > > > > > > > items:
> > >> > > > > > > > > - removed MaybeOf optional type
> > >> > > > > > > > > - switched to java protocol definitions
> > >> > > > > > > > > - simplified messages (normalized configs, removed topic
> > >> marked
> > >> > > > for
> > >> > > > > > > > > deletion)
> > >> > > > > > > > >
> > >> > > > > > > > > I also updated the KIP-4 with respective changes and
> > >> wrote down
> > >> > > > my
> > >> > > > > > > > > proposal for
> > >> > > > > > > > > pending items:
> > >> > > > > > > > > - Batch Admin Operations -> updated Wire Protocol schema
> > >> > > proposal
> > >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
> > >> > > > TopicMetadataRequest
> > >> > > > > > > > > - Admin Client -> updated my initial proposal to reflect
> > >> > > batching
> > >> > > > > > > > > - Error codes -> proposed fine-grained error code
> > instead
> > >> of
> > >> > > > > > > > > AdminRequestFailed
> > >> > > > > > > > >
> > >> > > > > > > > > I will also send a separate email to cover all comments
> > >> from
> > >> > > this
> > >> > > > > > > thread.
> > >> > > > > > > > >
> > >> > > > > > > > > Thanks,
> > >> > > > > > > > > Andrii Biletskyi
> > >> > > > > > > > >
> > >> > > > > > > > >
> > >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> > >> > > > > gshapira@cloudera.com
> > >> > > > > > >
> > >> > > > > > > > > wrote:
> > >> > > > > > > > >
> > >> > > > > > > > >> Found KIP-11 (
> > >> > > > > > > > >>
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > >> > > > > > > > >> )
> > >> > > > > > > > >> It actually specifies changes to the Metadata protocol,
> > >> so
> > >> > > > making
> > >> > > > > > sure
> > >> > > > > > > > >> both KIPs are consistent in this regard will be good.
> > >> > > > > > > > >>
> > >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> > >> > > > > > gshapira@cloudera.com
> > >> > > > > > > >
> > >> > > > > > > > >> wrote:
> > >> > > > > > > > >> > Specifically for ownership, I think the plan is to
> > add
> > >> ACL
> > >> > > (it
> > >> > > > > > > sounds
> > >> > > > > > > > >> > like you are describing ACL) via an external system
> > >> (Argus,
> > >> > > > > > Sentry).
> > >> > > > > > > > >> > I remember KIP-11 described this, but I can't find
> > the
> > >> KIP
> > >> > > any
> > >> > > > > > > longer.
> > >> > > > > > > > >> >
> > >> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting
> > >> information
> > >> > > that
> > >> > > > > > > already
> > >> > > > > > > > >> > exists from Kafka brokers, not on adding information
> > >> that
> > >> > > > > perhaps
> > >> > > > > > > > >> > should exist but doesn't yet?
> > >> > > > > > > > >> >
> > >> > > > > > > > >> > Gwen
> > >> > > > > > > > >> >
> > >> > > > > > > > >> >
> > >> > > > > > > > >> >
> > >> > > > > > > > >> >
> > >> > > > > > > > >> >
> > >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> > >> > > > > > wangguoz@gmail.com>
> > >> > > > > > > > >> wrote:
> > >> > > > > > > > >> >> Folks,
> > >> > > > > > > > >> >>
> > >> > > > > > > > >> >> Just want to elaborate a bit more on the
> > create-topic
> > >> > > > metadata
> > >> > > > > > and
> > >> > > > > > > > >> batching
> > >> > > > > > > > >> >> describe-topic based on config / metadata in my
> > >> previous
> > >> > > > email
> > >> > > > > as
> > >> > > > > > > we
> > >> > > > > > > > >> work
> > >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to have some
> > >> sort of
> > >> > > > > topic
> > >> > > > > > > > >> management
> > >> > > > > > > > >> >> mechanisms, which I think is quite important in a
> > >> > > > multi-tenant
> > >> > > > > /
> > >> > > > > > > > cloud
> > >> > > > > > > > >> >> architecture: today anyone can create topics in a
> > >> shared
> > >> > > > Kafka
> > >> > > > > > > > >> cluster, but
> > >> > > > > > > > >> >> there is no concept or "ownership" of topics that
> > are
> > >> > > created
> > >> > > > > by
> > >> > > > > > > > >> different
> > >> > > > > > > > >> >> users. For example, at LinkedIn we basically
> > >> distinguish
> > >> > > > topic
> > >> > > > > > > owners
> > >> > > > > > > > >> via
> > >> > > > > > > > >> >> some casual topic name prefix, which is a bit
> > awkward
> > >> and
> > >> > > > does
> > >> > > > > > not
> > >> > > > > > > > fly
> > >> > > > > > > > >> as
> > >> > > > > > > > >> >> we scale our customers. It would be great to use
> > >> > > > > describe-topics
> > >> > > > > > > such
> > >> > > > > > > > >> as:
> > >> > > > > > > > >> >>
> > >> > > > > > > > >> >> Describe all topics that is created by me.
> > >> > > > > > > > >> >>
> > >> > > > > > > > >> >> Describe all topics whose retention time is
> > overriden
> > >> to X.
> > >> > > > > > > > >> >>
> > >> > > > > > > > >> >> Describe all topics whose writable group include
> > user
> > >> Y
> > >> > > (this
> > >> > > > > is
> > >> > > > > > > > >> related to
> > >> > > > > > > > >> >> authorization), etc..
> > >> > > > > > > > >> >>
> > >> > > > > > > > >> >> One possible way to achieve this is to add a
> > metadata
> > >> file
> > >> > > in
> > >> > > > > the
> > >> > > > > > > > >> >> create-topic request, whose value will also be
> > >> written ZK
> > >> > > as
> > >> > > > we
> > >> > > > > > > > create
> > >> > > > > > > > >> the
> > >> > > > > > > > >> >> topic; then describe-topics can choose to batch
> > topics
> > >> > > based
> > >> > > > on
> > >> > > > > > 1)
> > >> > > > > > > > name
> > >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata regex,
> > etc.
> > >> > > > > > > > >> >>
> > >> > > > > > > > >> >> Thoughts?
> > >> > > > > > > > >> >>
> > >> > > > > > > > >> >> Guozhang
> > >> > > > > > > > >> >>
> > >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> > >> > > > > > wangguoz@gmail.com>
> > >> > > > > > > > >> wrote:
> > >> > > > > > > > >> >>
> > >> > > > > > > > >> >>> Thanks for the updated wiki. A few comments below:
> > >> > > > > > > > >> >>>
> > >> > > > > > > > >> >>> 1. Error description in response: I think if some
> > >> > > errorCode
> > >> > > > > > could
> > >> > > > > > > > >> indicate
> > >> > > > > > > > >> >>> several different error cases then we should really
> > >> change
> > >> > > > it
> > >> > > > > to
> > >> > > > > > > > >> multiple
> > >> > > > > > > > >> >>> codes. In general the errorCode itself would be
> > >> precise
> > >> > > and
> > >> > > > > > > > >> sufficient for
> > >> > > > > > > > >> >>> describing the server side errors.
> > >> > > > > > > > >> >>>
> > >> > > > > > > > >> >>> 2. Describe topic request: it would be great to go
> > >> beyond
> > >> > > > just
> > >> > > > > > > > >> batching on
> > >> > > > > > > > >> >>> topic name regex for this request. For example, a
> > >> very
> > >> > > > common
> > >> > > > > > use
> > >> > > > > > > > >> case of
> > >> > > > > > > > >> >>> the topic command is to list all topics whose
> > config
> > >> A's
> > >> > > > value
> > >> > > > > > is
> > >> > > > > > > B.
> > >> > > > > > > > >> With
> > >> > > > > > > > >> >>> topic name regex then we have to first retrieve
> > >> __all__
> > >> > > > > topics's
> > >> > > > > > > > >> >>> description info and then filter at the client end,
> > >> which
> > >> > > > will
> > >> > > > > > be
> > >> > > > > > > a
> > >> > > > > > > > >> huge
> > >> > > > > > > > >> >>> burden on ZK.
> > >> > > > > > > > >> >>>
> > >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is related to
> > >> the
> > >> > > > > previous
> > >> > > > > > > > point;
> > >> > > > > > > > >> >>> maybe we can add another metadata K-V or just a
> > >> metadata
> > >> > > > > string
> > >> > > > > > > > along
> > >> > > > > > > > >> side
> > >> > > > > > > > >> >>> with config K-V in create topic like we did for
> > >> offset
> > >> > > > commit
> > >> > > > > > > > >> request. This
> > >> > > > > > > > >> >>> field can be quite useful in storing information
> > like
> > >> > > > "owner"
> > >> > > > > of
> > >> > > > > > > the
> > >> > > > > > > > >> topic
> > >> > > > > > > > >> >>> who issue the create command, etc, which is quite
> > >> > > important
> > >> > > > > for
> > >> > > > > > a
> > >> > > > > > > > >> >>> multi-tenant setting. Then in the describe topic
> > >> request
> > >> > > we
> > >> > > > > can
> > >> > > > > > > also
> > >> > > > > > > > >> batch
> > >> > > > > > > > >> >>> on regex of the metadata field.
> > >> > > > > > > > >> >>>
> > >> > > > > > > > >> >>> 4. Today all the admin operations are async in the
> > >> sense
> > >> > > > that
> > >> > > > > > > > command
> > >> > > > > > > > >> will
> > >> > > > > > > > >> >>> return once it is written in ZK, and that is why we
> > >> need
> > >> > > > extra
> > >> > > > > > > > >> verification
> > >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify
> > >> partition
> > >> > > > > > > reassignment
> > >> > > > > > > > >> >>> request, etc. With admin requests we could add a
> > >> flag to
> > >> > > > > enable
> > >> > > > > > /
> > >> > > > > > > > >> disable
> > >> > > > > > > > >> >>> synchronous requests; when it is turned on, the
> > >> response
> > >> > > > will
> > >> > > > > > not
> > >> > > > > > > > >> return
> > >> > > > > > > > >> >>> until the request has been completed. And for async
> > >> > > requests
> > >> > > > > we
> > >> > > > > > > can
> > >> > > > > > > > >> add a
> > >> > > > > > > > >> >>> "token" field in the response, and then only need a
> > >> > > general
> > >> > > > > > "admin
> > >> > > > > > > > >> >>> verification request" with the given token to check
> > >> if the
> > >> > > > > async
> > >> > > > > > > > >> request
> > >> > > > > > > > >> >>> has been completed.
> > >> > > > > > > > >> >>>
> > >> > > > > > > > >> >>> 5. +1 for extending Metadata request to include
> > >> > > controller /
> > >> > > > > > > > >> coordinator
> > >> > > > > > > > >> >>> information, and then we can remove the
> > >> ConsumerMetadata /
> > >> > > > > > > > >> ClusterMetadata
> > >> > > > > > > > >> >>> requests.
> > >> > > > > > > > >> >>>
> > >> > > > > > > > >> >>> Guozhang
> > >> > > > > > > > >> >>>
> > >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> > >> > > > > > jjkoshy.w@gmail.com>
> > >> > > > > > > > >> wrote:
> > >> > > > > > > > >> >>>
> > >> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't think I
> > >> will be
> > >> > > > > able
> > >> > > > > > to
> > >> > > > > > > > >> make
> > >> > > > > > > > >> >>>> it today, so if notes can be sent out afterward
> > that
> > >> > > would
> > >> > > > be
> > >> > > > > > > > great.
> > >> > > > > > > > >> >>>>
> > >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen
> > >> Shapira
> > >> > > > wrote:
> > >> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking forward
> > >> to
> > >> > > > > chatting
> > >> > > > > > > with
> > >> > > > > > > > >> >>>> everyone :)
> > >> > > > > > > > >> >>>> >
> > >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> > >> > > > > > > joe.stein@stealth.ly>
> > >> > > > > > > > >> wrote:
> > >> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout invite
> > to
> > >> all
> > >> > > > pmc,
> > >> > > > > > > > >> committers
> > >> > > > > > > > >> >>>> and
> > >> > > > > > > > >> >>>> > > everyone I found working on a KIP. If I missed
> > >> anyone
> > >> > > > in
> > >> > > > > > the
> > >> > > > > > > > >> invite
> > >> > > > > > > > >> >>>> please
> > >> > > > > > > > >> >>>> > > let me know and can update it, np.
> > >> > > > > > > > >> >>>> > >
> > >> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm Eastern
> > >> Time.
> > >> > > > Maybe
> > >> > > > > > we
> > >> > > > > > > > can
> > >> > > > > > > > >> get
> > >> > > > > > > > >> >>>> INFRA
> > >> > > > > > > > >> >>>> > > help to make a google account so we can manage
> > >> > > better?
> > >> > > > > > > > >> >>>> > >
> > >> > > > > > > > >> >>>> > > To discuss
> > >> > > > > > > > >> >>>> > >
> > >> > > > > > > > >> >>>>
> > >> > > > > > > > >>
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > >> > > > > > > > >> >>>> > > in progress and related JIRA that are
> > >> interdependent
> > >> > > > and
> > >> > > > > > > common
> > >> > > > > > > > >> work.
> > >> > > > > > > > >> >>>> > >
> > >> > > > > > > > >> >>>> > > ~ Joe Stein
> > >> > > > > > > > >> >>>> > >
> > >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
> > >> > > > > > > > jay.kreps@gmail.com>
> > >> > > > > > > > >> >>>> wrote:
> > >> > > > > > > > >> >>>> > >
> > >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that will also
> > >> record
> > >> > > > and
> > >> > > > > > make
> > >> > > > > > > > the
> > >> > > > > > > > >> >>>> sessions
> > >> > > > > > > > >> >>>> > >> available on youtube.
> > >> > > > > > > > >> >>>> > >>
> > >> > > > > > > > >> >>>> > >> -Jay
> > >> > > > > > > > >> >>>> > >>
> > >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff
> > Holoman
> > >> <
> > >> > > > > > > > >> >>>> jholoman@cloudera.com>
> > >> > > > > > > > >> >>>> > >> wrote:
> > >> > > > > > > > >> >>>> > >>
> > >> > > > > > > > >> >>>> > >> > Jay / Joe
> > >> > > > > > > > >> >>>> > >> >
> > >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for this
> > >> purpose.
> > >> > > We
> > >> > > > > > could
> > >> > > > > > > > >> record
> > >> > > > > > > > >> >>>> the
> > >> > > > > > > > >> >>>> > >> > sessions if there is interest and publish
> > >> them
> > >> > > out.
> > >> > > > > > > > >> >>>> > >> >
> > >> > > > > > > > >> >>>> > >> > Thanks
> > >> > > > > > > > >> >>>> > >> >
> > >> > > > > > > > >> >>>> > >> > Jeff
> > >> > > > > > > > >> >>>> > >> >
> > >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay
> > Kreps <
> > >> > > > > > > > >> jay.kreps@gmail.com>
> > >> > > > > > > > >> >>>> wrote:
> > >> > > > > > > > >> >>>> > >> >
> > >> > > > > > > > >> >>>> > >> > > Let's try to get the technical hang-ups
> > >> sorted
> > >> > > > out,
> > >> > > > > > > > though.
> > >> > > > > > > > >> I
> > >> > > > > > > > >> >>>> really
> > >> > > > > > > > >> >>>> > >> > think
> > >> > > > > > > > >> >>>> > >> > > there is some benefit to live discussion
> > vs
> > >> > > > > writing. I
> > >> > > > > > > am
> > >> > > > > > > > >> >>>> hopeful that
> > >> > > > > > > > >> >>>> > >> if
> > >> > > > > > > > >> >>>> > >> > > we post instructions and give ourselves a
> > >> few
> > >> > > > > attempts
> > >> > > > > > > we
> > >> > > > > > > > >> can
> > >> > > > > > > > >> >>>> get it
> > >> > > > > > > > >> >>>> > >> > > working.
> > >> > > > > > > > >> >>>> > >> > >
> > >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work for
> > >> me...any
> > >> > > > > > objections?
> > >> > > > > > > > >> >>>> > >> > >
> > >> > > > > > > > >> >>>> > >> > > -Jay
> > >> > > > > > > > >> >>>> > >> > >
> > >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe
> > Stein
> > >> <
> > >> > > > > > > > >> joe.stein@stealth.ly
> > >> > > > > > > > >> >>>> >
> > >> > > > > > > > >> >>>> > >> wrote:
> > >> > > > > > > > >> >>>> > >> > >
> > >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe like every
> > >> > > Tuesday ~
> > >> > > > > 1pm
> > >> > > > > > > ET
> > >> > > > > > > > /
> > >> > > > > > > > >> 10am
> > >> > > > > > > > >> >>>> PT
> > >> > > > > > > > >> >>>> > >> ????
> > >> > > > > > > > >> >>>> > >> > > >
> > >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but there
> > is
> > >> > > always
> > >> > > > > some
> > >> > > > > > > > >> issue or
> > >> > > > > > > > >> >>>> > >> whatever
> > >> > > > > > > > >> >>>> > >> > so
> > >> > > > > > > > >> >>>> > >> > > > we know the apache irc channel works.
> > We
> > >> can
> > >> > > > start
> > >> > > > > > > there
> > >> > > > > > > > >> and
> > >> > > > > > > > >> >>>> see how
> > >> > > > > > > > >> >>>> > >> it
> > >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too and
> > >> > > associate
> > >> > > > to
> > >> > > > > > > > >> tickets if
> > >> > > > > > > > >> >>>> need be
> > >> > > > > > > > >> >>>> > >> > > makes
> > >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> > >> > > > > > > > >> >>>> > >> > > >
> > >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > >> > > > > > > > >> >>>> > >> > > >
> > >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay
> > >> Kreps <
> > >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > >> > > > > > > > >> >>>> > >> > wrote:
> > >> > > > > > > > >> >>>> > >> > > >
> > >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a Google
> > >> Hangout to
> > >> > > > chat
> > >> > > > > > > about
> > >> > > > > > > > >> this.
> > >> > > > > > > > >> >>>> What
> > >> > > > > > > > >> >>>> > >> > about
> > >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
> > further...I
> > >> > > > actually
> > >> > > > > > > think
> > >> > > > > > > > it
> > >> > > > > > > > >> >>>> would be
> > >> > > > > > > > >> >>>> > >> > good
> > >> > > > > > > > >> >>>> > >> > > > for
> > >> > > > > > > > >> >>>> > >> > > > > everyone spending a reasonable chunk
> > of
> > >> > > their
> > >> > > > > week
> > >> > > > > > > on
> > >> > > > > > > > >> Kafka
> > >> > > > > > > > >> >>>> stuff
> > >> > > > > > > > >> >>>> > >> to
> > >> > > > > > > > >> >>>> > >> > > > maybe
> > >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think we could
> > >> use
> > >> > > time
> > >> > > > > to
> > >> > > > > > > talk
> > >> > > > > > > > >> >>>> through
> > >> > > > > > > > >> >>>> > >> design
> > >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of
> > code
> > >> > > > reviews,
> > >> > > > > > talk
> > >> > > > > > > > >> through
> > >> > > > > > > > >> >>>> any
> > >> > > > > > > > >> >>>> > >> > tricky
> > >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > >> > > > > > > > >> >>>> > >> > > > >
> > >> > > > > > > > >> >>>> > >> > > > > We can make it publicly available so
> > >> that
> > >> > > any
> > >> > > > > one
> > >> > > > > > > can
> > >> > > > > > > > >> follow
> > >> > > > > > > > >> >>>> along
> > >> > > > > > > > >> >>>> > >> > who
> > >> > > > > > > > >> >>>> > >> > > > > likes.
> > >> > > > > > > > >> >>>> > >> > > > >
> > >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this? If so
> > I'll
> > >> try
> > >> > > to
> > >> > > > > set
> > >> > > > > > it
> > >> > > > > > > > up
> > >> > > > > > > > >> >>>> starting
> > >> > > > > > > > >> >>>> > >> next
> > >> > > > > > > > >> >>>> > >> > > > week.
> > >> > > > > > > > >> >>>> > >> > > > >
> > >> > > > > > > > >> >>>> > >> > > > > -Jay
> > >> > > > > > > > >> >>>> > >> > > > >
> > >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM,
> > Andrii
> > >> > > > > Biletskyi
> > >> > > > > > <
> > >> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > > > >> >>>> > >> > > > >
> > >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > >> > > > > > > > >> >>>> > >> > > > > >
> > >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed /
> > >> aligned
> > >> > > > > document
> > >> > > > > > > > >> structure.
> > >> > > > > > > > >> >>>> Also I
> > >> > > > > > > > >> >>>> > >> > > added
> > >> > > > > > > > >> >>>> > >> > > > > > some
> > >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
> > >> AdminClient so
> > >> > > we
> > >> > > > > have
> > >> > > > > > > > >> something
> > >> > > > > > > > >> >>>> to
> > >> > > > > > > > >> >>>> > >> start
> > >> > > > > > > > >> >>>> > >> > > > from
> > >> > > > > > > > >> >>>> > >> > > > > > while
> > >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > >> > > > > > > > >> >>>> > >> > > > > >
> > >> > > > > > > > >> >>>> > >> > > > > >
> > >> > > > > > > > >> >>>> > >> > > > > >
> > >> > > > > > > > >> >>>> > >> > > > >
> > >> > > > > > > > >> >>>> > >> > > >
> > >> > > > > > > > >> >>>> > >> > >
> > >> > > > > > > > >> >>>> > >> >
> > >> > > > > > > > >> >>>> > >>
> > >> > > > > > > > >> >>>>
> > >> > > > > > > > >>
> > >> > > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >>
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >> > > > > > > > >> >>>> > >> > > > > >
> > >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > >> > > > > > > > >> >>>> > >> > > > > >
> > >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM,
> > >> Andrii
> > >> > > > > > Biletskyi
> > >> > > > > > > <
> > >> > > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly>
> > wrote:
> > >> > > > > > > > >> >>>> > >> > > > > >
> > >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > >> > > > > > > > >> >>>> > >> > > > > > >
> > >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you are right,
> > >> in
> > >> > > most
> > >> > > > > > cases
> > >> > > > > > > > >> client
> > >> > > > > > > > >> >>>> will
> > >> > > > > > > > >> >>>> > >> have
> > >> > > > > > > > >> >>>> > >> > > > enough
> > >> > > > > > > > >> >>>> > >> > > > > > > context to show descriptive error
> > >> > > message.
> > >> > > > > My
> > >> > > > > > > > >> concern is
> > >> > > > > > > > >> >>>> that
> > >> > > > > > > > >> >>>> > >> we
> > >> > > > > > > > >> >>>> > >> > > will
> > >> > > > > > > > >> >>>> > >> > > > > > have
> > >> > > > > > > > >> >>>> > >> > > > > > > to
> > >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error codes for
> > >> each
> > >> > > > > possible
> > >> > > > > > > > >> error. Of
> > >> > > > > > > > >> >>>> course,
> > >> > > > > > > > >> >>>> > >> > we
> > >> > > > > > > > >> >>>> > >> > > > > could
> > >> > > > > > > > >> >>>> > >> > > > > > > reuse
> > >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
> > >> > > > > > > UknownTopicOrPartitionCode,
> > >> > > > > > > > >> but we
> > >> > > > > > > > >> >>>> will
> > >> > > > > > > > >> >>>> > >> > also
> > >> > > > > > > > >> >>>> > >> > > > need
> > >> > > > > > > > >> >>>> > >> > > > > > to
> > >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> > >> TopicAlreadyExistsCode,
> > >> > > > > > > > >> >>>> TopicConfigInvalid (both
> > >> > > > > > > > >> >>>> > >> > for
> > >> > > > > > > > >> >>>> > >> > > > > topic
> > >> > > > > > > > >> >>>> > >> > > > > > > name and config, and probably
> > user
> > >> would
> > >> > > > > like
> > >> > > > > > to
> > >> > > > > > > > >> know
> > >> > > > > > > > >> >>>> what
> > >> > > > > > > > >> >>>> > >> > exactly
> > >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
> > >> > > > > > > InvalidReplicaAssignment,
> > >> > > > > > > > >> >>>> > >> InternalError
> > >> > > > > > > > >> >>>> > >> > > > (e.g.
> > >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
> > TopicCommand,
> > >> we
> > >> > > will
> > >> > > > > > also
> > >> > > > > > > > >> need to
> > >> > > > > > > > >> >>>> add
> > >> > > > > > > > >> >>>> > >> > similar
> > >> > > > > > > > >> >>>> > >> > > > > stuff
> > >> > > > > > > > >> >>>> > >> > > > > > > for
> > >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> > >> PreferredReplica. So
> > >> > > > > we'll
> > >> > > > > > > end
> > >> > > > > > > > >> up
> > >> > > > > > > > >> >>>> with a
> > >> > > > > > > > >> >>>> > >> > large
> > >> > > > > > > > >> >>>> > >> > > > list
> > >> > > > > > > > >> >>>> > >> > > > > > of
> > >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in Admin
> > >> > > protocol.
> > >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree my
> > >> proposal is
> > >> > > > not
> > >> > > > > > > > >> consistent
> > >> > > > > > > > >> >>>> with
> > >> > > > > > > > >> >>>> > >> > other
> > >> > > > > > > > >> >>>> > >> > > > > cases.
> > >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better solution
> > >> or
> > >> > > > > something
> > >> > > > > > > > >> >>>> in-between.
> > >> > > > > > > > >> >>>> > >> > > > > > >
> > >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it is a
> > >> great
> > >> > > > idea.
> > >> > > > > > > This
> > >> > > > > > > > >> way we
> > >> > > > > > > > >> >>>> can
> > >> > > > > > > > >> >>>> > >> move
> > >> > > > > > > > >> >>>> > >> > > on
> > >> > > > > > > > >> >>>> > >> > > > > > > faster.
> > >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on date/time
> > so
> > >> > > people
> > >> > > > > can
> > >> > > > > > > > join.
> > >> > > > > > > > >> >>>> Will work
> > >> > > > > > > > >> >>>> > >> > for
> > >> > > > > > > > >> >>>> > >> > > me
> > >> > > > > > > > >> >>>> > >> > > > > > this
> > >> > > > > > > > >> >>>> > >> > > > > > > and
> > >> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime if
> > agreed
> > >> in
> > >> > > > > advance.
> > >> > > > > > > > >> >>>> > >> > > > > > >
> > >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > >> > > > > > > > >> >>>> > >> > > > > > >
> > >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM,
> > >> Jay
> > >> > > > Kreps <
> > >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > >> > > > > > > > >> >>>> > >> > > > > wrote:
> > >> > > > > > > > >> >>>> > >> > > > > > >
> > >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > >> > > > > > > > >> >>>> > >> > > > > > >>
> > >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good error
> > >> handling
> > >> > > > > > without
> > >> > > > > > > > >> needing
> > >> > > > > > > > >> >>>> custom
> > >> > > > > > > > >> >>>> > >> > > > > > server-side
> > >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the
> > >> client has
> > >> > > > the
> > >> > > > > > > > >> context to
> > >> > > > > > > > >> >>>> know
> > >> > > > > > > > >> >>>> > >> that
> > >> > > > > > > > >> >>>> > >> > > if
> > >> > > > > > > > >> >>>> > >> > > > it
> > >> > > > > > > > >> >>>> > >> > > > > > got
> > >> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic doesn't
> > >> exist
> > >> > > to
> > >> > > > > say
> > >> > > > > > > > >> "Topic X
> > >> > > > > > > > >> >>>> doesn't
> > >> > > > > > > > >> >>>> > >> > > exist"
> > >> > > > > > > > >> >>>> > >> > > > > > >> rather
> > >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or
> > >> whatever).
> > >> > > Maybe
> > >> > > > > > there
> > >> > > > > > > > are
> > >> > > > > > > > >> >>>> specific
> > >> > > > > > > > >> >>>> > >> > cases
> > >> > > > > > > > >> >>>> > >> > > > > where
> > >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to add
> > >> > > > server-side
> > >> > > > > > > error
> > >> > > > > > > > >> >>>> messages we
> > >> > > > > > > > >> >>>> > >> > > really
> > >> > > > > > > > >> >>>> > >> > > > > do
> > >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a consistent
> > >> way
> > >> > > > across
> > >> > > > > > the
> > >> > > > > > > > >> protocol.
> > >> > > > > > > > >> >>>> > >> > > > > > >>
> > >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open
> > >> questions
> > >> > > > here
> > >> > > > > > > from
> > >> > > > > > > > my
> > >> > > > > > > > >> >>>> previous
> > >> > > > > > > > >> >>>> > >> > > list. I
> > >> > > > > > > > >> >>>> > >> > > > > > will
> > >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few days for
> > >> Strata
> > >> > > > > > though.
> > >> > > > > > > > >> Maybe
> > >> > > > > > > > >> >>>> we could
> > >> > > > > > > > >> >>>> > >> > do
> > >> > > > > > > > >> >>>> > >> > > a
> > >> > > > > > > > >> >>>> > >> > > > > > Google
> > >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open issues
> > >> some
> > >> > > time
> > >> > > > > > > towards
> > >> > > > > > > > >> the
> > >> > > > > > > > >> >>>> end of
> > >> > > > > > > > >> >>>> > >> > next
> > >> > > > > > > > >> >>>> > >> > > > week
> > >> > > > > > > > >> >>>> > >> > > > > > for
> > >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in this
> > ticket?
> > >> I
> > >> > > have
> > >> > > > a
> > >> > > > > > > > feeling
> > >> > > > > > > > >> that
> > >> > > > > > > > >> >>>> might
> > >> > > > > > > > >> >>>> > >> > > > progress
> > >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster than
> > >> email--I
> > >> > > > think
> > >> > > > > we
> > >> > > > > > > > >> could talk
> > >> > > > > > > > >> >>>> > >> through
> > >> > > > > > > > >> >>>> > >> > > > those
> > >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly
> > >> quickly...
> > >> > > > > > > > >> >>>> > >> > > > > > >>
> > >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> > >> > > > > > > > >> >>>> > >> > > > > > >>
> > >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM,
> > >> Andrii
> > >> > > > > > > > Biletskyi <
> > >> > > > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly>
> > >> wrote:
> > >> > > > > > > > >> >>>> > >> > > > > > >>
> > >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address some of
> > >> the
> > >> > > > issues
> > >> > > > > > > which
> > >> > > > > > > > >> were
> > >> > > > > > > > >> >>>> > >> mentioned
> > >> > > > > > > > >> >>>> > >> > > > > earlier
> > >> > > > > > > > >> >>>> > >> > > > > > >> about
> > >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of
> > >> those was
> > >> > > > > about
> > >> > > > > > > > >> batching
> > >> > > > > > > > >> >>>> > >> > operations.
> > >> > > > > > > > >> >>>> > >> > > > What
> > >> > > > > > > > >> >>>> > >> > > > > > if
> > >> > > > > > > > >> >>>> > >> > > > > > >> we
> > >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand approach
> > >> and let
> > >> > > > > people
> > >> > > > > > > > >> specify
> > >> > > > > > > > >> >>>> > >> topic-name
> > >> > > > > > > > >> >>>> > >> > > by
> > >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> > >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most of the
> > use
> > >> > > cases?
> > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what information
> > >> should
> > >> > > we
> > >> > > > > > > > generally
> > >> > > > > > > > >> >>>> provide in
> > >> > > > > > > > >> >>>> > >> > > Admin
> > >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> > >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin commands
> > >> don't
> > >> > > > imply
> > >> > > > > > > they
> > >> > > > > > > > >> will
> > >> > > > > > > > >> >>>> be used
> > >> > > > > > > > >> >>>> > >> > only
> > >> > > > > > > > >> >>>> > >> > > > in
> > >> > > > > > > > >> >>>> > >> > > > > > CLI
> > >> > > > > > > > >> >>>> > >> > > > > > >> > but,
> > >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a very
> > >> > > important
> > >> > > > > > > client
> > >> > > > > > > > >> of this
> > >> > > > > > > > >> >>>> > >> > feature.
> > >> > > > > > > > >> >>>> > >> > > In
> > >> > > > > > > > >> >>>> > >> > > > > > this
> > >> > > > > > > > >> >>>> > >> > > > > > >> > case,
> > >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would like
> > to
> > >> > > provide
> > >> > > > > > users
> > >> > > > > > > > >> with
> > >> > > > > > > > >> >>>> rich
> > >> > > > > > > > >> >>>> > >> > > experience
> > >> > > > > > > > >> >>>> > >> > > > > in
> > >> > > > > > > > >> >>>> > >> > > > > > >> terms
> > >> > > > > > > > >> >>>> > >> > > > > > >> > of
> > >> > > > > > > > >> >>>> > >> > > > > > >> > getting results / errors of
> > the
> > >> > > > executed
> > >> > > > > > > > >> commands.
> > >> > > > > > > > >> >>>> Usually
> > >> > > > > > > > >> >>>> > >> we
> > >> > > > > > > > >> >>>> > >> > > > supply
> > >> > > > > > > > >> >>>> > >> > > > > > >> with
> > >> > > > > > > > >> >>>> > >> > > > > > >> > responses only errorCode,
> > which
> > >> looks
> > >> > > > > very
> > >> > > > > > > > >> limiting,
> > >> > > > > > > > >> >>>> in case
> > >> > > > > > > > >> >>>> > >> > of
> > >> > > > > > > > >> >>>> > >> > > > CLI
> > >> > > > > > > > >> >>>> > >> > > > > we
> > >> > > > > > > > >> >>>> > >> > > > > > >> may
> > >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human readable
> > >> error
> > >> > > > > > > description.
> > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into account
> > >> previous item
> > >> > > > > about
> > >> > > > > > > > >> batching,
> > >> > > > > > > > >> >>>> what
> > >> > > > > > > > >> >>>> > >> do
> > >> > > > > > > > >> >>>> > >> > > you
> > >> > > > > > > > >> >>>> > >> > > > > > think
> > >> > > > > > > > >> >>>> > >> > > > > > >> > about
> > >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support
> > >> regexp)
> > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest =>
> > TopicName
> > >> > > > > Partitions
> > >> > > > > > > > >> Replicas
> > >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> > >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse =>
> > ErrorCode
> > >> > > > > > > > ErrorDescription
> > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription => string
> > >> (empty
> > >> > > if
> > >> > > > > > > > >> successful)
> > >> > > > > > > > >> >>>> > >> > > > > > >> >
> > >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
> > >> TopicNameRegexp
> > >> > > >
> > >
> > > ...
> > >
> > > [Message clipped]
> >


Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Gwen Shapira <gs...@cloudera.com>.
Got it. Thanks for clarifying!

On Wed, Mar 18, 2015 at 11:54 AM, Andrii Biletskyi
<an...@stealth.ly> wrote:
> Gwen,
>
> Yes, looks like KAFKA-1927 will leave TopicMetadataRequest/Response.
> But I believe, KIP is still tightly related with KAFKA-1927 since we are
> not only
> going to update TopicMetadataRequest there but we will introduce a bunch
> of new requests too. And it probably makes sense to do those correctly from
> scratch - without introducing scala request objects. As I understand you'll
> have this common infrastructure code done in KAFKA-1927.
>
> Thanks,
> Andrii Biletskyi
>
> On Wed, Mar 18, 2015 at 8:38 PM, Gwen Shapira <gs...@cloudera.com> wrote:
>
>> On Wed, Mar 18, 2015 at 9:34 AM, Jun Rao <ju...@confluent.io> wrote:
>> > Andri,
>> >
>> > Thanks for the summary.
>> >
>> > 1. I just realized that in order to start working on KAFKA-1927, we will
>> > need to merge the changes to OffsetCommitRequest (from 0.8.2) to trunk.
>> > This is planned to be done as part of KAFKA-1634. So, we will need
>> Guozhang
>> > and Joel's help to wrap this up.
>>
>> I mentioned this in a separate thread, but it may be more relevant here:
>> It looks like the SimpleConsumer API exposes TopicMetadataRequest and
>> TopicMetadataResponse.
>> This means that KAFKA-1927 doesn't remove this duplication.
>>
>> So I'm not sure we actually need KAFKA-1927 before implementing this KIP.
>> This doesn't mean I'm stopping work on KAFKA-1927, but perhaps it
>> means we can proceed in parallel?
>>
>> > 2. Thinking about this a bit more, if the semantic of those "write"
>> > requests is async (i.e., after the client gets a response, it just means
>> > that the operation is initiated, but not necessarily completed), we don't
>> > really need to forward the requests to the controller. Instead, the
>> > receiving broker can just write the operation to ZK as the admin command
>> > line tool previously does. This will simplify the implementation.
>> >
>> > 8. There is another implementation detail for describe topic. Ideally, we
>> > want to read the topic config from the broker cache, instead of
>> ZooKeeper.
>> > Currently, every broker reads the topic-level config for all topics.
>> > However, it ignores those for topics not hosted on itself. So, we may
>> need
>> > to change TopicConfigManager a bit so that it caches the configs for all
>> > topics.
>> >
>> > Thanks,
>> >
>> > Jun
>> >
>> >
>> > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
>> > andrii.biletskyi@stealth.ly> wrote:
>> >
>> >> Guys,
>> >>
>> >> Thanks for a great discussion!
>> >> Here are the actions points:
>> >>
>> >> 1. Q: Get rid of all scala requests objects, use java protocol
>> definitions.
>> >>     A: Gwen kindly took that (KAFKA-1927). It's important to speed up
>> >> review procedure
>> >>          there since this ticket blocks other important changes.
>> >>
>> >> 2. Q: Generic re-reroute facility vs client maintaining cluster state.
>> >>     A: Jay has added pseudo code to KAFKA-1912 - need to consider
>> whether
>> >> this will be
>> >>         easy to implement as a server-side feature (comments are
>> >> welcomed!).
>> >>
>> >> 3. Q: Controller field in wire protocol.
>> >>     A: This might be useful for clients, add this to
>> TopicMetadataResponse
>> >> (already in KIP).
>> >>
>> >> 4. Q: Decoupling topic creation from TMR.
>> >>     A: I will add proposed by Jun solution (using clientId for that) to
>> the
>> >> KIP.
>> >>
>> >> 5. Q: Bumping new versions of TMR vs grabbing all protocol changes in
>> one
>> >> version.
>> >>     A: It was decided to try to gather all changes to protocol (before
>> >> release).
>> >>         In case of TMR it worth checking: KAFKA-2020 and KIP-13 (quotas)
>> >>
>> >> 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
>> >>     A: Use jackson for that, /tools project is a separate jar so
>> shouldn't
>> >> be a big deal.
>> >>
>> >> 7.  Q: VerifyReassingPartitions vs generic status check command.
>> >>      A: For long-running requests like reassign partitions *progress*
>> check
>> >> request is useful,
>> >>          it makes sense to introduce it.
>> >>
>> >>  Please add, correct me if I missed something.
>> >>
>> >> Thanks,
>> >> Andrii Biletskyi
>> >>
>> >> On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
>> >> andrii.biletskyi@stealth.ly> wrote:
>> >>
>> >> > Joel,
>> >> >
>> >> > You are right, I removed ClusterMetadata because we have partially
>> >> > what we need in TopicMetadata. Also, as Jay pointed out earlier, we
>> >> > would like to have "orthogonal" API, but at the same time we need
>> >> > to be backward compatible.
>> >> >
>> >> > But I like your idea and even have some other arguments for this
>> option:
>> >> > There is also DescribeTopicRequest which was proposed in this KIP,
>> >> > it returns topic configs, partitions, replication factor plus
>> partition
>> >> > ISR, ASR,
>> >> > leader replica. The later part is really already there in
>> >> > TopicMetadataRequest.
>> >> > So again we'll have to add stuff to TMR, not to duplicate some info in
>> >> > newly added requests. However, this way we'll end up with "monster"
>> >> > request which returns cluster metadata, topic replication and config
>> info
>> >> > plus partition replication data. Seems logical to split TMR to
>> >> > - ClusterMetadata (brokers + controller, maybe smth else)
>> >> > - TopicMetadata (topic info + partition details)
>> >> > But since current TMR is involved in lots of places (including network
>> >> > client,
>> >> > as I understand) this might be very serious change and it probably
>> makes
>> >> > sense to stick with current approach.
>> >> >
>> >> > Thanks,
>> >> > Andrii Biletskyi
>> >> >
>> >> >
>> >> > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jj...@gmail.com>
>> wrote:
>> >> >
>> >> >> I may be missing some context but hopefully this will also be covered
>> >> >> today: I thought the earlier proposal where there was an explicit
>> >> >> ClusterMetadata request was clearer and explicit. During the course
>> of
>> >> >> this thread I think the conclusion was that the main need was for
>> >> >> controller information and that can be rolled into the topic metadata
>> >> >> response but that seems a bit irrelevant to topic metadata. FWIW I
>> >> >> think the full broker-list is also irrelevant to topic metadata, but
>> >> >> it is already there and in use. I think there is still room for an
>> >> >> explicit ClusterMetadata request since there may be other
>> >> >> cluster-level information that we may want to add over time (and that
>> >> >> have nothing to do with topic metadata).
>> >> >>
>> >> >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
>> >> >> > Jun,
>> >> >> >
>> >> >> > 101. Okay, if you say that such use case is important. I also think
>> >> >> > using clientId for these purposes is fine - if we already have this
>> >> >> field
>> >> >> > as part of all Wire protocol messages, why not use that.
>> >> >> > I will update KIP-4 page if nobody has other ideas (which may come
>> up
>> >> >> > during the call today).
>> >> >> >
>> >> >> > 102.1 Agree, I'll update the KIP accordingly. I think we can add
>> new,
>> >> >> > fine-grained error codes if some error code received in specific
>> case
>> >> >> > won't give enough context to return a descriptive error message for
>> >> >> user.
>> >> >> >
>> >> >> > Look forward to discussing all outstanding issues in detail today
>> >> during
>> >> >> > the call.
>> >> >> >
>> >> >> > Thanks,
>> >> >> > Andrii Biletskyi
>> >> >> >
>> >> >> >
>> >> >> >
>> >> >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io>
>> wrote:
>> >> >> >
>> >> >> > > 101. There may be a use case where you only want the topics to be
>> >> >> created
>> >> >> > > manually by admins. Currently, you can do that by disabling auto
>> >> topic
>> >> >> > > creation and issue topic creation from the TopicCommand. If we
>> >> >> disable auto
>> >> >> > > topic creation completely on the broker and don't have a way to
>> >> >> distinguish
>> >> >> > > between topic creation requests from the regular clients and the
>> >> >> admin, we
>> >> >> > > can't support manual topic creation any more. I was thinking that
>> >> >> another
>> >> >> > > way of distinguishing the clients making the topic creation
>> requests
>> >> >> is
>> >> >> > > using clientId. For example, the admin tool can set it to
>> something
>> >> >> like
>> >> >> > > admin and the broker can treat that clientId specially.
>> >> >> > >
>> >> >> > > Also, there is a related discussion in KAFKA-2020. Currently, we
>> do
>> >> >> the
>> >> >> > > following in TopicMetadataResponse:
>> >> >> > >
>> >> >> > > 1. If leader is not available, we set the partition level error
>> code
>> >> >> to
>> >> >> > > LeaderNotAvailable.
>> >> >> > > 2. If a non-leader replica is not available, we take that replica
>> >> out
>> >> >> of
>> >> >> > > the assigned replica list and isr in the response. As an
>> indication
>> >> >> for
>> >> >> > > doing that, we set the partition level error code to
>> >> >> ReplicaNotAvailable.
>> >> >> > >
>> >> >> > > This has a few problems. First, ReplicaNotAvailable probably
>> >> >> shouldn't be
>> >> >> > > an error, at least for the normal producer/consumer clients that
>> >> just
>> >> >> want
>> >> >> > > to find out the leader. Second, it can happen that both the
>> leader
>> >> and
>> >> >> > > another replica are not available at the same time. There is no
>> >> error
>> >> >> code
>> >> >> > > to indicate both. Third, even if a replica is not available, it's
>> >> >> still
>> >> >> > > useful to return its replica id since some clients (e.g. admin
>> tool)
>> >> >> may
>> >> >> > > still make use of it.
>> >> >> > >
>> >> >> > > One way to address this issue is to always return the replica id
>> for
>> >> >> > > leader, assigned replicas, and isr regardless of whether the
>> >> >> corresponding
>> >> >> > > broker is live or not. Since we also return the list of live
>> >> brokers,
>> >> >> the
>> >> >> > > client can figure out whether a leader or a replica is live or
>> not
>> >> >> and act
>> >> >> > > accordingly. This way, we don't need to set the partition level
>> >> error
>> >> >> code
>> >> >> > > when the leader or a replica is not available. This doesn't
>> change
>> >> >> the wire
>> >> >> > > protocol, but does change the semantics. Since we are evolving
>> the
>> >> >> protocol
>> >> >> > > of TopicMetadataRequest here, we can potentially piggyback the
>> >> change.
>> >> >> > >
>> >> >> > > 102.1 For those types of errors due to invalid input, shouldn't
>> we
>> >> >> just
>> >> >> > > guard it at parameter validation time and throw
>> >> >> InvalidArgumentException
>> >> >> > > without even sending the request to the broker?
>> >> >> > >
>> >> >> > > Thanks,
>> >> >> > >
>> >> >> > > Jun
>> >> >> > >
>> >> >> > >
>> >> >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
>> >> >> > > andrii.biletskyi@stealth.ly> wrote:
>> >> >> > >
>> >> >> > > > Jun,
>> >> >> > > >
>> >> >> > > > Answering your questions:
>> >> >> > > >
>> >> >> > > > 101. If I understand you correctly, you are saying future
>> producer
>> >> >> > > versions
>> >> >> > > > (which
>> >> >> > > > will be ported to TMR_V1) won't be able to automatically create
>> >> >> topic (if
>> >> >> > > > we
>> >> >> > > > unconditionally remove topic creation from there). But we need
>> to
>> >> >> this
>> >> >> > > > preserve logic.
>> >> >> > > > Ok, about your proposal: I'm not a big fan too, when it comes
>> to
>> >> >> > > > differentiating
>> >> >> > > > clients directly in protocol schema. And also I'm not sure I
>> >> >> understand
>> >> >> > > at
>> >> >> > > > all why
>> >> >> > > > auto.create.topics.enable is a server side configuration. Can
>> we
>> >> >> > > deprecate
>> >> >> > > > this setting
>> >> >> > > > in future versions, add this setting to producer and based on
>> that
>> >> >> upon
>> >> >> > > > receiving
>> >> >> > > > UnknownTopic create topic explicitly by a separate producer
>> call
>> >> via
>> >> >> > > > adminClient?
>> >> >> > > >
>> >> >> > > > 102.1. Hm, yes. It's because we want to support batching and at
>> >> the
>> >> >> same
>> >> >> > > > time we
>> >> >> > > > want to give descriptive error messages for clients. Since
>> >> >> AdminClient
>> >> >> > > > holds the context
>> >> >> > > > to construct such messages (e.g. AdminClient layer can know
>> that
>> >> >> > > > InvalidArgumentsCode
>> >> >> > > > means two cases: either invalid number - e.g. -1; or
>> >> >> replication-factor
>> >> >> > > was
>> >> >> > > > provided while
>> >> >> > > > partitions argument wasn't) - I wrapped responses in
>> Exceptions.
>> >> >> But I'm
>> >> >> > > > open to any
>> >> >> > > > other ideas, this was just initial version.
>> >> >> > > > 102.2. Yes, I agree. I'll change that to probably some other
>> dto.
>> >> >> > > >
>> >> >> > > > Thanks,
>> >> >> > > > Andrii Biletskyi
>> >> >> > > >
>> >> >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io>
>> >> wrote:
>> >> >> > > >
>> >> >> > > > > Andrii,
>> >> >> > > > >
>> >> >> > > > > 101. That's what I was thinking too, but it may not be that
>> >> >> simple. In
>> >> >> > > > > TopicMetadataRequest_V1,
>> >> >> > > > > we can let it not trigger auto topic creation. Then, in the
>> >> >> producer
>> >> >> > > > side,
>> >> >> > > > > if it gets an UnknownTopicException, it can explicitly issue
>> a
>> >> >> > > > > createTopicRequest for auto topic creation. On the consumer
>> >> side,
>> >> >> it
>> >> >> > > will
>> >> >> > > > > never issue createTopicRequest. This works when auto topic
>> >> >> creation is
>> >> >> > > > > enabled on the broker side. However, I am not sure how things
>> >> >> will work
>> >> >> > > > > when auto topic creation is disabled on the broker side. In
>> this
>> >> >> case,
>> >> >> > > we
>> >> >> > > > > want to have a way to manually create a topic, potentially
>> >> through
>> >> >> > > admin
>> >> >> > > > > commands. However, then we need a way to distinguish
>> >> >> createTopicRequest
>> >> >> > > > > issued from the producer clients and the admin tools. May be
>> we
>> >> >> can
>> >> >> > > add a
>> >> >> > > > > new field in createTopicRequest and set it differently in the
>> >> >> producer
>> >> >> > > > > client and the admin client. However, I am not sure if that's
>> >> the
>> >> >> best
>> >> >> > > > > approach.
>> >> >> > > > >
>> >> >> > > > > 2. Yes, refactoring existing requests is a non-trivial
>> amount of
>> >> >> work.
>> >> >> > > I
>> >> >> > > > > posted some comments in KAFKA-1927. We will probably have to
>> fix
>> >> >> > > > KAFKA-1927
>> >> >> > > > > first, before adding the new logic in KAFKA-1694. Otherwise,
>> the
>> >> >> > > changes
>> >> >> > > > > will be too big.
>> >> >> > > > >
>> >> >> > > > > 102. About the AdminClient:
>> >> >> > > > > 102.1. It's a bit weird that we return exception in the api.
>> It
>> >> >> seems
>> >> >> > > > that
>> >> >> > > > > we should either return error code or throw an exception when
>> >> >> getting
>> >> >> > > the
>> >> >> > > > > response state.
>> >> >> > > > > 102.2. We probably shouldn't explicitly use the request
>> object
>> >> in
>> >> >> the
>> >> >> > > > api.
>> >> >> > > > > Not every request evolution requires an api change.
>> >> >> > > > >
>> >> >> > > > > Thanks,
>> >> >> > > > >
>> >> >> > > > > Jun
>> >> >> > > > >
>> >> >> > > > >
>> >> >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
>> >> >> > > > > andrii.biletskyi@stealth.ly> wrote:
>> >> >> > > > >
>> >> >> > > > > > Jun,
>> >> >> > > > > >
>> >> >> > > > > > Thanks for you comments. Answers inline:
>> >> >> > > > > >
>> >> >> > > > > > 100. There are a few fields such as ReplicaAssignment,
>> >> >> > > > > > > ReassignPartitionRequest,
>> >> >> > > > > > > and PartitionsSerialized that are represented as a
>> string,
>> >> but
>> >> >> > > > contain
>> >> >> > > > > > > composite structures in json. Could we flatten them out
>> >> >> directly in
>> >> >> > > > the
>> >> >> > > > > > > protocol definition as arrays/records?
>> >> >> > > > > >
>> >> >> > > > > >
>> >> >> > > > > > Yes, now with Admin Client this looks a bit weird. My
>> initial
>> >> >> > > > motivation
>> >> >> > > > > > was:
>> >> >> > > > > > ReassignPartitionCommand accepts input in json, we want to
>> >> >> remain
>> >> >> > > > tools'
>> >> >> > > > > > interfaces unchanged, where possible.
>> >> >> > > > > > If we port it to deserialized format, in CLI (/tools
>> project)
>> >> >> we will
>> >> >> > > > > have
>> >> >> > > > > > to add some
>> >> >> > > > > > json library since /tools is written in java and we'll
>> need to
>> >> >> > > > > deserialize
>> >> >> > > > > > json file
>> >> >> > > > > > provided by a user. Can we quickly agree on what this
>> library
>> >> >> should
>> >> >> > > be
>> >> >> > > > > > (Jackson, GSON, whatever)?
>> >> >> > > > > >
>> >> >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
>> >> >> creation?
>> >> >> > > > This
>> >> >> > > > > > > will be a bit weird now that we have a separate topic
>> >> >> creation api.
>> >> >> > > > > Have
>> >> >> > > > > > > you thought about how the new createTopicRequest and
>> >> >> > > > > TopicMetadataRequest
>> >> >> > > > > > > v1 will be used in the producer/consumer client, in
>> addition
>> >> >> to
>> >> >> > > admin
>> >> >> > > > > > > tools? For example, ideally, we don't want
>> >> >> TopicMetadataRequest
>> >> >> > > from
>> >> >> > > > > the
>> >> >> > > > > > > consumer to trigger auto topic creation.
>> >> >> > > > > >
>> >> >> > > > > >
>> >> >> > > > > > I agree, this strange logic should be fixed. I'm not
>> confident
>> >> >> in
>> >> >> > > this
>> >> >> > > > > > Kafka part so
>> >> >> > > > > > correct me if I'm wrong, but it doesn't look like a hard
>> thing
>> >> >> to
>> >> >> > > do, I
>> >> >> > > > > > think we can
>> >> >> > > > > > leverage AdminClient for that in Producer and
>> unconditionally
>> >> >> remove
>> >> >> > > > > topic
>> >> >> > > > > > creation from the TopicMetadataRequest_V1.
>> >> >> > > > > >
>> >> >> > > > > > 2. I think Jay meant getting rid of scala classes
>> >> >> > > > > > > like HeartbeatRequestAndHeader and
>> >> >> HeartbeatResponseAndHeader. We
>> >> >> > > did
>> >> >> > > > > > that
>> >> >> > > > > > > as a stop-gap thing when adding the new requests for the
>> >> >> consumers.
>> >> >> > > > > > > However, the long term plan is to get rid of all those
>> and
>> >> >> just
>> >> >> > > reuse
>> >> >> > > > > the
>> >> >> > > > > > > java request/response in the client. Since this KIP
>> proposes
>> >> >> to
>> >> >> > > add a
>> >> >> > > > > > > significant number of new requests, perhaps we should
>> bite
>> >> the
>> >> >> > > bullet
>> >> >> > > > > to
>> >> >> > > > > > > clean up the existing scala requests first before adding
>> new
>> >> >> ones?
>> >> >> > > > > > >
>> >> >> > > > > >
>> >> >> > > > > > Yes, looks like I misunderstood the point of
>> >> >> ...RequestAndHeader.
>> >> >> > > > Okay, I
>> >> >> > > > > > will
>> >> >> > > > > > rework that. The only thing is that I don't see any example
>> >> how
>> >> >> it
>> >> >> > > was
>> >> >> > > > > done
>> >> >> > > > > > for at
>> >> >> > > > > > least one existing protocol message. Thus, as I
>> understand, I
>> >> >> have to
>> >> >> > > > > think
>> >> >> > > > > > how we
>> >> >> > > > > > are going to do it.
>> >> >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
>> >> reasonable,
>> >> >> but
>> >> >> > > if
>> >> >> > > > > it's
>> >> >> > > > > > an *obligatory*
>> >> >> > > > > > requirement to have Admin KIP done, I'm afraid this can be
>> a
>> >> >> serious
>> >> >> > > > > > blocker for us.
>> >> >> > > > > > There are 13 protocol messages and all that would require
>> not
>> >> >> only
>> >> >> > > unit
>> >> >> > > > > > tests but quite
>> >> >> > > > > > intensive manual testing, no? I'm afraid I'm not the right
>> guy
>> >> >> to
>> >> >> > > cover
>> >> >> > > > > > pretty much all
>> >> >> > > > > > Kafka core internals :). Let me know your thoughts on this
>> >> >> item. Btw
>> >> >> > > > > there
>> >> >> > > > > > is a ticket to
>> >> >> > > > > > follow-up this issue (
>> >> >> > > https://issues.apache.org/jira/browse/KAFKA-2006
>> >> >> > > > ).
>> >> >> > > > > >
>> >> >> > > > > > Thanks,
>> >> >> > > > > > Andrii Biletskyi
>> >> >> > > > > >
>> >> >> > > > > >
>> >> >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <jun@confluent.io
>> >
>> >> >> wrote:
>> >> >> > > > > >
>> >> >> > > > > > > Andrii,
>> >> >> > > > > > >
>> >> >> > > > > > >
>> >> >> > > > > > > A few more comments.
>> >> >> > > > > > >
>> >> >> > > > > > > 100. There are a few fields such as ReplicaAssignment,
>> >> >> > > > > > > ReassignPartitionRequest,
>> >> >> > > > > > > and PartitionsSerialized that are represented as a
>> string,
>> >> but
>> >> >> > > > contain
>> >> >> > > > > > > composite structures in json. Could we flatten them out
>> >> >> directly in
>> >> >> > > > the
>> >> >> > > > > > > protocol definition as arrays/records?
>> >> >> > > > > > >
>> >> >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto
>> topic
>> >> >> > > creation?
>> >> >> > > > > This
>> >> >> > > > > > > will be a bit weird now that we have a separate topic
>> >> >> creation api.
>> >> >> > > > > Have
>> >> >> > > > > > > you thought about how the new createTopicRequest and
>> >> >> > > > > TopicMetadataRequest
>> >> >> > > > > > > v1 will be used in the producer/consumer client, in
>> addition
>> >> >> to
>> >> >> > > admin
>> >> >> > > > > > > tools? For example, ideally, we don't want
>> >> >> TopicMetadataRequest
>> >> >> > > from
>> >> >> > > > > the
>> >> >> > > > > > > consumer to trigger auto topic creation.
>> >> >> > > > > > >
>> >> >> > > > > > > 2. I think Jay meant getting rid of scala classes
>> >> >> > > > > > > like HeartbeatRequestAndHeader and
>> >> >> HeartbeatResponseAndHeader. We
>> >> >> > > did
>> >> >> > > > > > that
>> >> >> > > > > > > as a stop-gap thing when adding the new requests for the
>> >> >> consumers.
>> >> >> > > > > > > However, the long term plan is to get rid of all those
>> and
>> >> >> just
>> >> >> > > reuse
>> >> >> > > > > the
>> >> >> > > > > > > java request/response in the client. Since this KIP
>> proposes
>> >> >> to
>> >> >> > > add a
>> >> >> > > > > > > significant number of new requests, perhaps we should
>> bite
>> >> the
>> >> >> > > bullet
>> >> >> > > > > to
>> >> >> > > > > > > clean up the existing scala requests first before adding
>> new
>> >> >> ones?
>> >> >> > > > > > >
>> >> >> > > > > > > Thanks,
>> >> >> > > > > > >
>> >> >> > > > > > > Jun
>> >> >> > > > > > >
>> >> >> > > > > > >
>> >> >> > > > > > >
>> >> >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
>> >> >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> >> >> > > > > > >
>> >> >> > > > > > > > Hi,
>> >> >> > > > > > > >
>> >> >> > > > > > > > As said above - I list again all comments from this
>> thread
>> >> >> so we
>> >> >> > > > > > > > can see what's left and finalize all pending issues.
>> >> >> > > > > > > >
>> >> >> > > > > > > > Comments from Jay:
>> >> >> > > > > > > > 1. This is much needed functionality, but there are a
>> lot
>> >> >> of the
>> >> >> > > so
>> >> >> > > > > > let's
>> >> >> > > > > > > > really think these protocols through. We really want to
>> >> end
>> >> >> up
>> >> >> > > > with a
>> >> >> > > > > > set
>> >> >> > > > > > > > of well thought-out, orthoganol apis. For this reason I
>> >> >> think it
>> >> >> > > is
>> >> >> > > > > > > really
>> >> >> > > > > > > > important to think through the end state even if that
>> >> >> includes
>> >> >> > > APIs
>> >> >> > > > > we
>> >> >> > > > > > > > won't implement in the first phase.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: Definitely behind this. Would appreciate if there
>> are
>> >> >> concrete
>> >> >> > > > > > > comments
>> >> >> > > > > > > > how this can be improved.
>> >> >> > > > > > > >
>> >> >> > > > > > > > 2. Let's please please please wait until we have
>> switched
>> >> >> the
>> >> >> > > > server
>> >> >> > > > > > over
>> >> >> > > > > > > > to the new java protocol definitions. If we add upteen
>> >> more
>> >> >> ad
>> >> >> > > hoc
>> >> >> > > > > > scala
>> >> >> > > > > > > > objects that is just generating more work for the
>> >> >> conversion we
>> >> >> > > > know
>> >> >> > > > > we
>> >> >> > > > > > > > have to do.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: Fixed in the latest patch - removed scala protocol
>> >> >> classes.
>> >> >> > > > > > > >
>> >> >> > > > > > > > 3. This proposal introduces a new type of optional
>> >> >> parameter.
>> >> >> > > This
>> >> >> > > > is
>> >> >> > > > > > > > inconsistent with everything else in the protocol
>> where we
>> >> >> use -1
>> >> >> > > > or
>> >> >> > > > > > some
>> >> >> > > > > > > > other marker value. You could argue either way but
>> let's
>> >> >> stick
>> >> >> > > with
>> >> >> > > > > > that
>> >> >> > > > > > > > for consistency. For clients that implemented the
>> protocol
>> >> >> in a
>> >> >> > > > > better
>> >> >> > > > > > > way
>> >> >> > > > > > > > than our scala code these basic primitives are hard to
>> >> >> change.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf type and
>> >> >> changed
>> >> >> > > > > > protocol
>> >> >> > > > > > > > accordingly.
>> >> >> > > > > > > >
>> >> >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
>> >> >> TopicMetadataRequest
>> >> >> > > > > which
>> >> >> > > > > > > has
>> >> >> > > > > > > > brokers, topics, and partitions. I think we should
>> rename
>> >> >> that
>> >> >> > > > > request
>> >> >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest) and
>> >> >> include the
>> >> >> > > id
>> >> >> > > > > of
>> >> >> > > > > > > the
>> >> >> > > > > > > > controller. Or are there other things we could add
>> here?
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: I agree. Updated the KIP. Let's extends
>> TopicMetadata
>> >> to
>> >> >> > > > version 2
>> >> >> > > > > > and
>> >> >> > > > > > > > include controller.
>> >> >> > > > > > > >
>> >> >> > > > > > > > 5. We have a tendency to try to make a lot of requests
>> >> that
>> >> >> can
>> >> >> > > > only
>> >> >> > > > > go
>> >> >> > > > > > > to
>> >> >> > > > > > > > particular nodes. This adds a lot of burden for client
>> >> >> > > > > implementations
>> >> >> > > > > > > (it
>> >> >> > > > > > > > sounds easy but each discovery can fail in many parts
>> so
>> >> it
>> >> >> ends
>> >> >> > > up
>> >> >> > > > > > > being a
>> >> >> > > > > > > > full state machine to do right). I think we should
>> >> consider
>> >> >> > > making
>> >> >> > > > > > admin
>> >> >> > > > > > > > commands and ideally as many of the other apis as
>> possible
>> >> >> > > > available
>> >> >> > > > > on
>> >> >> > > > > > > all
>> >> >> > > > > > > > brokers and just redirect to the controller on the
>> broker
>> >> >> side.
>> >> >> > > > > Perhaps
>> >> >> > > > > > > > there would be a general way to encapsulate this
>> >> re-routing
>> >> >> > > > behavior.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: It's a very interesting idea, but seems there are
>> some
>> >> >> > > concerns
>> >> >> > > > > > about
>> >> >> > > > > > > > this
>> >> >> > > > > > > > feature (like performance considerations, how this will
>> >> >> > > complicate
>> >> >> > > > > > server
>> >> >> > > > > > > > etc).
>> >> >> > > > > > > > I believe this shouldn't be a blocker. If this feature
>> is
>> >> >> > > > implemented
>> >> >> > > > > > at
>> >> >> > > > > > > > some
>> >> >> > > > > > > > point it won't affect Admin changes - at least no
>> changes
>> >> to
>> >> >> > > public
>> >> >> > > > > API
>> >> >> > > > > > > > will be required.
>> >> >> > > > > > > >
>> >> >> > > > > > > > 6. We should probably normalize the key value pairs
>> used
>> >> for
>> >> >> > > > configs
>> >> >> > > > > > > rather
>> >> >> > > > > > > > than embedding a new formatting. So two strings rather
>> >> than
>> >> >> one
>> >> >> > > > with
>> >> >> > > > > an
>> >> >> > > > > > > > internal equals sign.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: Fixed in the latest patch - normalized configs and
>> >> >> changed
>> >> >> > > > > protocol
>> >> >> > > > > > > > accordingly.
>> >> >> > > > > > > >
>> >> >> > > > > > > > 7. Is the postcondition of these APIs that the command
>> has
>> >> >> begun
>> >> >> > > or
>> >> >> > > > > > that
>> >> >> > > > > > > > the command has been completed? It is a lot more
>> usable if
>> >> >> the
>> >> >> > > > > command
>> >> >> > > > > > > has
>> >> >> > > > > > > > been completed so you know that if you create a topic
>> and
>> >> >> then
>> >> >> > > > > publish
>> >> >> > > > > > to
>> >> >> > > > > > > > it you won't get an exception about there being no such
>> >> >> topic.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: For long running requests (like reassign
>> partitions) -
>> >> >> the
>> >> >> > > post
>> >> >> > > > > > > > condition is
>> >> >> > > > > > > > command has begun - so we don't block the client. In
>> case
>> >> >> of your
>> >> >> > > > > > > example -
>> >> >> > > > > > > > topic commands, this will be refactored and topic
>> commands
>> >> >> will
>> >> >> > > be
>> >> >> > > > > > > executed
>> >> >> > > > > > > > immediately, since the Controller will serve Admin
>> >> requests
>> >> >> > > > > > > > (follow-up ticket KAFKA-1777).
>> >> >> > > > > > > >
>> >> >> > > > > > > > 8. Describe topic and list topics duplicate a lot of
>> stuff
>> >> >> in the
>> >> >> > > > > > > metadata
>> >> >> > > > > > > > request. Is there a reason to give back topics marked
>> for
>> >> >> > > > deletion? I
>> >> >> > > > > > > feel
>> >> >> > > > > > > > like if we just make the post-condition of the delete
>> >> >> command be
>> >> >> > > > that
>> >> >> > > > > > the
>> >> >> > > > > > > > topic is deleted that will get rid of the need for this
>> >> >> right?
>> >> >> > > And
>> >> >> > > > it
>> >> >> > > > > > > will
>> >> >> > > > > > > > be much more intuitive.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: Fixed in the latest patch - removed topics marked
>> for
>> >> >> deletion
>> >> >> > > > in
>> >> >> > > > > > > > ListTopicsRequest.
>> >> >> > > > > > > >
>> >> >> > > > > > > > 9. Should we consider batching these requests? We have
>> >> >> generally
>> >> >> > > > > tried
>> >> >> > > > > > to
>> >> >> > > > > > > > allow multiple operations to be batched. My suspicion
>> is
>> >> >> that
>> >> >> > > > without
>> >> >> > > > > > > this
>> >> >> > > > > > > > we will get a lot of code that does something like
>> >> >> > > > > > > >    for(topic: adminClient.listTopics())
>> >> >> > > > > > > >       adminClient.describeTopic(topic)
>> >> >> > > > > > > > this code will work great when you test on 5 topics but
>> >> not
>> >> >> do as
>> >> >> > > > > well
>> >> >> > > > > > if
>> >> >> > > > > > > > you have 50k.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: Updated the KIP - please check "Topic Admin Schema"
>> >> >> section.
>> >> >> > > > > > > >
>> >> >> > > > > > > > 10. I think we should also discuss how we want to
>> expose a
>> >> >> > > > > programmatic
>> >> >> > > > > > > JVM
>> >> >> > > > > > > > client api for these operations. Currently people rely
>> on
>> >> >> > > > AdminUtils
>> >> >> > > > > > > which
>> >> >> > > > > > > > is totally sketchy. I think we probably need another
>> >> client
>> >> >> under
>> >> >> > > > > > > clients/
>> >> >> > > > > > > > that exposes administrative functionality. We will need
>> >> >> this just
>> >> >> > > > to
>> >> >> > > > > > > > properly test the new apis, I suspect. We should figure
>> >> out
>> >> >> that
>> >> >> > > > API.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: Updated the KIP - please check "Admin Client"
>> section
>> >> >> with an
>> >> >> > > > > > initial
>> >> >> > > > > > > > API proposal.
>> >> >> > > > > > > >
>> >> >> > > > > > > > 11. The other information that would be really useful
>> to
>> >> get
>> >> >> > > would
>> >> >> > > > be
>> >> >> > > > > > > > information about partitions--how much data is in the
>> >> >> partition,
>> >> >> > > > what
>> >> >> > > > > > are
>> >> >> > > > > > > > the segment offsets, what is the log-end offset (i.e.
>> last
>> >> >> > > offset),
>> >> >> > > > > > what
>> >> >> > > > > > > is
>> >> >> > > > > > > > the compaction point, etc. I think that done right this
>> >> >> would be
>> >> >> > > > the
>> >> >> > > > > > > > successor to the very awkward OffsetRequest we have
>> today.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest
>> >> >> patch. I
>> >> >> > > > > believe
>> >> >> > > > > > > > this should
>> >> >> > > > > > > > be resolved in a separate KIP / jira ticket.
>> >> >> > > > > > > >
>> >> >> > > > > > > > 12. Generally we can do good error handling without
>> >> needing
>> >> >> > > custom
>> >> >> > > > > > > > server-side
>> >> >> > > > > > > > messages. I.e. generally the client has the context to
>> >> know
>> >> >> that
>> >> >> > > if
>> >> >> > > > > it
>> >> >> > > > > > > got
>> >> >> > > > > > > > an error that the topic doesn't exist to say "Topic X
>> >> >> doesn't
>> >> >> > > > exist"
>> >> >> > > > > > > rather
>> >> >> > > > > > > > than "error code 14" (or whatever). Maybe there are
>> >> specific
>> >> >> > > cases
>> >> >> > > > > > where
>> >> >> > > > > > > > this is hard? If we want to add server-side error
>> messages
>> >> >> we
>> >> >> > > > really
>> >> >> > > > > do
>> >> >> > > > > > > > need to do this in a consistent way across the
>> protocol.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: Updated the KIP - please check "Protocol Errors"
>> >> >> section. I
>> >> >> > > > added
>> >> >> > > > > > the
>> >> >> > > > > > > > comprehensive, fine-grained list of error codes.
>> >> >> > > > > > > >
>> >> >> > > > > > > > Comments from Guozhang:
>> >> >> > > > > > > > 13. Describe topic request: it would be great to go
>> beyond
>> >> >> just
>> >> >> > > > > > batching
>> >> >> > > > > > > on
>> >> >> > > > > > > > topic name regex for this request. For example, a very
>> >> >> common use
>> >> >> > > > > case
>> >> >> > > > > > of
>> >> >> > > > > > > > the topic command is to list all topics whose config
>> A's
>> >> >> value is
>> >> >> > > > B.
>> >> >> > > > > > With
>> >> >> > > > > > > > topic name regex then we have to first retrieve __all__
>> >> >> topics's
>> >> >> > > > > > > > description info and then filter at the client end,
>> which
>> >> >> will
>> >> >> > > be a
>> >> >> > > > > > huge
>> >> >> > > > > > > > burden on ZK.
>> >> >> > > > > > > > AND
>> >> >> > > > > > > > 14. Config K-Vs in create topic: this is related to the
>> >> >> previous
>> >> >> > > > > point;
>> >> >> > > > > > > > maybe we can add another metadata K-V or just a
>> metadata
>> >> >> string
>> >> >> > > > along
>> >> >> > > > > > > side
>> >> >> > > > > > > > with config K-V in create topic like we did for offset
>> >> >> commit
>> >> >> > > > > request.
>> >> >> > > > > > > This
>> >> >> > > > > > > > field can be quite useful in storing information like
>> >> >> "owner" of
>> >> >> > > > the
>> >> >> > > > > > > topic
>> >> >> > > > > > > > who issue the create command, etc, which is quite
>> >> important
>> >> >> for a
>> >> >> > > > > > > > multi-tenant setting. Then in the describe topic
>> request
>> >> we
>> >> >> can
>> >> >> > > > also
>> >> >> > > > > > > batch
>> >> >> > > > > > > > on regex of the metadata field.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: As discussed it is very interesting but can be
>> >> >> implemented
>> >> >> > > later
>> >> >> > > > > > after
>> >> >> > > > > > > > we have some basic functionality there.
>> >> >> > > > > > > >
>> >> >> > > > > > > > 15. Today all the admin operations are async in the
>> sense
>> >> >> that
>> >> >> > > > > command
>> >> >> > > > > > > will
>> >> >> > > > > > > > return once it is written in ZK, and that is why we
>> need
>> >> >> extra
>> >> >> > > > > > > verification
>> >> >> > > > > > > > like testUtil.waitForTopicCreated() / verify partition
>> >> >> > > reassignment
>> >> >> > > > > > > > request, etc. With admin requests we could add a flag
>> to
>> >> >> enable /
>> >> >> > > > > > disable
>> >> >> > > > > > > > synchronous requests; when it is turned on, the
>> response
>> >> >> will not
>> >> >> > > > > > return
>> >> >> > > > > > > > until the request has been completed. And for async
>> >> >> requests we
>> >> >> > > can
>> >> >> > > > > > add a
>> >> >> > > > > > > > "token" field in the response, and then only need a
>> >> general
>> >> >> > > "admin
>> >> >> > > > > > > > verification request" with the given token to check if
>> the
>> >> >> async
>> >> >> > > > > > request
>> >> >> > > > > > > > has been completed.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: I see your point. My idea was to provide specific
>> >> >> > > > Verify...Request
>> >> >> > > > > > per
>> >> >> > > > > > > > each
>> >> >> > > > > > > > long running request, where needed. We can do it the
>> way
>> >> you
>> >> >> > > > suggest.
>> >> >> > > > > > The
>> >> >> > > > > > > > only
>> >> >> > > > > > > > concern is that introducing a token we again will make
>> >> >> schema
>> >> >> > > > > > "dynamic".
>> >> >> > > > > > > We
>> >> >> > > > > > > > wanted
>> >> >> > > > > > > > to do similar thing introducing single AdminRequest for
>> >> all
>> >> >> topic
>> >> >> > > > > > > commands
>> >> >> > > > > > > > but rejected
>> >> >> > > > > > > > this idea because we wanted to have schema defined. So
>> >> this
>> >> >> is
>> >> >> > > > more a
>> >> >> > > > > > > > choice between:
>> >> >> > > > > > > > a) have fixed schema but introduce each time new
>> >> >> Verify...Request
>> >> >> > > > for
>> >> >> > > > > > > > long-running requests
>> >> >> > > > > > > > b) use one request for verification but generalize it
>> with
>> >> >> token
>> >> >> > > > > > > > I'm fine with whatever decision community come to. Just
>> >> let
>> >> >> me
>> >> >> > > know
>> >> >> > > > > > your
>> >> >> > > > > > > > thoughts.
>> >> >> > > > > > > >
>> >> >> > > > > > > > Comment from Gwen:
>> >> >> > > > > > > > 16. Specifically for ownership, I think the plan is to
>> add
>> >> >> ACL
>> >> >> > > (it
>> >> >> > > > > > sounds
>> >> >> > > > > > > > like you are describing ACL) via an external system
>> >> (Argus,
>> >> >> > > > Sentry).
>> >> >> > > > > > > > I remember KIP-11 described this, but I can't find the
>> KIP
>> >> >> any
>> >> >> > > > > longer.
>> >> >> > > > > > > >
>> >> >> > > > > > > > A: Okay, no problem. Not sure though how we are going
>> to
>> >> >> handle
>> >> >> > > it.
>> >> >> > > > > > Wait
>> >> >> > > > > > > > which KIP
>> >> >> > > > > > > > will be committed first and include changes to
>> >> >> TopicMetadata from
>> >> >> > > > the
>> >> >> > > > > > > later
>> >> >> > > > > > > > one?
>> >> >> > > > > > > > Anyway, I added this note to "Open Questions" section
>> so
>> >> we
>> >> >> don't
>> >> >> > > > > miss
>> >> >> > > > > > > this
>> >> >> > > > > > > > piece.
>> >> >> > > > > > > >
>> >> >> > > > > > > > Thanks,
>> >> >> > > > > > > > Andrii Biletskyi
>> >> >> > > > > > > >
>> >> >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
>> >> >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> >> >> > > > > > > >
>> >> >> > > > > > > > > Hi all,
>> >> >> > > > > > > > >
>> >> >> > > > > > > > > Today I uploaded the patch that covers some of the
>> >> >> discussed
>> >> >> > > and
>> >> >> > > > > > agreed
>> >> >> > > > > > > > > items:
>> >> >> > > > > > > > > - removed MaybeOf optional type
>> >> >> > > > > > > > > - switched to java protocol definitions
>> >> >> > > > > > > > > - simplified messages (normalized configs, removed
>> topic
>> >> >> marked
>> >> >> > > > for
>> >> >> > > > > > > > > deletion)
>> >> >> > > > > > > > >
>> >> >> > > > > > > > > I also updated the KIP-4 with respective changes and
>> >> >> wrote down
>> >> >> > > > my
>> >> >> > > > > > > > > proposal for
>> >> >> > > > > > > > > pending items:
>> >> >> > > > > > > > > - Batch Admin Operations -> updated Wire Protocol
>> schema
>> >> >> > > proposal
>> >> >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
>> >> >> > > > TopicMetadataRequest
>> >> >> > > > > > > > > - Admin Client -> updated my initial proposal to
>> reflect
>> >> >> > > batching
>> >> >> > > > > > > > > - Error codes -> proposed fine-grained error code
>> >> instead
>> >> >> of
>> >> >> > > > > > > > > AdminRequestFailed
>> >> >> > > > > > > > >
>> >> >> > > > > > > > > I will also send a separate email to cover all
>> comments
>> >> >> from
>> >> >> > > this
>> >> >> > > > > > > thread.
>> >> >> > > > > > > > >
>> >> >> > > > > > > > > Thanks,
>> >> >> > > > > > > > > Andrii Biletskyi
>> >> >> > > > > > > > >
>> >> >> > > > > > > > >
>> >> >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
>> >> >> > > > > gshapira@cloudera.com
>> >> >> > > > > > >
>> >> >> > > > > > > > > wrote:
>> >> >> > > > > > > > >
>> >> >> > > > > > > > >> Found KIP-11 (
>> >> >> > > > > > > > >>
>> >> >> > > > > > > >
>> >> >> > > > > > >
>> >> >> > > > > >
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >>
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
>> >> >> > > > > > > > >> )
>> >> >> > > > > > > > >> It actually specifies changes to the Metadata
>> protocol,
>> >> >> so
>> >> >> > > > making
>> >> >> > > > > > sure
>> >> >> > > > > > > > >> both KIPs are consistent in this regard will be
>> good.
>> >> >> > > > > > > > >>
>> >> >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
>> >> >> > > > > > gshapira@cloudera.com
>> >> >> > > > > > > >
>> >> >> > > > > > > > >> wrote:
>> >> >> > > > > > > > >> > Specifically for ownership, I think the plan is to
>> >> add
>> >> >> ACL
>> >> >> > > (it
>> >> >> > > > > > > sounds
>> >> >> > > > > > > > >> > like you are describing ACL) via an external
>> system
>> >> >> (Argus,
>> >> >> > > > > > Sentry).
>> >> >> > > > > > > > >> > I remember KIP-11 described this, but I can't find
>> >> the
>> >> >> KIP
>> >> >> > > any
>> >> >> > > > > > > longer.
>> >> >> > > > > > > > >> >
>> >> >> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting
>> >> >> information
>> >> >> > > that
>> >> >> > > > > > > already
>> >> >> > > > > > > > >> > exists from Kafka brokers, not on adding
>> information
>> >> >> that
>> >> >> > > > > perhaps
>> >> >> > > > > > > > >> > should exist but doesn't yet?
>> >> >> > > > > > > > >> >
>> >> >> > > > > > > > >> > Gwen
>> >> >> > > > > > > > >> >
>> >> >> > > > > > > > >> >
>> >> >> > > > > > > > >> >
>> >> >> > > > > > > > >> >
>> >> >> > > > > > > > >> >
>> >> >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
>> >> >> > > > > > wangguoz@gmail.com>
>> >> >> > > > > > > > >> wrote:
>> >> >> > > > > > > > >> >> Folks,
>> >> >> > > > > > > > >> >>
>> >> >> > > > > > > > >> >> Just want to elaborate a bit more on the
>> >> create-topic
>> >> >> > > > metadata
>> >> >> > > > > > and
>> >> >> > > > > > > > >> batching
>> >> >> > > > > > > > >> >> describe-topic based on config / metadata in my
>> >> >> previous
>> >> >> > > > email
>> >> >> > > > > as
>> >> >> > > > > > > we
>> >> >> > > > > > > > >> work
>> >> >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to have
>> some
>> >> >> sort of
>> >> >> > > > > topic
>> >> >> > > > > > > > >> management
>> >> >> > > > > > > > >> >> mechanisms, which I think is quite important in a
>> >> >> > > > multi-tenant
>> >> >> > > > > /
>> >> >> > > > > > > > cloud
>> >> >> > > > > > > > >> >> architecture: today anyone can create topics in a
>> >> >> shared
>> >> >> > > > Kafka
>> >> >> > > > > > > > >> cluster, but
>> >> >> > > > > > > > >> >> there is no concept or "ownership" of topics that
>> >> are
>> >> >> > > created
>> >> >> > > > > by
>> >> >> > > > > > > > >> different
>> >> >> > > > > > > > >> >> users. For example, at LinkedIn we basically
>> >> >> distinguish
>> >> >> > > > topic
>> >> >> > > > > > > owners
>> >> >> > > > > > > > >> via
>> >> >> > > > > > > > >> >> some casual topic name prefix, which is a bit
>> >> awkward
>> >> >> and
>> >> >> > > > does
>> >> >> > > > > > not
>> >> >> > > > > > > > fly
>> >> >> > > > > > > > >> as
>> >> >> > > > > > > > >> >> we scale our customers. It would be great to use
>> >> >> > > > > describe-topics
>> >> >> > > > > > > such
>> >> >> > > > > > > > >> as:
>> >> >> > > > > > > > >> >>
>> >> >> > > > > > > > >> >> Describe all topics that is created by me.
>> >> >> > > > > > > > >> >>
>> >> >> > > > > > > > >> >> Describe all topics whose retention time is
>> >> overriden
>> >> >> to X.
>> >> >> > > > > > > > >> >>
>> >> >> > > > > > > > >> >> Describe all topics whose writable group include
>> >> user
>> >> >> Y
>> >> >> > > (this
>> >> >> > > > > is
>> >> >> > > > > > > > >> related to
>> >> >> > > > > > > > >> >> authorization), etc..
>> >> >> > > > > > > > >> >>
>> >> >> > > > > > > > >> >> One possible way to achieve this is to add a
>> >> metadata
>> >> >> file
>> >> >> > > in
>> >> >> > > > > the
>> >> >> > > > > > > > >> >> create-topic request, whose value will also be
>> >> >> written ZK
>> >> >> > > as
>> >> >> > > > we
>> >> >> > > > > > > > create
>> >> >> > > > > > > > >> the
>> >> >> > > > > > > > >> >> topic; then describe-topics can choose to batch
>> >> topics
>> >> >> > > based
>> >> >> > > > on
>> >> >> > > > > > 1)
>> >> >> > > > > > > > name
>> >> >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata regex,
>> >> etc.
>> >> >> > > > > > > > >> >>
>> >> >> > > > > > > > >> >> Thoughts?
>> >> >> > > > > > > > >> >>
>> >> >> > > > > > > > >> >> Guozhang
>> >> >> > > > > > > > >> >>
>> >> >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
>> >> >> > > > > > wangguoz@gmail.com>
>> >> >> > > > > > > > >> wrote:
>> >> >> > > > > > > > >> >>
>> >> >> > > > > > > > >> >>> Thanks for the updated wiki. A few comments
>> below:
>> >> >> > > > > > > > >> >>>
>> >> >> > > > > > > > >> >>> 1. Error description in response: I think if
>> some
>> >> >> > > errorCode
>> >> >> > > > > > could
>> >> >> > > > > > > > >> indicate
>> >> >> > > > > > > > >> >>> several different error cases then we should
>> really
>> >> >> change
>> >> >> > > > it
>> >> >> > > > > to
>> >> >> > > > > > > > >> multiple
>> >> >> > > > > > > > >> >>> codes. In general the errorCode itself would be
>> >> >> precise
>> >> >> > > and
>> >> >> > > > > > > > >> sufficient for
>> >> >> > > > > > > > >> >>> describing the server side errors.
>> >> >> > > > > > > > >> >>>
>> >> >> > > > > > > > >> >>> 2. Describe topic request: it would be great to
>> go
>> >> >> beyond
>> >> >> > > > just
>> >> >> > > > > > > > >> batching on
>> >> >> > > > > > > > >> >>> topic name regex for this request. For example,
>> a
>> >> >> very
>> >> >> > > > common
>> >> >> > > > > > use
>> >> >> > > > > > > > >> case of
>> >> >> > > > > > > > >> >>> the topic command is to list all topics whose
>> >> config
>> >> >> A's
>> >> >> > > > value
>> >> >> > > > > > is
>> >> >> > > > > > > B.
>> >> >> > > > > > > > >> With
>> >> >> > > > > > > > >> >>> topic name regex then we have to first retrieve
>> >> >> __all__
>> >> >> > > > > topics's
>> >> >> > > > > > > > >> >>> description info and then filter at the client
>> end,
>> >> >> which
>> >> >> > > > will
>> >> >> > > > > > be
>> >> >> > > > > > > a
>> >> >> > > > > > > > >> huge
>> >> >> > > > > > > > >> >>> burden on ZK.
>> >> >> > > > > > > > >> >>>
>> >> >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is related
>> to
>> >> >> the
>> >> >> > > > > previous
>> >> >> > > > > > > > point;
>> >> >> > > > > > > > >> >>> maybe we can add another metadata K-V or just a
>> >> >> metadata
>> >> >> > > > > string
>> >> >> > > > > > > > along
>> >> >> > > > > > > > >> side
>> >> >> > > > > > > > >> >>> with config K-V in create topic like we did for
>> >> >> offset
>> >> >> > > > commit
>> >> >> > > > > > > > >> request. This
>> >> >> > > > > > > > >> >>> field can be quite useful in storing information
>> >> like
>> >> >> > > > "owner"
>> >> >> > > > > of
>> >> >> > > > > > > the
>> >> >> > > > > > > > >> topic
>> >> >> > > > > > > > >> >>> who issue the create command, etc, which is
>> quite
>> >> >> > > important
>> >> >> > > > > for
>> >> >> > > > > > a
>> >> >> > > > > > > > >> >>> multi-tenant setting. Then in the describe topic
>> >> >> request
>> >> >> > > we
>> >> >> > > > > can
>> >> >> > > > > > > also
>> >> >> > > > > > > > >> batch
>> >> >> > > > > > > > >> >>> on regex of the metadata field.
>> >> >> > > > > > > > >> >>>
>> >> >> > > > > > > > >> >>> 4. Today all the admin operations are async in
>> the
>> >> >> sense
>> >> >> > > > that
>> >> >> > > > > > > > command
>> >> >> > > > > > > > >> will
>> >> >> > > > > > > > >> >>> return once it is written in ZK, and that is
>> why we
>> >> >> need
>> >> >> > > > extra
>> >> >> > > > > > > > >> verification
>> >> >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify
>> >> >> partition
>> >> >> > > > > > > reassignment
>> >> >> > > > > > > > >> >>> request, etc. With admin requests we could add a
>> >> >> flag to
>> >> >> > > > > enable
>> >> >> > > > > > /
>> >> >> > > > > > > > >> disable
>> >> >> > > > > > > > >> >>> synchronous requests; when it is turned on, the
>> >> >> response
>> >> >> > > > will
>> >> >> > > > > > not
>> >> >> > > > > > > > >> return
>> >> >> > > > > > > > >> >>> until the request has been completed. And for
>> async
>> >> >> > > requests
>> >> >> > > > > we
>> >> >> > > > > > > can
>> >> >> > > > > > > > >> add a
>> >> >> > > > > > > > >> >>> "token" field in the response, and then only
>> need a
>> >> >> > > general
>> >> >> > > > > > "admin
>> >> >> > > > > > > > >> >>> verification request" with the given token to
>> check
>> >> >> if the
>> >> >> > > > > async
>> >> >> > > > > > > > >> request
>> >> >> > > > > > > > >> >>> has been completed.
>> >> >> > > > > > > > >> >>>
>> >> >> > > > > > > > >> >>> 5. +1 for extending Metadata request to include
>> >> >> > > controller /
>> >> >> > > > > > > > >> coordinator
>> >> >> > > > > > > > >> >>> information, and then we can remove the
>> >> >> ConsumerMetadata /
>> >> >> > > > > > > > >> ClusterMetadata
>> >> >> > > > > > > > >> >>> requests.
>> >> >> > > > > > > > >> >>>
>> >> >> > > > > > > > >> >>> Guozhang
>> >> >> > > > > > > > >> >>>
>> >> >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
>> >> >> > > > > > jjkoshy.w@gmail.com>
>> >> >> > > > > > > > >> wrote:
>> >> >> > > > > > > > >> >>>
>> >> >> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't
>> think I
>> >> >> will be
>> >> >> > > > > able
>> >> >> > > > > > to
>> >> >> > > > > > > > >> make
>> >> >> > > > > > > > >> >>>> it today, so if notes can be sent out afterward
>> >> that
>> >> >> > > would
>> >> >> > > > be
>> >> >> > > > > > > > great.
>> >> >> > > > > > > > >> >>>>
>> >> >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen
>> >> >> Shapira
>> >> >> > > > wrote:
>> >> >> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking
>> forward
>> >> >> to
>> >> >> > > > > chatting
>> >> >> > > > > > > with
>> >> >> > > > > > > > >> >>>> everyone :)
>> >> >> > > > > > > > >> >>>> >
>> >> >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
>> >> >> > > > > > > joe.stein@stealth.ly>
>> >> >> > > > > > > > >> wrote:
>> >> >> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout
>> invite
>> >> to
>> >> >> all
>> >> >> > > > pmc,
>> >> >> > > > > > > > >> committers
>> >> >> > > > > > > > >> >>>> and
>> >> >> > > > > > > > >> >>>> > > everyone I found working on a KIP. If I
>> missed
>> >> >> anyone
>> >> >> > > > in
>> >> >> > > > > > the
>> >> >> > > > > > > > >> invite
>> >> >> > > > > > > > >> >>>> please
>> >> >> > > > > > > > >> >>>> > > let me know and can update it, np.
>> >> >> > > > > > > > >> >>>> > >
>> >> >> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm
>> Eastern
>> >> >> Time.
>> >> >> > > > Maybe
>> >> >> > > > > > we
>> >> >> > > > > > > > can
>> >> >> > > > > > > > >> get
>> >> >> > > > > > > > >> >>>> INFRA
>> >> >> > > > > > > > >> >>>> > > help to make a google account so we can
>> manage
>> >> >> > > better?
>> >> >> > > > > > > > >> >>>> > >
>> >> >> > > > > > > > >> >>>> > > To discuss
>> >> >> > > > > > > > >> >>>> > >
>> >> >> > > > > > > > >> >>>>
>> >> >> > > > > > > > >>
>> >> >> > > > > > > >
>> >> >> > > > > > >
>> >> >> > > > > >
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >>
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>> >> >> > > > > > > > >> >>>> > > in progress and related JIRA that are
>> >> >> interdependent
>> >> >> > > > and
>> >> >> > > > > > > common
>> >> >> > > > > > > > >> work.
>> >> >> > > > > > > > >> >>>> > >
>> >> >> > > > > > > > >> >>>> > > ~ Joe Stein
>> >> >> > > > > > > > >> >>>> > >
>> >> >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps
>> <
>> >> >> > > > > > > > jay.kreps@gmail.com>
>> >> >> > > > > > > > >> >>>> wrote:
>> >> >> > > > > > > > >> >>>> > >
>> >> >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that will
>> also
>> >> >> record
>> >> >> > > > and
>> >> >> > > > > > make
>> >> >> > > > > > > > the
>> >> >> > > > > > > > >> >>>> sessions
>> >> >> > > > > > > > >> >>>> > >> available on youtube.
>> >> >> > > > > > > > >> >>>> > >>
>> >> >> > > > > > > > >> >>>> > >> -Jay
>> >> >> > > > > > > > >> >>>> > >>
>> >> >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff
>> >> Holoman
>> >> >> <
>> >> >> > > > > > > > >> >>>> jholoman@cloudera.com>
>> >> >> > > > > > > > >> >>>> > >> wrote:
>> >> >> > > > > > > > >> >>>> > >>
>> >> >> > > > > > > > >> >>>> > >> > Jay / Joe
>> >> >> > > > > > > > >> >>>> > >> >
>> >> >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for this
>> >> >> purpose.
>> >> >> > > We
>> >> >> > > > > > could
>> >> >> > > > > > > > >> record
>> >> >> > > > > > > > >> >>>> the
>> >> >> > > > > > > > >> >>>> > >> > sessions if there is interest and
>> publish
>> >> >> them
>> >> >> > > out.
>> >> >> > > > > > > > >> >>>> > >> >
>> >> >> > > > > > > > >> >>>> > >> > Thanks
>> >> >> > > > > > > > >> >>>> > >> >
>> >> >> > > > > > > > >> >>>> > >> > Jeff
>> >> >> > > > > > > > >> >>>> > >> >
>> >> >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay
>> >> Kreps <
>> >> >> > > > > > > > >> jay.kreps@gmail.com>
>> >> >> > > > > > > > >> >>>> wrote:
>> >> >> > > > > > > > >> >>>> > >> >
>> >> >> > > > > > > > >> >>>> > >> > > Let's try to get the technical
>> hang-ups
>> >> >> sorted
>> >> >> > > > out,
>> >> >> > > > > > > > though.
>> >> >> > > > > > > > >> I
>> >> >> > > > > > > > >> >>>> really
>> >> >> > > > > > > > >> >>>> > >> > think
>> >> >> > > > > > > > >> >>>> > >> > > there is some benefit to live
>> discussion
>> >> vs
>> >> >> > > > > writing. I
>> >> >> > > > > > > am
>> >> >> > > > > > > > >> >>>> hopeful that
>> >> >> > > > > > > > >> >>>> > >> if
>> >> >> > > > > > > > >> >>>> > >> > > we post instructions and give
>> ourselves a
>> >> >> few
>> >> >> > > > > attempts
>> >> >> > > > > > > we
>> >> >> > > > > > > > >> can
>> >> >> > > > > > > > >> >>>> get it
>> >> >> > > > > > > > >> >>>> > >> > > working.
>> >> >> > > > > > > > >> >>>> > >> > >
>> >> >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work for
>> >> >> me...any
>> >> >> > > > > > objections?
>> >> >> > > > > > > > >> >>>> > >> > >
>> >> >> > > > > > > > >> >>>> > >> > > -Jay
>> >> >> > > > > > > > >> >>>> > >> > >
>> >> >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe
>> >> Stein
>> >> >> <
>> >> >> > > > > > > > >> joe.stein@stealth.ly
>> >> >> > > > > > > > >> >>>> >
>> >> >> > > > > > > > >> >>>> > >> wrote:
>> >> >> > > > > > > > >> >>>> > >> > >
>> >> >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe like
>> every
>> >> >> > > Tuesday ~
>> >> >> > > > > 1pm
>> >> >> > > > > > > ET
>> >> >> > > > > > > > /
>> >> >> > > > > > > > >> 10am
>> >> >> > > > > > > > >> >>>> PT
>> >> >> > > > > > > > >> >>>> > >> ????
>> >> >> > > > > > > > >> >>>> > >> > > >
>> >> >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but
>> there
>> >> is
>> >> >> > > always
>> >> >> > > > > some
>> >> >> > > > > > > > >> issue or
>> >> >> > > > > > > > >> >>>> > >> whatever
>> >> >> > > > > > > > >> >>>> > >> > so
>> >> >> > > > > > > > >> >>>> > >> > > > we know the apache irc channel
>> works.
>> >> We
>> >> >> can
>> >> >> > > > start
>> >> >> > > > > > > there
>> >> >> > > > > > > > >> and
>> >> >> > > > > > > > >> >>>> see how
>> >> >> > > > > > > > >> >>>> > >> it
>> >> >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too
>> and
>> >> >> > > associate
>> >> >> > > > to
>> >> >> > > > > > > > >> tickets if
>> >> >> > > > > > > > >> >>>> need be
>> >> >> > > > > > > > >> >>>> > >> > > makes
>> >> >> > > > > > > > >> >>>> > >> > > > it helpful for things.
>> >> >> > > > > > > > >> >>>> > >> > > >
>> >> >> > > > > > > > >> >>>> > >> > > > ~ Joestein
>> >> >> > > > > > > > >> >>>> > >> > > >
>> >> >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM,
>> Jay
>> >> >> Kreps <
>> >> >> > > > > > > > >> >>>> jay.kreps@gmail.com>
>> >> >> > > > > > > > >> >>>> > >> > wrote:
>> >> >> > > > > > > > >> >>>> > >> > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a Google
>> >> >> Hangout to
>> >> >> > > > chat
>> >> >> > > > > > > about
>> >> >> > > > > > > > >> this.
>> >> >> > > > > > > > >> >>>> What
>> >> >> > > > > > > > >> >>>> > >> > about
>> >> >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
>> >> further...I
>> >> >> > > > actually
>> >> >> > > > > > > think
>> >> >> > > > > > > > it
>> >> >> > > > > > > > >> >>>> would be
>> >> >> > > > > > > > >> >>>> > >> > good
>> >> >> > > > > > > > >> >>>> > >> > > > for
>> >> >> > > > > > > > >> >>>> > >> > > > > everyone spending a reasonable
>> chunk
>> >> of
>> >> >> > > their
>> >> >> > > > > week
>> >> >> > > > > > > on
>> >> >> > > > > > > > >> Kafka
>> >> >> > > > > > > > >> >>>> stuff
>> >> >> > > > > > > > >> >>>> > >> to
>> >> >> > > > > > > > >> >>>> > >> > > > maybe
>> >> >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think we
>> could
>> >> >> use
>> >> >> > > time
>> >> >> > > > > to
>> >> >> > > > > > > talk
>> >> >> > > > > > > > >> >>>> through
>> >> >> > > > > > > > >> >>>> > >> design
>> >> >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of
>> >> code
>> >> >> > > > reviews,
>> >> >> > > > > > talk
>> >> >> > > > > > > > >> through
>> >> >> > > > > > > > >> >>>> any
>> >> >> > > > > > > > >> >>>> > >> > tricky
>> >> >> > > > > > > > >> >>>> > >> > > > > issues, etc.
>> >> >> > > > > > > > >> >>>> > >> > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > We can make it publicly available
>> so
>> >> >> that
>> >> >> > > any
>> >> >> > > > > one
>> >> >> > > > > > > can
>> >> >> > > > > > > > >> follow
>> >> >> > > > > > > > >> >>>> along
>> >> >> > > > > > > > >> >>>> > >> > who
>> >> >> > > > > > > > >> >>>> > >> > > > > likes.
>> >> >> > > > > > > > >> >>>> > >> > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this? If so
>> >> I'll
>> >> >> try
>> >> >> > > to
>> >> >> > > > > set
>> >> >> > > > > > it
>> >> >> > > > > > > > up
>> >> >> > > > > > > > >> >>>> starting
>> >> >> > > > > > > > >> >>>> > >> next
>> >> >> > > > > > > > >> >>>> > >> > > > week.
>> >> >> > > > > > > > >> >>>> > >> > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > -Jay
>> >> >> > > > > > > > >> >>>> > >> > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM,
>> >> Andrii
>> >> >> > > > > Biletskyi
>> >> >> > > > > > <
>> >> >> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly>
>> wrote:
>> >> >> > > > > > > > >> >>>> > >> > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > > Hi all,
>> >> >> > > > > > > > >> >>>> > >> > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed /
>> >> >> aligned
>> >> >> > > > > document
>> >> >> > > > > > > > >> structure.
>> >> >> > > > > > > > >> >>>> Also I
>> >> >> > > > > > > > >> >>>> > >> > > added
>> >> >> > > > > > > > >> >>>> > >> > > > > > some
>> >> >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
>> >> >> AdminClient so
>> >> >> > > we
>> >> >> > > > > have
>> >> >> > > > > > > > >> something
>> >> >> > > > > > > > >> >>>> to
>> >> >> > > > > > > > >> >>>> > >> start
>> >> >> > > > > > > > >> >>>> > >> > > > from
>> >> >> > > > > > > > >> >>>> > >> > > > > > while
>> >> >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
>> >> >> > > > > > > > >> >>>> > >> > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > >
>> >> >> > > > > > > > >> >>>> > >> > > >
>> >> >> > > > > > > > >> >>>> > >> > >
>> >> >> > > > > > > > >> >>>> > >> >
>> >> >> > > > > > > > >> >>>> > >>
>> >> >> > > > > > > > >> >>>>
>> >> >> > > > > > > > >>
>> >> >> > > > > > > >
>> >> >> > > > > > >
>> >> >> > > > > >
>> >> >> > > > >
>> >> >> > > >
>> >> >> > >
>> >> >>
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>> >> >> > > > > > > > >> >>>> > >> > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > > Thanks,
>> >> >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
>> >> >> > > > > > > > >> >>>> > >> > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM,
>> >> >> Andrii
>> >> >> > > > > > Biletskyi
>> >> >> > > > > > > <
>> >> >> > > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly>
>> >> wrote:
>> >> >> > > > > > > > >> >>>> > >> > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > > > Jay,
>> >> >> > > > > > > > >> >>>> > >> > > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you are
>> right,
>> >> >> in
>> >> >> > > most
>> >> >> > > > > > cases
>> >> >> > > > > > > > >> client
>> >> >> > > > > > > > >> >>>> will
>> >> >> > > > > > > > >> >>>> > >> have
>> >> >> > > > > > > > >> >>>> > >> > > > enough
>> >> >> > > > > > > > >> >>>> > >> > > > > > > context to show descriptive
>> error
>> >> >> > > message.
>> >> >> > > > > My
>> >> >> > > > > > > > >> concern is
>> >> >> > > > > > > > >> >>>> that
>> >> >> > > > > > > > >> >>>> > >> we
>> >> >> > > > > > > > >> >>>> > >> > > will
>> >> >> > > > > > > > >> >>>> > >> > > > > > have
>> >> >> > > > > > > > >> >>>> > >> > > > > > > to
>> >> >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error codes
>> for
>> >> >> each
>> >> >> > > > > possible
>> >> >> > > > > > > > >> error. Of
>> >> >> > > > > > > > >> >>>> course,
>> >> >> > > > > > > > >> >>>> > >> > we
>> >> >> > > > > > > > >> >>>> > >> > > > > could
>> >> >> > > > > > > > >> >>>> > >> > > > > > > reuse
>> >> >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
>> >> >> > > > > > > UknownTopicOrPartitionCode,
>> >> >> > > > > > > > >> but we
>> >> >> > > > > > > > >> >>>> will
>> >> >> > > > > > > > >> >>>> > >> > also
>> >> >> > > > > > > > >> >>>> > >> > > > need
>> >> >> > > > > > > > >> >>>> > >> > > > > > to
>> >> >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
>> >> >> TopicAlreadyExistsCode,
>> >> >> > > > > > > > >> >>>> TopicConfigInvalid (both
>> >> >> > > > > > > > >> >>>> > >> > for
>> >> >> > > > > > > > >> >>>> > >> > > > > topic
>> >> >> > > > > > > > >> >>>> > >> > > > > > > name and config, and probably
>> >> user
>> >> >> would
>> >> >> > > > > like
>> >> >> > > > > > to
>> >> >> > > > > > > > >> know
>> >> >> > > > > > > > >> >>>> what
>> >> >> > > > > > > > >> >>>> > >> > exactly
>> >> >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
>> >> >> > > > > > > InvalidReplicaAssignment,
>> >> >> > > > > > > > >> >>>> > >> InternalError
>> >> >> > > > > > > > >> >>>> > >> > > > (e.g.
>> >> >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
>> >> >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
>> >> TopicCommand,
>> >> >> we
>> >> >> > > will
>> >> >> > > > > > also
>> >> >> > > > > > > > >> need to
>> >> >> > > > > > > > >> >>>> add
>> >> >> > > > > > > > >> >>>> > >> > similar
>> >> >> > > > > > > > >> >>>> > >> > > > > stuff
>> >> >> > > > > > > > >> >>>> > >> > > > > > > for
>> >> >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
>> >> >> PreferredReplica. So
>> >> >> > > > > we'll
>> >> >> > > > > > > end
>> >> >> > > > > > > > >> up
>> >> >> > > > > > > > >> >>>> with a
>> >> >> > > > > > > > >> >>>> > >> > large
>> >> >> > > > > > > > >> >>>> > >> > > > list
>> >> >> > > > > > > > >> >>>> > >> > > > > > of
>> >> >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in
>> Admin
>> >> >> > > protocol.
>> >> >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree my
>> >> >> proposal is
>> >> >> > > > not
>> >> >> > > > > > > > >> consistent
>> >> >> > > > > > > > >> >>>> with
>> >> >> > > > > > > > >> >>>> > >> > other
>> >> >> > > > > > > > >> >>>> > >> > > > > cases.
>> >> >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better
>> solution
>> >> >> or
>> >> >> > > > > something
>> >> >> > > > > > > > >> >>>> in-between.
>> >> >> > > > > > > > >> >>>> > >> > > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it
>> is a
>> >> >> great
>> >> >> > > > idea.
>> >> >> > > > > > > This
>> >> >> > > > > > > > >> way we
>> >> >> > > > > > > > >> >>>> can
>> >> >> > > > > > > > >> >>>> > >> move
>> >> >> > > > > > > > >> >>>> > >> > > on
>> >> >> > > > > > > > >> >>>> > >> > > > > > > faster.
>> >> >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on
>> date/time
>> >> so
>> >> >> > > people
>> >> >> > > > > can
>> >> >> > > > > > > > join.
>> >> >> > > > > > > > >> >>>> Will work
>> >> >> > > > > > > > >> >>>> > >> > for
>> >> >> > > > > > > > >> >>>> > >> > > me
>> >> >> > > > > > > > >> >>>> > >> > > > > > this
>> >> >> > > > > > > > >> >>>> > >> > > > > > > and
>> >> >> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime if
>> >> agreed
>> >> >> in
>> >> >> > > > > advance.
>> >> >> > > > > > > > >> >>>> > >> > > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
>> >> >> > > > > > > > >> >>>> > >> > > > > > > Andrii
>> >> >> > > > > > > > >> >>>> > >> > > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09
>> PM,
>> >> >> Jay
>> >> >> > > > Kreps <
>> >> >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
>> >> >> > > > > > > > >> >>>> > >> > > > > wrote:
>> >> >> > > > > > > > >> >>>> > >> > > > > > >
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
>> >> >> > > > > > > > >> >>>> > >> > > > > > >>
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good
>> error
>> >> >> handling
>> >> >> > > > > > without
>> >> >> > > > > > > > >> needing
>> >> >> > > > > > > > >> >>>> custom
>> >> >> > > > > > > > >> >>>> > >> > > > > > server-side
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the
>> >> >> client has
>> >> >> > > > the
>> >> >> > > > > > > > >> context to
>> >> >> > > > > > > > >> >>>> know
>> >> >> > > > > > > > >> >>>> > >> that
>> >> >> > > > > > > > >> >>>> > >> > > if
>> >> >> > > > > > > > >> >>>> > >> > > > it
>> >> >> > > > > > > > >> >>>> > >> > > > > > got
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic
>> doesn't
>> >> >> exist
>> >> >> > > to
>> >> >> > > > > say
>> >> >> > > > > > > > >> "Topic X
>> >> >> > > > > > > > >> >>>> doesn't
>> >> >> > > > > > > > >> >>>> > >> > > exist"
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> rather
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or
>> >> >> whatever).
>> >> >> > > Maybe
>> >> >> > > > > > there
>> >> >> > > > > > > > are
>> >> >> > > > > > > > >> >>>> specific
>> >> >> > > > > > > > >> >>>> > >> > cases
>> >> >> > > > > > > > >> >>>> > >> > > > > where
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to
>> add
>> >> >> > > > server-side
>> >> >> > > > > > > error
>> >> >> > > > > > > > >> >>>> messages we
>> >> >> > > > > > > > >> >>>> > >> > > really
>> >> >> > > > > > > > >> >>>> > >> > > > > do
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a
>> consistent
>> >> >> way
>> >> >> > > > across
>> >> >> > > > > > the
>> >> >> > > > > > > > >> protocol.
>> >> >> > > > > > > > >> >>>> > >> > > > > > >>
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open
>> >> >> questions
>> >> >> > > > here
>> >> >> > > > > > > from
>> >> >> > > > > > > > my
>> >> >> > > > > > > > >> >>>> previous
>> >> >> > > > > > > > >> >>>> > >> > > list. I
>> >> >> > > > > > > > >> >>>> > >> > > > > > will
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few days
>> for
>> >> >> Strata
>> >> >> > > > > > though.
>> >> >> > > > > > > > >> Maybe
>> >> >> > > > > > > > >> >>>> we could
>> >> >> > > > > > > > >> >>>> > >> > do
>> >> >> > > > > > > > >> >>>> > >> > > a
>> >> >> > > > > > > > >> >>>> > >> > > > > > Google
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open
>> issues
>> >> >> some
>> >> >> > > time
>> >> >> > > > > > > towards
>> >> >> > > > > > > > >> the
>> >> >> > > > > > > > >> >>>> end of
>> >> >> > > > > > > > >> >>>> > >> > next
>> >> >> > > > > > > > >> >>>> > >> > > > week
>> >> >> > > > > > > > >> >>>> > >> > > > > > for
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in this
>> >> ticket?
>> >> >> I
>> >> >> > > have
>> >> >> > > > a
>> >> >> > > > > > > > feeling
>> >> >> > > > > > > > >> that
>> >> >> > > > > > > > >> >>>> might
>> >> >> > > > > > > > >> >>>> > >> > > > progress
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster than
>> >> >> email--I
>> >> >> > > > think
>> >> >> > > > > we
>> >> >> > > > > > > > >> could talk
>> >> >> > > > > > > > >> >>>> > >> through
>> >> >> > > > > > > > >> >>>> > >> > > > those
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly
>> >> >> quickly...
>> >> >> > > > > > > > >> >>>> > >> > > > > > >>
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
>> >> >> > > > > > > > >> >>>> > >> > > > > > >>
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27
>> AM,
>> >> >> Andrii
>> >> >> > > > > > > > Biletskyi <
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly>
>> >> >> wrote:
>> >> >> > > > > > > > >> >>>> > >> > > > > > >>
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> >
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address some
>> of
>> >> >> the
>> >> >> > > > issues
>> >> >> > > > > > > which
>> >> >> > > > > > > > >> were
>> >> >> > > > > > > > >> >>>> > >> mentioned
>> >> >> > > > > > > > >> >>>> > >> > > > > earlier
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> about
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of
>> >> >> those was
>> >> >> > > > > about
>> >> >> > > > > > > > >> batching
>> >> >> > > > > > > > >> >>>> > >> > operations.
>> >> >> > > > > > > > >> >>>> > >> > > > What
>> >> >> > > > > > > > >> >>>> > >> > > > > > if
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> we
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand
>> approach
>> >> >> and let
>> >> >> > > > > people
>> >> >> > > > > > > > >> specify
>> >> >> > > > > > > > >> >>>> > >> topic-name
>> >> >> > > > > > > > >> >>>> > >> > > by
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most of
>> the
>> >> use
>> >> >> > > cases?
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> >
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what
>> information
>> >> >> should
>> >> >> > > we
>> >> >> > > > > > > > generally
>> >> >> > > > > > > > >> >>>> provide in
>> >> >> > > > > > > > >> >>>> > >> > > Admin
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin
>> commands
>> >> >> don't
>> >> >> > > > imply
>> >> >> > > > > > > they
>> >> >> > > > > > > > >> will
>> >> >> > > > > > > > >> >>>> be used
>> >> >> > > > > > > > >> >>>> > >> > only
>> >> >> > > > > > > > >> >>>> > >> > > > in
>> >> >> > > > > > > > >> >>>> > >> > > > > > CLI
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > but,
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a
>> very
>> >> >> > > important
>> >> >> > > > > > > client
>> >> >> > > > > > > > >> of this
>> >> >> > > > > > > > >> >>>> > >> > feature.
>> >> >> > > > > > > > >> >>>> > >> > > In
>> >> >> > > > > > > > >> >>>> > >> > > > > > this
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > case,
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would
>> like
>> >> to
>> >> >> > > provide
>> >> >> > > > > > users
>> >> >> > > > > > > > >> with
>> >> >> > > > > > > > >> >>>> rich
>> >> >> > > > > > > > >> >>>> > >> > > experience
>> >> >> > > > > > > > >> >>>> > >> > > > > in
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> terms
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > of
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > getting results / errors of
>> >> the
>> >> >> > > > executed
>> >> >> > > > > > > > >> commands.
>> >> >> > > > > > > > >> >>>> Usually
>> >> >> > > > > > > > >> >>>> > >> we
>> >> >> > > > > > > > >> >>>> > >> > > > supply
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> with
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > responses only errorCode,
>> >> which
>> >> >> looks
>> >> >> > > > > very
>> >> >> > > > > > > > >> limiting,
>> >> >> > > > > > > > >> >>>> in case
>> >> >> > > > > > > > >> >>>> > >> > of
>> >> >> > > > > > > > >> >>>> > >> > > > CLI
>> >> >> > > > > > > > >> >>>> > >> > > > > we
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> may
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human
>> readable
>> >> >> error
>> >> >> > > > > > > description.
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> >
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into account
>> >> >> previous item
>> >> >> > > > > about
>> >> >> > > > > > > > >> batching,
>> >> >> > > > > > > > >> >>>> what
>> >> >> > > > > > > > >> >>>> > >> do
>> >> >> > > > > > > > >> >>>> > >> > > you
>> >> >> > > > > > > > >> >>>> > >> > > > > > think
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > about
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> >
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support
>> >> >> regexp)
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest =>
>> >> TopicName
>> >> >> > > > > Partitions
>> >> >> > > > > > > > >> Replicas
>> >> >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse =>
>> >> ErrorCode
>> >> >> > > > > > > > ErrorDescription
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription =>
>> string
>> >> >> (empty
>> >> >> > > if
>> >> >> > > > > > > > >> successful)
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> >
>> >> >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
>> >> >> TopicNameRegexp
>> >> >> > > >
>> >> >
>> >> > ...
>> >> >
>> >> > [Message clipped]
>> >>
>>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Gwen,

Yes, looks like KAFKA-1927 will leave TopicMetadataRequest/Response.
But I believe, KIP is still tightly related with KAFKA-1927 since we are
not only
going to update TopicMetadataRequest there but we will introduce a bunch
of new requests too. And it probably makes sense to do those correctly from
scratch - without introducing scala request objects. As I understand you'll
have this common infrastructure code done in KAFKA-1927.

Thanks,
Andrii Biletskyi

On Wed, Mar 18, 2015 at 8:38 PM, Gwen Shapira <gs...@cloudera.com> wrote:

> On Wed, Mar 18, 2015 at 9:34 AM, Jun Rao <ju...@confluent.io> wrote:
> > Andri,
> >
> > Thanks for the summary.
> >
> > 1. I just realized that in order to start working on KAFKA-1927, we will
> > need to merge the changes to OffsetCommitRequest (from 0.8.2) to trunk.
> > This is planned to be done as part of KAFKA-1634. So, we will need
> Guozhang
> > and Joel's help to wrap this up.
>
> I mentioned this in a separate thread, but it may be more relevant here:
> It looks like the SimpleConsumer API exposes TopicMetadataRequest and
> TopicMetadataResponse.
> This means that KAFKA-1927 doesn't remove this duplication.
>
> So I'm not sure we actually need KAFKA-1927 before implementing this KIP.
> This doesn't mean I'm stopping work on KAFKA-1927, but perhaps it
> means we can proceed in parallel?
>
> > 2. Thinking about this a bit more, if the semantic of those "write"
> > requests is async (i.e., after the client gets a response, it just means
> > that the operation is initiated, but not necessarily completed), we don't
> > really need to forward the requests to the controller. Instead, the
> > receiving broker can just write the operation to ZK as the admin command
> > line tool previously does. This will simplify the implementation.
> >
> > 8. There is another implementation detail for describe topic. Ideally, we
> > want to read the topic config from the broker cache, instead of
> ZooKeeper.
> > Currently, every broker reads the topic-level config for all topics.
> > However, it ignores those for topics not hosted on itself. So, we may
> need
> > to change TopicConfigManager a bit so that it caches the configs for all
> > topics.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> >> Guys,
> >>
> >> Thanks for a great discussion!
> >> Here are the actions points:
> >>
> >> 1. Q: Get rid of all scala requests objects, use java protocol
> definitions.
> >>     A: Gwen kindly took that (KAFKA-1927). It's important to speed up
> >> review procedure
> >>          there since this ticket blocks other important changes.
> >>
> >> 2. Q: Generic re-reroute facility vs client maintaining cluster state.
> >>     A: Jay has added pseudo code to KAFKA-1912 - need to consider
> whether
> >> this will be
> >>         easy to implement as a server-side feature (comments are
> >> welcomed!).
> >>
> >> 3. Q: Controller field in wire protocol.
> >>     A: This might be useful for clients, add this to
> TopicMetadataResponse
> >> (already in KIP).
> >>
> >> 4. Q: Decoupling topic creation from TMR.
> >>     A: I will add proposed by Jun solution (using clientId for that) to
> the
> >> KIP.
> >>
> >> 5. Q: Bumping new versions of TMR vs grabbing all protocol changes in
> one
> >> version.
> >>     A: It was decided to try to gather all changes to protocol (before
> >> release).
> >>         In case of TMR it worth checking: KAFKA-2020 and KIP-13 (quotas)
> >>
> >> 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
> >>     A: Use jackson for that, /tools project is a separate jar so
> shouldn't
> >> be a big deal.
> >>
> >> 7.  Q: VerifyReassingPartitions vs generic status check command.
> >>      A: For long-running requests like reassign partitions *progress*
> check
> >> request is useful,
> >>          it makes sense to introduce it.
> >>
> >>  Please add, correct me if I missed something.
> >>
> >> Thanks,
> >> Andrii Biletskyi
> >>
> >> On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> >> andrii.biletskyi@stealth.ly> wrote:
> >>
> >> > Joel,
> >> >
> >> > You are right, I removed ClusterMetadata because we have partially
> >> > what we need in TopicMetadata. Also, as Jay pointed out earlier, we
> >> > would like to have "orthogonal" API, but at the same time we need
> >> > to be backward compatible.
> >> >
> >> > But I like your idea and even have some other arguments for this
> option:
> >> > There is also DescribeTopicRequest which was proposed in this KIP,
> >> > it returns topic configs, partitions, replication factor plus
> partition
> >> > ISR, ASR,
> >> > leader replica. The later part is really already there in
> >> > TopicMetadataRequest.
> >> > So again we'll have to add stuff to TMR, not to duplicate some info in
> >> > newly added requests. However, this way we'll end up with "monster"
> >> > request which returns cluster metadata, topic replication and config
> info
> >> > plus partition replication data. Seems logical to split TMR to
> >> > - ClusterMetadata (brokers + controller, maybe smth else)
> >> > - TopicMetadata (topic info + partition details)
> >> > But since current TMR is involved in lots of places (including network
> >> > client,
> >> > as I understand) this might be very serious change and it probably
> makes
> >> > sense to stick with current approach.
> >> >
> >> > Thanks,
> >> > Andrii Biletskyi
> >> >
> >> >
> >> > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jj...@gmail.com>
> wrote:
> >> >
> >> >> I may be missing some context but hopefully this will also be covered
> >> >> today: I thought the earlier proposal where there was an explicit
> >> >> ClusterMetadata request was clearer and explicit. During the course
> of
> >> >> this thread I think the conclusion was that the main need was for
> >> >> controller information and that can be rolled into the topic metadata
> >> >> response but that seems a bit irrelevant to topic metadata. FWIW I
> >> >> think the full broker-list is also irrelevant to topic metadata, but
> >> >> it is already there and in use. I think there is still room for an
> >> >> explicit ClusterMetadata request since there may be other
> >> >> cluster-level information that we may want to add over time (and that
> >> >> have nothing to do with topic metadata).
> >> >>
> >> >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
> >> >> > Jun,
> >> >> >
> >> >> > 101. Okay, if you say that such use case is important. I also think
> >> >> > using clientId for these purposes is fine - if we already have this
> >> >> field
> >> >> > as part of all Wire protocol messages, why not use that.
> >> >> > I will update KIP-4 page if nobody has other ideas (which may come
> up
> >> >> > during the call today).
> >> >> >
> >> >> > 102.1 Agree, I'll update the KIP accordingly. I think we can add
> new,
> >> >> > fine-grained error codes if some error code received in specific
> case
> >> >> > won't give enough context to return a descriptive error message for
> >> >> user.
> >> >> >
> >> >> > Look forward to discussing all outstanding issues in detail today
> >> during
> >> >> > the call.
> >> >> >
> >> >> > Thanks,
> >> >> > Andrii Biletskyi
> >> >> >
> >> >> >
> >> >> >
> >> >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io>
> wrote:
> >> >> >
> >> >> > > 101. There may be a use case where you only want the topics to be
> >> >> created
> >> >> > > manually by admins. Currently, you can do that by disabling auto
> >> topic
> >> >> > > creation and issue topic creation from the TopicCommand. If we
> >> >> disable auto
> >> >> > > topic creation completely on the broker and don't have a way to
> >> >> distinguish
> >> >> > > between topic creation requests from the regular clients and the
> >> >> admin, we
> >> >> > > can't support manual topic creation any more. I was thinking that
> >> >> another
> >> >> > > way of distinguishing the clients making the topic creation
> requests
> >> >> is
> >> >> > > using clientId. For example, the admin tool can set it to
> something
> >> >> like
> >> >> > > admin and the broker can treat that clientId specially.
> >> >> > >
> >> >> > > Also, there is a related discussion in KAFKA-2020. Currently, we
> do
> >> >> the
> >> >> > > following in TopicMetadataResponse:
> >> >> > >
> >> >> > > 1. If leader is not available, we set the partition level error
> code
> >> >> to
> >> >> > > LeaderNotAvailable.
> >> >> > > 2. If a non-leader replica is not available, we take that replica
> >> out
> >> >> of
> >> >> > > the assigned replica list and isr in the response. As an
> indication
> >> >> for
> >> >> > > doing that, we set the partition level error code to
> >> >> ReplicaNotAvailable.
> >> >> > >
> >> >> > > This has a few problems. First, ReplicaNotAvailable probably
> >> >> shouldn't be
> >> >> > > an error, at least for the normal producer/consumer clients that
> >> just
> >> >> want
> >> >> > > to find out the leader. Second, it can happen that both the
> leader
> >> and
> >> >> > > another replica are not available at the same time. There is no
> >> error
> >> >> code
> >> >> > > to indicate both. Third, even if a replica is not available, it's
> >> >> still
> >> >> > > useful to return its replica id since some clients (e.g. admin
> tool)
> >> >> may
> >> >> > > still make use of it.
> >> >> > >
> >> >> > > One way to address this issue is to always return the replica id
> for
> >> >> > > leader, assigned replicas, and isr regardless of whether the
> >> >> corresponding
> >> >> > > broker is live or not. Since we also return the list of live
> >> brokers,
> >> >> the
> >> >> > > client can figure out whether a leader or a replica is live or
> not
> >> >> and act
> >> >> > > accordingly. This way, we don't need to set the partition level
> >> error
> >> >> code
> >> >> > > when the leader or a replica is not available. This doesn't
> change
> >> >> the wire
> >> >> > > protocol, but does change the semantics. Since we are evolving
> the
> >> >> protocol
> >> >> > > of TopicMetadataRequest here, we can potentially piggyback the
> >> change.
> >> >> > >
> >> >> > > 102.1 For those types of errors due to invalid input, shouldn't
> we
> >> >> just
> >> >> > > guard it at parameter validation time and throw
> >> >> InvalidArgumentException
> >> >> > > without even sending the request to the broker?
> >> >> > >
> >> >> > > Thanks,
> >> >> > >
> >> >> > > Jun
> >> >> > >
> >> >> > >
> >> >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> >> >> > > andrii.biletskyi@stealth.ly> wrote:
> >> >> > >
> >> >> > > > Jun,
> >> >> > > >
> >> >> > > > Answering your questions:
> >> >> > > >
> >> >> > > > 101. If I understand you correctly, you are saying future
> producer
> >> >> > > versions
> >> >> > > > (which
> >> >> > > > will be ported to TMR_V1) won't be able to automatically create
> >> >> topic (if
> >> >> > > > we
> >> >> > > > unconditionally remove topic creation from there). But we need
> to
> >> >> this
> >> >> > > > preserve logic.
> >> >> > > > Ok, about your proposal: I'm not a big fan too, when it comes
> to
> >> >> > > > differentiating
> >> >> > > > clients directly in protocol schema. And also I'm not sure I
> >> >> understand
> >> >> > > at
> >> >> > > > all why
> >> >> > > > auto.create.topics.enable is a server side configuration. Can
> we
> >> >> > > deprecate
> >> >> > > > this setting
> >> >> > > > in future versions, add this setting to producer and based on
> that
> >> >> upon
> >> >> > > > receiving
> >> >> > > > UnknownTopic create topic explicitly by a separate producer
> call
> >> via
> >> >> > > > adminClient?
> >> >> > > >
> >> >> > > > 102.1. Hm, yes. It's because we want to support batching and at
> >> the
> >> >> same
> >> >> > > > time we
> >> >> > > > want to give descriptive error messages for clients. Since
> >> >> AdminClient
> >> >> > > > holds the context
> >> >> > > > to construct such messages (e.g. AdminClient layer can know
> that
> >> >> > > > InvalidArgumentsCode
> >> >> > > > means two cases: either invalid number - e.g. -1; or
> >> >> replication-factor
> >> >> > > was
> >> >> > > > provided while
> >> >> > > > partitions argument wasn't) - I wrapped responses in
> Exceptions.
> >> >> But I'm
> >> >> > > > open to any
> >> >> > > > other ideas, this was just initial version.
> >> >> > > > 102.2. Yes, I agree. I'll change that to probably some other
> dto.
> >> >> > > >
> >> >> > > > Thanks,
> >> >> > > > Andrii Biletskyi
> >> >> > > >
> >> >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io>
> >> wrote:
> >> >> > > >
> >> >> > > > > Andrii,
> >> >> > > > >
> >> >> > > > > 101. That's what I was thinking too, but it may not be that
> >> >> simple. In
> >> >> > > > > TopicMetadataRequest_V1,
> >> >> > > > > we can let it not trigger auto topic creation. Then, in the
> >> >> producer
> >> >> > > > side,
> >> >> > > > > if it gets an UnknownTopicException, it can explicitly issue
> a
> >> >> > > > > createTopicRequest for auto topic creation. On the consumer
> >> side,
> >> >> it
> >> >> > > will
> >> >> > > > > never issue createTopicRequest. This works when auto topic
> >> >> creation is
> >> >> > > > > enabled on the broker side. However, I am not sure how things
> >> >> will work
> >> >> > > > > when auto topic creation is disabled on the broker side. In
> this
> >> >> case,
> >> >> > > we
> >> >> > > > > want to have a way to manually create a topic, potentially
> >> through
> >> >> > > admin
> >> >> > > > > commands. However, then we need a way to distinguish
> >> >> createTopicRequest
> >> >> > > > > issued from the producer clients and the admin tools. May be
> we
> >> >> can
> >> >> > > add a
> >> >> > > > > new field in createTopicRequest and set it differently in the
> >> >> producer
> >> >> > > > > client and the admin client. However, I am not sure if that's
> >> the
> >> >> best
> >> >> > > > > approach.
> >> >> > > > >
> >> >> > > > > 2. Yes, refactoring existing requests is a non-trivial
> amount of
> >> >> work.
> >> >> > > I
> >> >> > > > > posted some comments in KAFKA-1927. We will probably have to
> fix
> >> >> > > > KAFKA-1927
> >> >> > > > > first, before adding the new logic in KAFKA-1694. Otherwise,
> the
> >> >> > > changes
> >> >> > > > > will be too big.
> >> >> > > > >
> >> >> > > > > 102. About the AdminClient:
> >> >> > > > > 102.1. It's a bit weird that we return exception in the api.
> It
> >> >> seems
> >> >> > > > that
> >> >> > > > > we should either return error code or throw an exception when
> >> >> getting
> >> >> > > the
> >> >> > > > > response state.
> >> >> > > > > 102.2. We probably shouldn't explicitly use the request
> object
> >> in
> >> >> the
> >> >> > > > api.
> >> >> > > > > Not every request evolution requires an api change.
> >> >> > > > >
> >> >> > > > > Thanks,
> >> >> > > > >
> >> >> > > > > Jun
> >> >> > > > >
> >> >> > > > >
> >> >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> >> >> > > > > andrii.biletskyi@stealth.ly> wrote:
> >> >> > > > >
> >> >> > > > > > Jun,
> >> >> > > > > >
> >> >> > > > > > Thanks for you comments. Answers inline:
> >> >> > > > > >
> >> >> > > > > > 100. There are a few fields such as ReplicaAssignment,
> >> >> > > > > > > ReassignPartitionRequest,
> >> >> > > > > > > and PartitionsSerialized that are represented as a
> string,
> >> but
> >> >> > > > contain
> >> >> > > > > > > composite structures in json. Could we flatten them out
> >> >> directly in
> >> >> > > > the
> >> >> > > > > > > protocol definition as arrays/records?
> >> >> > > > > >
> >> >> > > > > >
> >> >> > > > > > Yes, now with Admin Client this looks a bit weird. My
> initial
> >> >> > > > motivation
> >> >> > > > > > was:
> >> >> > > > > > ReassignPartitionCommand accepts input in json, we want to
> >> >> remain
> >> >> > > > tools'
> >> >> > > > > > interfaces unchanged, where possible.
> >> >> > > > > > If we port it to deserialized format, in CLI (/tools
> project)
> >> >> we will
> >> >> > > > > have
> >> >> > > > > > to add some
> >> >> > > > > > json library since /tools is written in java and we'll
> need to
> >> >> > > > > deserialize
> >> >> > > > > > json file
> >> >> > > > > > provided by a user. Can we quickly agree on what this
> library
> >> >> should
> >> >> > > be
> >> >> > > > > > (Jackson, GSON, whatever)?
> >> >> > > > > >
> >> >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> >> >> creation?
> >> >> > > > This
> >> >> > > > > > > will be a bit weird now that we have a separate topic
> >> >> creation api.
> >> >> > > > > Have
> >> >> > > > > > > you thought about how the new createTopicRequest and
> >> >> > > > > TopicMetadataRequest
> >> >> > > > > > > v1 will be used in the producer/consumer client, in
> addition
> >> >> to
> >> >> > > admin
> >> >> > > > > > > tools? For example, ideally, we don't want
> >> >> TopicMetadataRequest
> >> >> > > from
> >> >> > > > > the
> >> >> > > > > > > consumer to trigger auto topic creation.
> >> >> > > > > >
> >> >> > > > > >
> >> >> > > > > > I agree, this strange logic should be fixed. I'm not
> confident
> >> >> in
> >> >> > > this
> >> >> > > > > > Kafka part so
> >> >> > > > > > correct me if I'm wrong, but it doesn't look like a hard
> thing
> >> >> to
> >> >> > > do, I
> >> >> > > > > > think we can
> >> >> > > > > > leverage AdminClient for that in Producer and
> unconditionally
> >> >> remove
> >> >> > > > > topic
> >> >> > > > > > creation from the TopicMetadataRequest_V1.
> >> >> > > > > >
> >> >> > > > > > 2. I think Jay meant getting rid of scala classes
> >> >> > > > > > > like HeartbeatRequestAndHeader and
> >> >> HeartbeatResponseAndHeader. We
> >> >> > > did
> >> >> > > > > > that
> >> >> > > > > > > as a stop-gap thing when adding the new requests for the
> >> >> consumers.
> >> >> > > > > > > However, the long term plan is to get rid of all those
> and
> >> >> just
> >> >> > > reuse
> >> >> > > > > the
> >> >> > > > > > > java request/response in the client. Since this KIP
> proposes
> >> >> to
> >> >> > > add a
> >> >> > > > > > > significant number of new requests, perhaps we should
> bite
> >> the
> >> >> > > bullet
> >> >> > > > > to
> >> >> > > > > > > clean up the existing scala requests first before adding
> new
> >> >> ones?
> >> >> > > > > > >
> >> >> > > > > >
> >> >> > > > > > Yes, looks like I misunderstood the point of
> >> >> ...RequestAndHeader.
> >> >> > > > Okay, I
> >> >> > > > > > will
> >> >> > > > > > rework that. The only thing is that I don't see any example
> >> how
> >> >> it
> >> >> > > was
> >> >> > > > > done
> >> >> > > > > > for at
> >> >> > > > > > least one existing protocol message. Thus, as I
> understand, I
> >> >> have to
> >> >> > > > > think
> >> >> > > > > > how we
> >> >> > > > > > are going to do it.
> >> >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
> >> reasonable,
> >> >> but
> >> >> > > if
> >> >> > > > > it's
> >> >> > > > > > an *obligatory*
> >> >> > > > > > requirement to have Admin KIP done, I'm afraid this can be
> a
> >> >> serious
> >> >> > > > > > blocker for us.
> >> >> > > > > > There are 13 protocol messages and all that would require
> not
> >> >> only
> >> >> > > unit
> >> >> > > > > > tests but quite
> >> >> > > > > > intensive manual testing, no? I'm afraid I'm not the right
> guy
> >> >> to
> >> >> > > cover
> >> >> > > > > > pretty much all
> >> >> > > > > > Kafka core internals :). Let me know your thoughts on this
> >> >> item. Btw
> >> >> > > > > there
> >> >> > > > > > is a ticket to
> >> >> > > > > > follow-up this issue (
> >> >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> >> >> > > > ).
> >> >> > > > > >
> >> >> > > > > > Thanks,
> >> >> > > > > > Andrii Biletskyi
> >> >> > > > > >
> >> >> > > > > >
> >> >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <jun@confluent.io
> >
> >> >> wrote:
> >> >> > > > > >
> >> >> > > > > > > Andrii,
> >> >> > > > > > >
> >> >> > > > > > >
> >> >> > > > > > > A few more comments.
> >> >> > > > > > >
> >> >> > > > > > > 100. There are a few fields such as ReplicaAssignment,
> >> >> > > > > > > ReassignPartitionRequest,
> >> >> > > > > > > and PartitionsSerialized that are represented as a
> string,
> >> but
> >> >> > > > contain
> >> >> > > > > > > composite structures in json. Could we flatten them out
> >> >> directly in
> >> >> > > > the
> >> >> > > > > > > protocol definition as arrays/records?
> >> >> > > > > > >
> >> >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto
> topic
> >> >> > > creation?
> >> >> > > > > This
> >> >> > > > > > > will be a bit weird now that we have a separate topic
> >> >> creation api.
> >> >> > > > > Have
> >> >> > > > > > > you thought about how the new createTopicRequest and
> >> >> > > > > TopicMetadataRequest
> >> >> > > > > > > v1 will be used in the producer/consumer client, in
> addition
> >> >> to
> >> >> > > admin
> >> >> > > > > > > tools? For example, ideally, we don't want
> >> >> TopicMetadataRequest
> >> >> > > from
> >> >> > > > > the
> >> >> > > > > > > consumer to trigger auto topic creation.
> >> >> > > > > > >
> >> >> > > > > > > 2. I think Jay meant getting rid of scala classes
> >> >> > > > > > > like HeartbeatRequestAndHeader and
> >> >> HeartbeatResponseAndHeader. We
> >> >> > > did
> >> >> > > > > > that
> >> >> > > > > > > as a stop-gap thing when adding the new requests for the
> >> >> consumers.
> >> >> > > > > > > However, the long term plan is to get rid of all those
> and
> >> >> just
> >> >> > > reuse
> >> >> > > > > the
> >> >> > > > > > > java request/response in the client. Since this KIP
> proposes
> >> >> to
> >> >> > > add a
> >> >> > > > > > > significant number of new requests, perhaps we should
> bite
> >> the
> >> >> > > bullet
> >> >> > > > > to
> >> >> > > > > > > clean up the existing scala requests first before adding
> new
> >> >> ones?
> >> >> > > > > > >
> >> >> > > > > > > Thanks,
> >> >> > > > > > >
> >> >> > > > > > > Jun
> >> >> > > > > > >
> >> >> > > > > > >
> >> >> > > > > > >
> >> >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> >> >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> >> > > > > > >
> >> >> > > > > > > > Hi,
> >> >> > > > > > > >
> >> >> > > > > > > > As said above - I list again all comments from this
> thread
> >> >> so we
> >> >> > > > > > > > can see what's left and finalize all pending issues.
> >> >> > > > > > > >
> >> >> > > > > > > > Comments from Jay:
> >> >> > > > > > > > 1. This is much needed functionality, but there are a
> lot
> >> >> of the
> >> >> > > so
> >> >> > > > > > let's
> >> >> > > > > > > > really think these protocols through. We really want to
> >> end
> >> >> up
> >> >> > > > with a
> >> >> > > > > > set
> >> >> > > > > > > > of well thought-out, orthoganol apis. For this reason I
> >> >> think it
> >> >> > > is
> >> >> > > > > > > really
> >> >> > > > > > > > important to think through the end state even if that
> >> >> includes
> >> >> > > APIs
> >> >> > > > > we
> >> >> > > > > > > > won't implement in the first phase.
> >> >> > > > > > > >
> >> >> > > > > > > > A: Definitely behind this. Would appreciate if there
> are
> >> >> concrete
> >> >> > > > > > > comments
> >> >> > > > > > > > how this can be improved.
> >> >> > > > > > > >
> >> >> > > > > > > > 2. Let's please please please wait until we have
> switched
> >> >> the
> >> >> > > > server
> >> >> > > > > > over
> >> >> > > > > > > > to the new java protocol definitions. If we add upteen
> >> more
> >> >> ad
> >> >> > > hoc
> >> >> > > > > > scala
> >> >> > > > > > > > objects that is just generating more work for the
> >> >> conversion we
> >> >> > > > know
> >> >> > > > > we
> >> >> > > > > > > > have to do.
> >> >> > > > > > > >
> >> >> > > > > > > > A: Fixed in the latest patch - removed scala protocol
> >> >> classes.
> >> >> > > > > > > >
> >> >> > > > > > > > 3. This proposal introduces a new type of optional
> >> >> parameter.
> >> >> > > This
> >> >> > > > is
> >> >> > > > > > > > inconsistent with everything else in the protocol
> where we
> >> >> use -1
> >> >> > > > or
> >> >> > > > > > some
> >> >> > > > > > > > other marker value. You could argue either way but
> let's
> >> >> stick
> >> >> > > with
> >> >> > > > > > that
> >> >> > > > > > > > for consistency. For clients that implemented the
> protocol
> >> >> in a
> >> >> > > > > better
> >> >> > > > > > > way
> >> >> > > > > > > > than our scala code these basic primitives are hard to
> >> >> change.
> >> >> > > > > > > >
> >> >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf type and
> >> >> changed
> >> >> > > > > > protocol
> >> >> > > > > > > > accordingly.
> >> >> > > > > > > >
> >> >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> >> >> TopicMetadataRequest
> >> >> > > > > which
> >> >> > > > > > > has
> >> >> > > > > > > > brokers, topics, and partitions. I think we should
> rename
> >> >> that
> >> >> > > > > request
> >> >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest) and
> >> >> include the
> >> >> > > id
> >> >> > > > > of
> >> >> > > > > > > the
> >> >> > > > > > > > controller. Or are there other things we could add
> here?
> >> >> > > > > > > >
> >> >> > > > > > > > A: I agree. Updated the KIP. Let's extends
> TopicMetadata
> >> to
> >> >> > > > version 2
> >> >> > > > > > and
> >> >> > > > > > > > include controller.
> >> >> > > > > > > >
> >> >> > > > > > > > 5. We have a tendency to try to make a lot of requests
> >> that
> >> >> can
> >> >> > > > only
> >> >> > > > > go
> >> >> > > > > > > to
> >> >> > > > > > > > particular nodes. This adds a lot of burden for client
> >> >> > > > > implementations
> >> >> > > > > > > (it
> >> >> > > > > > > > sounds easy but each discovery can fail in many parts
> so
> >> it
> >> >> ends
> >> >> > > up
> >> >> > > > > > > being a
> >> >> > > > > > > > full state machine to do right). I think we should
> >> consider
> >> >> > > making
> >> >> > > > > > admin
> >> >> > > > > > > > commands and ideally as many of the other apis as
> possible
> >> >> > > > available
> >> >> > > > > on
> >> >> > > > > > > all
> >> >> > > > > > > > brokers and just redirect to the controller on the
> broker
> >> >> side.
> >> >> > > > > Perhaps
> >> >> > > > > > > > there would be a general way to encapsulate this
> >> re-routing
> >> >> > > > behavior.
> >> >> > > > > > > >
> >> >> > > > > > > > A: It's a very interesting idea, but seems there are
> some
> >> >> > > concerns
> >> >> > > > > > about
> >> >> > > > > > > > this
> >> >> > > > > > > > feature (like performance considerations, how this will
> >> >> > > complicate
> >> >> > > > > > server
> >> >> > > > > > > > etc).
> >> >> > > > > > > > I believe this shouldn't be a blocker. If this feature
> is
> >> >> > > > implemented
> >> >> > > > > > at
> >> >> > > > > > > > some
> >> >> > > > > > > > point it won't affect Admin changes - at least no
> changes
> >> to
> >> >> > > public
> >> >> > > > > API
> >> >> > > > > > > > will be required.
> >> >> > > > > > > >
> >> >> > > > > > > > 6. We should probably normalize the key value pairs
> used
> >> for
> >> >> > > > configs
> >> >> > > > > > > rather
> >> >> > > > > > > > than embedding a new formatting. So two strings rather
> >> than
> >> >> one
> >> >> > > > with
> >> >> > > > > an
> >> >> > > > > > > > internal equals sign.
> >> >> > > > > > > >
> >> >> > > > > > > > A: Fixed in the latest patch - normalized configs and
> >> >> changed
> >> >> > > > > protocol
> >> >> > > > > > > > accordingly.
> >> >> > > > > > > >
> >> >> > > > > > > > 7. Is the postcondition of these APIs that the command
> has
> >> >> begun
> >> >> > > or
> >> >> > > > > > that
> >> >> > > > > > > > the command has been completed? It is a lot more
> usable if
> >> >> the
> >> >> > > > > command
> >> >> > > > > > > has
> >> >> > > > > > > > been completed so you know that if you create a topic
> and
> >> >> then
> >> >> > > > > publish
> >> >> > > > > > to
> >> >> > > > > > > > it you won't get an exception about there being no such
> >> >> topic.
> >> >> > > > > > > >
> >> >> > > > > > > > A: For long running requests (like reassign
> partitions) -
> >> >> the
> >> >> > > post
> >> >> > > > > > > > condition is
> >> >> > > > > > > > command has begun - so we don't block the client. In
> case
> >> >> of your
> >> >> > > > > > > example -
> >> >> > > > > > > > topic commands, this will be refactored and topic
> commands
> >> >> will
> >> >> > > be
> >> >> > > > > > > executed
> >> >> > > > > > > > immediately, since the Controller will serve Admin
> >> requests
> >> >> > > > > > > > (follow-up ticket KAFKA-1777).
> >> >> > > > > > > >
> >> >> > > > > > > > 8. Describe topic and list topics duplicate a lot of
> stuff
> >> >> in the
> >> >> > > > > > > metadata
> >> >> > > > > > > > request. Is there a reason to give back topics marked
> for
> >> >> > > > deletion? I
> >> >> > > > > > > feel
> >> >> > > > > > > > like if we just make the post-condition of the delete
> >> >> command be
> >> >> > > > that
> >> >> > > > > > the
> >> >> > > > > > > > topic is deleted that will get rid of the need for this
> >> >> right?
> >> >> > > And
> >> >> > > > it
> >> >> > > > > > > will
> >> >> > > > > > > > be much more intuitive.
> >> >> > > > > > > >
> >> >> > > > > > > > A: Fixed in the latest patch - removed topics marked
> for
> >> >> deletion
> >> >> > > > in
> >> >> > > > > > > > ListTopicsRequest.
> >> >> > > > > > > >
> >> >> > > > > > > > 9. Should we consider batching these requests? We have
> >> >> generally
> >> >> > > > > tried
> >> >> > > > > > to
> >> >> > > > > > > > allow multiple operations to be batched. My suspicion
> is
> >> >> that
> >> >> > > > without
> >> >> > > > > > > this
> >> >> > > > > > > > we will get a lot of code that does something like
> >> >> > > > > > > >    for(topic: adminClient.listTopics())
> >> >> > > > > > > >       adminClient.describeTopic(topic)
> >> >> > > > > > > > this code will work great when you test on 5 topics but
> >> not
> >> >> do as
> >> >> > > > > well
> >> >> > > > > > if
> >> >> > > > > > > > you have 50k.
> >> >> > > > > > > >
> >> >> > > > > > > > A: Updated the KIP - please check "Topic Admin Schema"
> >> >> section.
> >> >> > > > > > > >
> >> >> > > > > > > > 10. I think we should also discuss how we want to
> expose a
> >> >> > > > > programmatic
> >> >> > > > > > > JVM
> >> >> > > > > > > > client api for these operations. Currently people rely
> on
> >> >> > > > AdminUtils
> >> >> > > > > > > which
> >> >> > > > > > > > is totally sketchy. I think we probably need another
> >> client
> >> >> under
> >> >> > > > > > > clients/
> >> >> > > > > > > > that exposes administrative functionality. We will need
> >> >> this just
> >> >> > > > to
> >> >> > > > > > > > properly test the new apis, I suspect. We should figure
> >> out
> >> >> that
> >> >> > > > API.
> >> >> > > > > > > >
> >> >> > > > > > > > A: Updated the KIP - please check "Admin Client"
> section
> >> >> with an
> >> >> > > > > > initial
> >> >> > > > > > > > API proposal.
> >> >> > > > > > > >
> >> >> > > > > > > > 11. The other information that would be really useful
> to
> >> get
> >> >> > > would
> >> >> > > > be
> >> >> > > > > > > > information about partitions--how much data is in the
> >> >> partition,
> >> >> > > > what
> >> >> > > > > > are
> >> >> > > > > > > > the segment offsets, what is the log-end offset (i.e.
> last
> >> >> > > offset),
> >> >> > > > > > what
> >> >> > > > > > > is
> >> >> > > > > > > > the compaction point, etc. I think that done right this
> >> >> would be
> >> >> > > > the
> >> >> > > > > > > > successor to the very awkward OffsetRequest we have
> today.
> >> >> > > > > > > >
> >> >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest
> >> >> patch. I
> >> >> > > > > believe
> >> >> > > > > > > > this should
> >> >> > > > > > > > be resolved in a separate KIP / jira ticket.
> >> >> > > > > > > >
> >> >> > > > > > > > 12. Generally we can do good error handling without
> >> needing
> >> >> > > custom
> >> >> > > > > > > > server-side
> >> >> > > > > > > > messages. I.e. generally the client has the context to
> >> know
> >> >> that
> >> >> > > if
> >> >> > > > > it
> >> >> > > > > > > got
> >> >> > > > > > > > an error that the topic doesn't exist to say "Topic X
> >> >> doesn't
> >> >> > > > exist"
> >> >> > > > > > > rather
> >> >> > > > > > > > than "error code 14" (or whatever). Maybe there are
> >> specific
> >> >> > > cases
> >> >> > > > > > where
> >> >> > > > > > > > this is hard? If we want to add server-side error
> messages
> >> >> we
> >> >> > > > really
> >> >> > > > > do
> >> >> > > > > > > > need to do this in a consistent way across the
> protocol.
> >> >> > > > > > > >
> >> >> > > > > > > > A: Updated the KIP - please check "Protocol Errors"
> >> >> section. I
> >> >> > > > added
> >> >> > > > > > the
> >> >> > > > > > > > comprehensive, fine-grained list of error codes.
> >> >> > > > > > > >
> >> >> > > > > > > > Comments from Guozhang:
> >> >> > > > > > > > 13. Describe topic request: it would be great to go
> beyond
> >> >> just
> >> >> > > > > > batching
> >> >> > > > > > > on
> >> >> > > > > > > > topic name regex for this request. For example, a very
> >> >> common use
> >> >> > > > > case
> >> >> > > > > > of
> >> >> > > > > > > > the topic command is to list all topics whose config
> A's
> >> >> value is
> >> >> > > > B.
> >> >> > > > > > With
> >> >> > > > > > > > topic name regex then we have to first retrieve __all__
> >> >> topics's
> >> >> > > > > > > > description info and then filter at the client end,
> which
> >> >> will
> >> >> > > be a
> >> >> > > > > > huge
> >> >> > > > > > > > burden on ZK.
> >> >> > > > > > > > AND
> >> >> > > > > > > > 14. Config K-Vs in create topic: this is related to the
> >> >> previous
> >> >> > > > > point;
> >> >> > > > > > > > maybe we can add another metadata K-V or just a
> metadata
> >> >> string
> >> >> > > > along
> >> >> > > > > > > side
> >> >> > > > > > > > with config K-V in create topic like we did for offset
> >> >> commit
> >> >> > > > > request.
> >> >> > > > > > > This
> >> >> > > > > > > > field can be quite useful in storing information like
> >> >> "owner" of
> >> >> > > > the
> >> >> > > > > > > topic
> >> >> > > > > > > > who issue the create command, etc, which is quite
> >> important
> >> >> for a
> >> >> > > > > > > > multi-tenant setting. Then in the describe topic
> request
> >> we
> >> >> can
> >> >> > > > also
> >> >> > > > > > > batch
> >> >> > > > > > > > on regex of the metadata field.
> >> >> > > > > > > >
> >> >> > > > > > > > A: As discussed it is very interesting but can be
> >> >> implemented
> >> >> > > later
> >> >> > > > > > after
> >> >> > > > > > > > we have some basic functionality there.
> >> >> > > > > > > >
> >> >> > > > > > > > 15. Today all the admin operations are async in the
> sense
> >> >> that
> >> >> > > > > command
> >> >> > > > > > > will
> >> >> > > > > > > > return once it is written in ZK, and that is why we
> need
> >> >> extra
> >> >> > > > > > > verification
> >> >> > > > > > > > like testUtil.waitForTopicCreated() / verify partition
> >> >> > > reassignment
> >> >> > > > > > > > request, etc. With admin requests we could add a flag
> to
> >> >> enable /
> >> >> > > > > > disable
> >> >> > > > > > > > synchronous requests; when it is turned on, the
> response
> >> >> will not
> >> >> > > > > > return
> >> >> > > > > > > > until the request has been completed. And for async
> >> >> requests we
> >> >> > > can
> >> >> > > > > > add a
> >> >> > > > > > > > "token" field in the response, and then only need a
> >> general
> >> >> > > "admin
> >> >> > > > > > > > verification request" with the given token to check if
> the
> >> >> async
> >> >> > > > > > request
> >> >> > > > > > > > has been completed.
> >> >> > > > > > > >
> >> >> > > > > > > > A: I see your point. My idea was to provide specific
> >> >> > > > Verify...Request
> >> >> > > > > > per
> >> >> > > > > > > > each
> >> >> > > > > > > > long running request, where needed. We can do it the
> way
> >> you
> >> >> > > > suggest.
> >> >> > > > > > The
> >> >> > > > > > > > only
> >> >> > > > > > > > concern is that introducing a token we again will make
> >> >> schema
> >> >> > > > > > "dynamic".
> >> >> > > > > > > We
> >> >> > > > > > > > wanted
> >> >> > > > > > > > to do similar thing introducing single AdminRequest for
> >> all
> >> >> topic
> >> >> > > > > > > commands
> >> >> > > > > > > > but rejected
> >> >> > > > > > > > this idea because we wanted to have schema defined. So
> >> this
> >> >> is
> >> >> > > > more a
> >> >> > > > > > > > choice between:
> >> >> > > > > > > > a) have fixed schema but introduce each time new
> >> >> Verify...Request
> >> >> > > > for
> >> >> > > > > > > > long-running requests
> >> >> > > > > > > > b) use one request for verification but generalize it
> with
> >> >> token
> >> >> > > > > > > > I'm fine with whatever decision community come to. Just
> >> let
> >> >> me
> >> >> > > know
> >> >> > > > > > your
> >> >> > > > > > > > thoughts.
> >> >> > > > > > > >
> >> >> > > > > > > > Comment from Gwen:
> >> >> > > > > > > > 16. Specifically for ownership, I think the plan is to
> add
> >> >> ACL
> >> >> > > (it
> >> >> > > > > > sounds
> >> >> > > > > > > > like you are describing ACL) via an external system
> >> (Argus,
> >> >> > > > Sentry).
> >> >> > > > > > > > I remember KIP-11 described this, but I can't find the
> KIP
> >> >> any
> >> >> > > > > longer.
> >> >> > > > > > > >
> >> >> > > > > > > > A: Okay, no problem. Not sure though how we are going
> to
> >> >> handle
> >> >> > > it.
> >> >> > > > > > Wait
> >> >> > > > > > > > which KIP
> >> >> > > > > > > > will be committed first and include changes to
> >> >> TopicMetadata from
> >> >> > > > the
> >> >> > > > > > > later
> >> >> > > > > > > > one?
> >> >> > > > > > > > Anyway, I added this note to "Open Questions" section
> so
> >> we
> >> >> don't
> >> >> > > > > miss
> >> >> > > > > > > this
> >> >> > > > > > > > piece.
> >> >> > > > > > > >
> >> >> > > > > > > > Thanks,
> >> >> > > > > > > > Andrii Biletskyi
> >> >> > > > > > > >
> >> >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> >> >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> >> > > > > > > >
> >> >> > > > > > > > > Hi all,
> >> >> > > > > > > > >
> >> >> > > > > > > > > Today I uploaded the patch that covers some of the
> >> >> discussed
> >> >> > > and
> >> >> > > > > > agreed
> >> >> > > > > > > > > items:
> >> >> > > > > > > > > - removed MaybeOf optional type
> >> >> > > > > > > > > - switched to java protocol definitions
> >> >> > > > > > > > > - simplified messages (normalized configs, removed
> topic
> >> >> marked
> >> >> > > > for
> >> >> > > > > > > > > deletion)
> >> >> > > > > > > > >
> >> >> > > > > > > > > I also updated the KIP-4 with respective changes and
> >> >> wrote down
> >> >> > > > my
> >> >> > > > > > > > > proposal for
> >> >> > > > > > > > > pending items:
> >> >> > > > > > > > > - Batch Admin Operations -> updated Wire Protocol
> schema
> >> >> > > proposal
> >> >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
> >> >> > > > TopicMetadataRequest
> >> >> > > > > > > > > - Admin Client -> updated my initial proposal to
> reflect
> >> >> > > batching
> >> >> > > > > > > > > - Error codes -> proposed fine-grained error code
> >> instead
> >> >> of
> >> >> > > > > > > > > AdminRequestFailed
> >> >> > > > > > > > >
> >> >> > > > > > > > > I will also send a separate email to cover all
> comments
> >> >> from
> >> >> > > this
> >> >> > > > > > > thread.
> >> >> > > > > > > > >
> >> >> > > > > > > > > Thanks,
> >> >> > > > > > > > > Andrii Biletskyi
> >> >> > > > > > > > >
> >> >> > > > > > > > >
> >> >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> >> >> > > > > gshapira@cloudera.com
> >> >> > > > > > >
> >> >> > > > > > > > > wrote:
> >> >> > > > > > > > >
> >> >> > > > > > > > >> Found KIP-11 (
> >> >> > > > > > > > >>
> >> >> > > > > > > >
> >> >> > > > > > >
> >> >> > > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> >> >> > > > > > > > >> )
> >> >> > > > > > > > >> It actually specifies changes to the Metadata
> protocol,
> >> >> so
> >> >> > > > making
> >> >> > > > > > sure
> >> >> > > > > > > > >> both KIPs are consistent in this regard will be
> good.
> >> >> > > > > > > > >>
> >> >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> >> >> > > > > > gshapira@cloudera.com
> >> >> > > > > > > >
> >> >> > > > > > > > >> wrote:
> >> >> > > > > > > > >> > Specifically for ownership, I think the plan is to
> >> add
> >> >> ACL
> >> >> > > (it
> >> >> > > > > > > sounds
> >> >> > > > > > > > >> > like you are describing ACL) via an external
> system
> >> >> (Argus,
> >> >> > > > > > Sentry).
> >> >> > > > > > > > >> > I remember KIP-11 described this, but I can't find
> >> the
> >> >> KIP
> >> >> > > any
> >> >> > > > > > > longer.
> >> >> > > > > > > > >> >
> >> >> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting
> >> >> information
> >> >> > > that
> >> >> > > > > > > already
> >> >> > > > > > > > >> > exists from Kafka brokers, not on adding
> information
> >> >> that
> >> >> > > > > perhaps
> >> >> > > > > > > > >> > should exist but doesn't yet?
> >> >> > > > > > > > >> >
> >> >> > > > > > > > >> > Gwen
> >> >> > > > > > > > >> >
> >> >> > > > > > > > >> >
> >> >> > > > > > > > >> >
> >> >> > > > > > > > >> >
> >> >> > > > > > > > >> >
> >> >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> >> >> > > > > > wangguoz@gmail.com>
> >> >> > > > > > > > >> wrote:
> >> >> > > > > > > > >> >> Folks,
> >> >> > > > > > > > >> >>
> >> >> > > > > > > > >> >> Just want to elaborate a bit more on the
> >> create-topic
> >> >> > > > metadata
> >> >> > > > > > and
> >> >> > > > > > > > >> batching
> >> >> > > > > > > > >> >> describe-topic based on config / metadata in my
> >> >> previous
> >> >> > > > email
> >> >> > > > > as
> >> >> > > > > > > we
> >> >> > > > > > > > >> work
> >> >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to have
> some
> >> >> sort of
> >> >> > > > > topic
> >> >> > > > > > > > >> management
> >> >> > > > > > > > >> >> mechanisms, which I think is quite important in a
> >> >> > > > multi-tenant
> >> >> > > > > /
> >> >> > > > > > > > cloud
> >> >> > > > > > > > >> >> architecture: today anyone can create topics in a
> >> >> shared
> >> >> > > > Kafka
> >> >> > > > > > > > >> cluster, but
> >> >> > > > > > > > >> >> there is no concept or "ownership" of topics that
> >> are
> >> >> > > created
> >> >> > > > > by
> >> >> > > > > > > > >> different
> >> >> > > > > > > > >> >> users. For example, at LinkedIn we basically
> >> >> distinguish
> >> >> > > > topic
> >> >> > > > > > > owners
> >> >> > > > > > > > >> via
> >> >> > > > > > > > >> >> some casual topic name prefix, which is a bit
> >> awkward
> >> >> and
> >> >> > > > does
> >> >> > > > > > not
> >> >> > > > > > > > fly
> >> >> > > > > > > > >> as
> >> >> > > > > > > > >> >> we scale our customers. It would be great to use
> >> >> > > > > describe-topics
> >> >> > > > > > > such
> >> >> > > > > > > > >> as:
> >> >> > > > > > > > >> >>
> >> >> > > > > > > > >> >> Describe all topics that is created by me.
> >> >> > > > > > > > >> >>
> >> >> > > > > > > > >> >> Describe all topics whose retention time is
> >> overriden
> >> >> to X.
> >> >> > > > > > > > >> >>
> >> >> > > > > > > > >> >> Describe all topics whose writable group include
> >> user
> >> >> Y
> >> >> > > (this
> >> >> > > > > is
> >> >> > > > > > > > >> related to
> >> >> > > > > > > > >> >> authorization), etc..
> >> >> > > > > > > > >> >>
> >> >> > > > > > > > >> >> One possible way to achieve this is to add a
> >> metadata
> >> >> file
> >> >> > > in
> >> >> > > > > the
> >> >> > > > > > > > >> >> create-topic request, whose value will also be
> >> >> written ZK
> >> >> > > as
> >> >> > > > we
> >> >> > > > > > > > create
> >> >> > > > > > > > >> the
> >> >> > > > > > > > >> >> topic; then describe-topics can choose to batch
> >> topics
> >> >> > > based
> >> >> > > > on
> >> >> > > > > > 1)
> >> >> > > > > > > > name
> >> >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata regex,
> >> etc.
> >> >> > > > > > > > >> >>
> >> >> > > > > > > > >> >> Thoughts?
> >> >> > > > > > > > >> >>
> >> >> > > > > > > > >> >> Guozhang
> >> >> > > > > > > > >> >>
> >> >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> >> >> > > > > > wangguoz@gmail.com>
> >> >> > > > > > > > >> wrote:
> >> >> > > > > > > > >> >>
> >> >> > > > > > > > >> >>> Thanks for the updated wiki. A few comments
> below:
> >> >> > > > > > > > >> >>>
> >> >> > > > > > > > >> >>> 1. Error description in response: I think if
> some
> >> >> > > errorCode
> >> >> > > > > > could
> >> >> > > > > > > > >> indicate
> >> >> > > > > > > > >> >>> several different error cases then we should
> really
> >> >> change
> >> >> > > > it
> >> >> > > > > to
> >> >> > > > > > > > >> multiple
> >> >> > > > > > > > >> >>> codes. In general the errorCode itself would be
> >> >> precise
> >> >> > > and
> >> >> > > > > > > > >> sufficient for
> >> >> > > > > > > > >> >>> describing the server side errors.
> >> >> > > > > > > > >> >>>
> >> >> > > > > > > > >> >>> 2. Describe topic request: it would be great to
> go
> >> >> beyond
> >> >> > > > just
> >> >> > > > > > > > >> batching on
> >> >> > > > > > > > >> >>> topic name regex for this request. For example,
> a
> >> >> very
> >> >> > > > common
> >> >> > > > > > use
> >> >> > > > > > > > >> case of
> >> >> > > > > > > > >> >>> the topic command is to list all topics whose
> >> config
> >> >> A's
> >> >> > > > value
> >> >> > > > > > is
> >> >> > > > > > > B.
> >> >> > > > > > > > >> With
> >> >> > > > > > > > >> >>> topic name regex then we have to first retrieve
> >> >> __all__
> >> >> > > > > topics's
> >> >> > > > > > > > >> >>> description info and then filter at the client
> end,
> >> >> which
> >> >> > > > will
> >> >> > > > > > be
> >> >> > > > > > > a
> >> >> > > > > > > > >> huge
> >> >> > > > > > > > >> >>> burden on ZK.
> >> >> > > > > > > > >> >>>
> >> >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is related
> to
> >> >> the
> >> >> > > > > previous
> >> >> > > > > > > > point;
> >> >> > > > > > > > >> >>> maybe we can add another metadata K-V or just a
> >> >> metadata
> >> >> > > > > string
> >> >> > > > > > > > along
> >> >> > > > > > > > >> side
> >> >> > > > > > > > >> >>> with config K-V in create topic like we did for
> >> >> offset
> >> >> > > > commit
> >> >> > > > > > > > >> request. This
> >> >> > > > > > > > >> >>> field can be quite useful in storing information
> >> like
> >> >> > > > "owner"
> >> >> > > > > of
> >> >> > > > > > > the
> >> >> > > > > > > > >> topic
> >> >> > > > > > > > >> >>> who issue the create command, etc, which is
> quite
> >> >> > > important
> >> >> > > > > for
> >> >> > > > > > a
> >> >> > > > > > > > >> >>> multi-tenant setting. Then in the describe topic
> >> >> request
> >> >> > > we
> >> >> > > > > can
> >> >> > > > > > > also
> >> >> > > > > > > > >> batch
> >> >> > > > > > > > >> >>> on regex of the metadata field.
> >> >> > > > > > > > >> >>>
> >> >> > > > > > > > >> >>> 4. Today all the admin operations are async in
> the
> >> >> sense
> >> >> > > > that
> >> >> > > > > > > > command
> >> >> > > > > > > > >> will
> >> >> > > > > > > > >> >>> return once it is written in ZK, and that is
> why we
> >> >> need
> >> >> > > > extra
> >> >> > > > > > > > >> verification
> >> >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify
> >> >> partition
> >> >> > > > > > > reassignment
> >> >> > > > > > > > >> >>> request, etc. With admin requests we could add a
> >> >> flag to
> >> >> > > > > enable
> >> >> > > > > > /
> >> >> > > > > > > > >> disable
> >> >> > > > > > > > >> >>> synchronous requests; when it is turned on, the
> >> >> response
> >> >> > > > will
> >> >> > > > > > not
> >> >> > > > > > > > >> return
> >> >> > > > > > > > >> >>> until the request has been completed. And for
> async
> >> >> > > requests
> >> >> > > > > we
> >> >> > > > > > > can
> >> >> > > > > > > > >> add a
> >> >> > > > > > > > >> >>> "token" field in the response, and then only
> need a
> >> >> > > general
> >> >> > > > > > "admin
> >> >> > > > > > > > >> >>> verification request" with the given token to
> check
> >> >> if the
> >> >> > > > > async
> >> >> > > > > > > > >> request
> >> >> > > > > > > > >> >>> has been completed.
> >> >> > > > > > > > >> >>>
> >> >> > > > > > > > >> >>> 5. +1 for extending Metadata request to include
> >> >> > > controller /
> >> >> > > > > > > > >> coordinator
> >> >> > > > > > > > >> >>> information, and then we can remove the
> >> >> ConsumerMetadata /
> >> >> > > > > > > > >> ClusterMetadata
> >> >> > > > > > > > >> >>> requests.
> >> >> > > > > > > > >> >>>
> >> >> > > > > > > > >> >>> Guozhang
> >> >> > > > > > > > >> >>>
> >> >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> >> >> > > > > > jjkoshy.w@gmail.com>
> >> >> > > > > > > > >> wrote:
> >> >> > > > > > > > >> >>>
> >> >> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't
> think I
> >> >> will be
> >> >> > > > > able
> >> >> > > > > > to
> >> >> > > > > > > > >> make
> >> >> > > > > > > > >> >>>> it today, so if notes can be sent out afterward
> >> that
> >> >> > > would
> >> >> > > > be
> >> >> > > > > > > > great.
> >> >> > > > > > > > >> >>>>
> >> >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen
> >> >> Shapira
> >> >> > > > wrote:
> >> >> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking
> forward
> >> >> to
> >> >> > > > > chatting
> >> >> > > > > > > with
> >> >> > > > > > > > >> >>>> everyone :)
> >> >> > > > > > > > >> >>>> >
> >> >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> >> >> > > > > > > joe.stein@stealth.ly>
> >> >> > > > > > > > >> wrote:
> >> >> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout
> invite
> >> to
> >> >> all
> >> >> > > > pmc,
> >> >> > > > > > > > >> committers
> >> >> > > > > > > > >> >>>> and
> >> >> > > > > > > > >> >>>> > > everyone I found working on a KIP. If I
> missed
> >> >> anyone
> >> >> > > > in
> >> >> > > > > > the
> >> >> > > > > > > > >> invite
> >> >> > > > > > > > >> >>>> please
> >> >> > > > > > > > >> >>>> > > let me know and can update it, np.
> >> >> > > > > > > > >> >>>> > >
> >> >> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm
> Eastern
> >> >> Time.
> >> >> > > > Maybe
> >> >> > > > > > we
> >> >> > > > > > > > can
> >> >> > > > > > > > >> get
> >> >> > > > > > > > >> >>>> INFRA
> >> >> > > > > > > > >> >>>> > > help to make a google account so we can
> manage
> >> >> > > better?
> >> >> > > > > > > > >> >>>> > >
> >> >> > > > > > > > >> >>>> > > To discuss
> >> >> > > > > > > > >> >>>> > >
> >> >> > > > > > > > >> >>>>
> >> >> > > > > > > > >>
> >> >> > > > > > > >
> >> >> > > > > > >
> >> >> > > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >> >> > > > > > > > >> >>>> > > in progress and related JIRA that are
> >> >> interdependent
> >> >> > > > and
> >> >> > > > > > > common
> >> >> > > > > > > > >> work.
> >> >> > > > > > > > >> >>>> > >
> >> >> > > > > > > > >> >>>> > > ~ Joe Stein
> >> >> > > > > > > > >> >>>> > >
> >> >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps
> <
> >> >> > > > > > > > jay.kreps@gmail.com>
> >> >> > > > > > > > >> >>>> wrote:
> >> >> > > > > > > > >> >>>> > >
> >> >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that will
> also
> >> >> record
> >> >> > > > and
> >> >> > > > > > make
> >> >> > > > > > > > the
> >> >> > > > > > > > >> >>>> sessions
> >> >> > > > > > > > >> >>>> > >> available on youtube.
> >> >> > > > > > > > >> >>>> > >>
> >> >> > > > > > > > >> >>>> > >> -Jay
> >> >> > > > > > > > >> >>>> > >>
> >> >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff
> >> Holoman
> >> >> <
> >> >> > > > > > > > >> >>>> jholoman@cloudera.com>
> >> >> > > > > > > > >> >>>> > >> wrote:
> >> >> > > > > > > > >> >>>> > >>
> >> >> > > > > > > > >> >>>> > >> > Jay / Joe
> >> >> > > > > > > > >> >>>> > >> >
> >> >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for this
> >> >> purpose.
> >> >> > > We
> >> >> > > > > > could
> >> >> > > > > > > > >> record
> >> >> > > > > > > > >> >>>> the
> >> >> > > > > > > > >> >>>> > >> > sessions if there is interest and
> publish
> >> >> them
> >> >> > > out.
> >> >> > > > > > > > >> >>>> > >> >
> >> >> > > > > > > > >> >>>> > >> > Thanks
> >> >> > > > > > > > >> >>>> > >> >
> >> >> > > > > > > > >> >>>> > >> > Jeff
> >> >> > > > > > > > >> >>>> > >> >
> >> >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay
> >> Kreps <
> >> >> > > > > > > > >> jay.kreps@gmail.com>
> >> >> > > > > > > > >> >>>> wrote:
> >> >> > > > > > > > >> >>>> > >> >
> >> >> > > > > > > > >> >>>> > >> > > Let's try to get the technical
> hang-ups
> >> >> sorted
> >> >> > > > out,
> >> >> > > > > > > > though.
> >> >> > > > > > > > >> I
> >> >> > > > > > > > >> >>>> really
> >> >> > > > > > > > >> >>>> > >> > think
> >> >> > > > > > > > >> >>>> > >> > > there is some benefit to live
> discussion
> >> vs
> >> >> > > > > writing. I
> >> >> > > > > > > am
> >> >> > > > > > > > >> >>>> hopeful that
> >> >> > > > > > > > >> >>>> > >> if
> >> >> > > > > > > > >> >>>> > >> > > we post instructions and give
> ourselves a
> >> >> few
> >> >> > > > > attempts
> >> >> > > > > > > we
> >> >> > > > > > > > >> can
> >> >> > > > > > > > >> >>>> get it
> >> >> > > > > > > > >> >>>> > >> > > working.
> >> >> > > > > > > > >> >>>> > >> > >
> >> >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work for
> >> >> me...any
> >> >> > > > > > objections?
> >> >> > > > > > > > >> >>>> > >> > >
> >> >> > > > > > > > >> >>>> > >> > > -Jay
> >> >> > > > > > > > >> >>>> > >> > >
> >> >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe
> >> Stein
> >> >> <
> >> >> > > > > > > > >> joe.stein@stealth.ly
> >> >> > > > > > > > >> >>>> >
> >> >> > > > > > > > >> >>>> > >> wrote:
> >> >> > > > > > > > >> >>>> > >> > >
> >> >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe like
> every
> >> >> > > Tuesday ~
> >> >> > > > > 1pm
> >> >> > > > > > > ET
> >> >> > > > > > > > /
> >> >> > > > > > > > >> 10am
> >> >> > > > > > > > >> >>>> PT
> >> >> > > > > > > > >> >>>> > >> ????
> >> >> > > > > > > > >> >>>> > >> > > >
> >> >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but
> there
> >> is
> >> >> > > always
> >> >> > > > > some
> >> >> > > > > > > > >> issue or
> >> >> > > > > > > > >> >>>> > >> whatever
> >> >> > > > > > > > >> >>>> > >> > so
> >> >> > > > > > > > >> >>>> > >> > > > we know the apache irc channel
> works.
> >> We
> >> >> can
> >> >> > > > start
> >> >> > > > > > > there
> >> >> > > > > > > > >> and
> >> >> > > > > > > > >> >>>> see how
> >> >> > > > > > > > >> >>>> > >> it
> >> >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too
> and
> >> >> > > associate
> >> >> > > > to
> >> >> > > > > > > > >> tickets if
> >> >> > > > > > > > >> >>>> need be
> >> >> > > > > > > > >> >>>> > >> > > makes
> >> >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> >> >> > > > > > > > >> >>>> > >> > > >
> >> >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> >> >> > > > > > > > >> >>>> > >> > > >
> >> >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM,
> Jay
> >> >> Kreps <
> >> >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> >> >> > > > > > > > >> >>>> > >> > wrote:
> >> >> > > > > > > > >> >>>> > >> > > >
> >> >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a Google
> >> >> Hangout to
> >> >> > > > chat
> >> >> > > > > > > about
> >> >> > > > > > > > >> this.
> >> >> > > > > > > > >> >>>> What
> >> >> > > > > > > > >> >>>> > >> > about
> >> >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
> >> further...I
> >> >> > > > actually
> >> >> > > > > > > think
> >> >> > > > > > > > it
> >> >> > > > > > > > >> >>>> would be
> >> >> > > > > > > > >> >>>> > >> > good
> >> >> > > > > > > > >> >>>> > >> > > > for
> >> >> > > > > > > > >> >>>> > >> > > > > everyone spending a reasonable
> chunk
> >> of
> >> >> > > their
> >> >> > > > > week
> >> >> > > > > > > on
> >> >> > > > > > > > >> Kafka
> >> >> > > > > > > > >> >>>> stuff
> >> >> > > > > > > > >> >>>> > >> to
> >> >> > > > > > > > >> >>>> > >> > > > maybe
> >> >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think we
> could
> >> >> use
> >> >> > > time
> >> >> > > > > to
> >> >> > > > > > > talk
> >> >> > > > > > > > >> >>>> through
> >> >> > > > > > > > >> >>>> > >> design
> >> >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of
> >> code
> >> >> > > > reviews,
> >> >> > > > > > talk
> >> >> > > > > > > > >> through
> >> >> > > > > > > > >> >>>> any
> >> >> > > > > > > > >> >>>> > >> > tricky
> >> >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> >> >> > > > > > > > >> >>>> > >> > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > We can make it publicly available
> so
> >> >> that
> >> >> > > any
> >> >> > > > > one
> >> >> > > > > > > can
> >> >> > > > > > > > >> follow
> >> >> > > > > > > > >> >>>> along
> >> >> > > > > > > > >> >>>> > >> > who
> >> >> > > > > > > > >> >>>> > >> > > > > likes.
> >> >> > > > > > > > >> >>>> > >> > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this? If so
> >> I'll
> >> >> try
> >> >> > > to
> >> >> > > > > set
> >> >> > > > > > it
> >> >> > > > > > > > up
> >> >> > > > > > > > >> >>>> starting
> >> >> > > > > > > > >> >>>> > >> next
> >> >> > > > > > > > >> >>>> > >> > > > week.
> >> >> > > > > > > > >> >>>> > >> > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > -Jay
> >> >> > > > > > > > >> >>>> > >> > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM,
> >> Andrii
> >> >> > > > > Biletskyi
> >> >> > > > > > <
> >> >> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly>
> wrote:
> >> >> > > > > > > > >> >>>> > >> > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> >> >> > > > > > > > >> >>>> > >> > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed /
> >> >> aligned
> >> >> > > > > document
> >> >> > > > > > > > >> structure.
> >> >> > > > > > > > >> >>>> Also I
> >> >> > > > > > > > >> >>>> > >> > > added
> >> >> > > > > > > > >> >>>> > >> > > > > > some
> >> >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
> >> >> AdminClient so
> >> >> > > we
> >> >> > > > > have
> >> >> > > > > > > > >> something
> >> >> > > > > > > > >> >>>> to
> >> >> > > > > > > > >> >>>> > >> start
> >> >> > > > > > > > >> >>>> > >> > > > from
> >> >> > > > > > > > >> >>>> > >> > > > > > while
> >> >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> >> >> > > > > > > > >> >>>> > >> > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > >
> >> >> > > > > > > > >> >>>> > >> > > >
> >> >> > > > > > > > >> >>>> > >> > >
> >> >> > > > > > > > >> >>>> > >> >
> >> >> > > > > > > > >> >>>> > >>
> >> >> > > > > > > > >> >>>>
> >> >> > > > > > > > >>
> >> >> > > > > > > >
> >> >> > > > > > >
> >> >> > > > > >
> >> >> > > > >
> >> >> > > >
> >> >> > >
> >> >>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> >> > > > > > > > >> >>>> > >> > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> >> >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> >> >> > > > > > > > >> >>>> > >> > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM,
> >> >> Andrii
> >> >> > > > > > Biletskyi
> >> >> > > > > > > <
> >> >> > > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly>
> >> wrote:
> >> >> > > > > > > > >> >>>> > >> > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> >> >> > > > > > > > >> >>>> > >> > > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you are
> right,
> >> >> in
> >> >> > > most
> >> >> > > > > > cases
> >> >> > > > > > > > >> client
> >> >> > > > > > > > >> >>>> will
> >> >> > > > > > > > >> >>>> > >> have
> >> >> > > > > > > > >> >>>> > >> > > > enough
> >> >> > > > > > > > >> >>>> > >> > > > > > > context to show descriptive
> error
> >> >> > > message.
> >> >> > > > > My
> >> >> > > > > > > > >> concern is
> >> >> > > > > > > > >> >>>> that
> >> >> > > > > > > > >> >>>> > >> we
> >> >> > > > > > > > >> >>>> > >> > > will
> >> >> > > > > > > > >> >>>> > >> > > > > > have
> >> >> > > > > > > > >> >>>> > >> > > > > > > to
> >> >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error codes
> for
> >> >> each
> >> >> > > > > possible
> >> >> > > > > > > > >> error. Of
> >> >> > > > > > > > >> >>>> course,
> >> >> > > > > > > > >> >>>> > >> > we
> >> >> > > > > > > > >> >>>> > >> > > > > could
> >> >> > > > > > > > >> >>>> > >> > > > > > > reuse
> >> >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
> >> >> > > > > > > UknownTopicOrPartitionCode,
> >> >> > > > > > > > >> but we
> >> >> > > > > > > > >> >>>> will
> >> >> > > > > > > > >> >>>> > >> > also
> >> >> > > > > > > > >> >>>> > >> > > > need
> >> >> > > > > > > > >> >>>> > >> > > > > > to
> >> >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> >> >> TopicAlreadyExistsCode,
> >> >> > > > > > > > >> >>>> TopicConfigInvalid (both
> >> >> > > > > > > > >> >>>> > >> > for
> >> >> > > > > > > > >> >>>> > >> > > > > topic
> >> >> > > > > > > > >> >>>> > >> > > > > > > name and config, and probably
> >> user
> >> >> would
> >> >> > > > > like
> >> >> > > > > > to
> >> >> > > > > > > > >> know
> >> >> > > > > > > > >> >>>> what
> >> >> > > > > > > > >> >>>> > >> > exactly
> >> >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
> >> >> > > > > > > InvalidReplicaAssignment,
> >> >> > > > > > > > >> >>>> > >> InternalError
> >> >> > > > > > > > >> >>>> > >> > > > (e.g.
> >> >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> >> >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
> >> TopicCommand,
> >> >> we
> >> >> > > will
> >> >> > > > > > also
> >> >> > > > > > > > >> need to
> >> >> > > > > > > > >> >>>> add
> >> >> > > > > > > > >> >>>> > >> > similar
> >> >> > > > > > > > >> >>>> > >> > > > > stuff
> >> >> > > > > > > > >> >>>> > >> > > > > > > for
> >> >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> >> >> PreferredReplica. So
> >> >> > > > > we'll
> >> >> > > > > > > end
> >> >> > > > > > > > >> up
> >> >> > > > > > > > >> >>>> with a
> >> >> > > > > > > > >> >>>> > >> > large
> >> >> > > > > > > > >> >>>> > >> > > > list
> >> >> > > > > > > > >> >>>> > >> > > > > > of
> >> >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in
> Admin
> >> >> > > protocol.
> >> >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree my
> >> >> proposal is
> >> >> > > > not
> >> >> > > > > > > > >> consistent
> >> >> > > > > > > > >> >>>> with
> >> >> > > > > > > > >> >>>> > >> > other
> >> >> > > > > > > > >> >>>> > >> > > > > cases.
> >> >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better
> solution
> >> >> or
> >> >> > > > > something
> >> >> > > > > > > > >> >>>> in-between.
> >> >> > > > > > > > >> >>>> > >> > > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it
> is a
> >> >> great
> >> >> > > > idea.
> >> >> > > > > > > This
> >> >> > > > > > > > >> way we
> >> >> > > > > > > > >> >>>> can
> >> >> > > > > > > > >> >>>> > >> move
> >> >> > > > > > > > >> >>>> > >> > > on
> >> >> > > > > > > > >> >>>> > >> > > > > > > faster.
> >> >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on
> date/time
> >> so
> >> >> > > people
> >> >> > > > > can
> >> >> > > > > > > > join.
> >> >> > > > > > > > >> >>>> Will work
> >> >> > > > > > > > >> >>>> > >> > for
> >> >> > > > > > > > >> >>>> > >> > > me
> >> >> > > > > > > > >> >>>> > >> > > > > > this
> >> >> > > > > > > > >> >>>> > >> > > > > > > and
> >> >> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime if
> >> agreed
> >> >> in
> >> >> > > > > advance.
> >> >> > > > > > > > >> >>>> > >> > > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> >> >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> >> >> > > > > > > > >> >>>> > >> > > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09
> PM,
> >> >> Jay
> >> >> > > > Kreps <
> >> >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> >> >> > > > > > > > >> >>>> > >> > > > > wrote:
> >> >> > > > > > > > >> >>>> > >> > > > > > >
> >> >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> >> >> > > > > > > > >> >>>> > >> > > > > > >>
> >> >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good
> error
> >> >> handling
> >> >> > > > > > without
> >> >> > > > > > > > >> needing
> >> >> > > > > > > > >> >>>> custom
> >> >> > > > > > > > >> >>>> > >> > > > > > server-side
> >> >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the
> >> >> client has
> >> >> > > > the
> >> >> > > > > > > > >> context to
> >> >> > > > > > > > >> >>>> know
> >> >> > > > > > > > >> >>>> > >> that
> >> >> > > > > > > > >> >>>> > >> > > if
> >> >> > > > > > > > >> >>>> > >> > > > it
> >> >> > > > > > > > >> >>>> > >> > > > > > got
> >> >> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic
> doesn't
> >> >> exist
> >> >> > > to
> >> >> > > > > say
> >> >> > > > > > > > >> "Topic X
> >> >> > > > > > > > >> >>>> doesn't
> >> >> > > > > > > > >> >>>> > >> > > exist"
> >> >> > > > > > > > >> >>>> > >> > > > > > >> rather
> >> >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or
> >> >> whatever).
> >> >> > > Maybe
> >> >> > > > > > there
> >> >> > > > > > > > are
> >> >> > > > > > > > >> >>>> specific
> >> >> > > > > > > > >> >>>> > >> > cases
> >> >> > > > > > > > >> >>>> > >> > > > > where
> >> >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to
> add
> >> >> > > > server-side
> >> >> > > > > > > error
> >> >> > > > > > > > >> >>>> messages we
> >> >> > > > > > > > >> >>>> > >> > > really
> >> >> > > > > > > > >> >>>> > >> > > > > do
> >> >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a
> consistent
> >> >> way
> >> >> > > > across
> >> >> > > > > > the
> >> >> > > > > > > > >> protocol.
> >> >> > > > > > > > >> >>>> > >> > > > > > >>
> >> >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open
> >> >> questions
> >> >> > > > here
> >> >> > > > > > > from
> >> >> > > > > > > > my
> >> >> > > > > > > > >> >>>> previous
> >> >> > > > > > > > >> >>>> > >> > > list. I
> >> >> > > > > > > > >> >>>> > >> > > > > > will
> >> >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few days
> for
> >> >> Strata
> >> >> > > > > > though.
> >> >> > > > > > > > >> Maybe
> >> >> > > > > > > > >> >>>> we could
> >> >> > > > > > > > >> >>>> > >> > do
> >> >> > > > > > > > >> >>>> > >> > > a
> >> >> > > > > > > > >> >>>> > >> > > > > > Google
> >> >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open
> issues
> >> >> some
> >> >> > > time
> >> >> > > > > > > towards
> >> >> > > > > > > > >> the
> >> >> > > > > > > > >> >>>> end of
> >> >> > > > > > > > >> >>>> > >> > next
> >> >> > > > > > > > >> >>>> > >> > > > week
> >> >> > > > > > > > >> >>>> > >> > > > > > for
> >> >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in this
> >> ticket?
> >> >> I
> >> >> > > have
> >> >> > > > a
> >> >> > > > > > > > feeling
> >> >> > > > > > > > >> that
> >> >> > > > > > > > >> >>>> might
> >> >> > > > > > > > >> >>>> > >> > > > progress
> >> >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster than
> >> >> email--I
> >> >> > > > think
> >> >> > > > > we
> >> >> > > > > > > > >> could talk
> >> >> > > > > > > > >> >>>> > >> through
> >> >> > > > > > > > >> >>>> > >> > > > those
> >> >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly
> >> >> quickly...
> >> >> > > > > > > > >> >>>> > >> > > > > > >>
> >> >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> >> >> > > > > > > > >> >>>> > >> > > > > > >>
> >> >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27
> AM,
> >> >> Andrii
> >> >> > > > > > > > Biletskyi <
> >> >> > > > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly>
> >> >> wrote:
> >> >> > > > > > > > >> >>>> > >> > > > > > >>
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> >> >> > > > > > > > >> >>>> > >> > > > > > >> >
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address some
> of
> >> >> the
> >> >> > > > issues
> >> >> > > > > > > which
> >> >> > > > > > > > >> were
> >> >> > > > > > > > >> >>>> > >> mentioned
> >> >> > > > > > > > >> >>>> > >> > > > > earlier
> >> >> > > > > > > > >> >>>> > >> > > > > > >> about
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of
> >> >> those was
> >> >> > > > > about
> >> >> > > > > > > > >> batching
> >> >> > > > > > > > >> >>>> > >> > operations.
> >> >> > > > > > > > >> >>>> > >> > > > What
> >> >> > > > > > > > >> >>>> > >> > > > > > if
> >> >> > > > > > > > >> >>>> > >> > > > > > >> we
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand
> approach
> >> >> and let
> >> >> > > > > people
> >> >> > > > > > > > >> specify
> >> >> > > > > > > > >> >>>> > >> topic-name
> >> >> > > > > > > > >> >>>> > >> > > by
> >> >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most of
> the
> >> use
> >> >> > > cases?
> >> >> > > > > > > > >> >>>> > >> > > > > > >> >
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what
> information
> >> >> should
> >> >> > > we
> >> >> > > > > > > > generally
> >> >> > > > > > > > >> >>>> provide in
> >> >> > > > > > > > >> >>>> > >> > > Admin
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin
> commands
> >> >> don't
> >> >> > > > imply
> >> >> > > > > > > they
> >> >> > > > > > > > >> will
> >> >> > > > > > > > >> >>>> be used
> >> >> > > > > > > > >> >>>> > >> > only
> >> >> > > > > > > > >> >>>> > >> > > > in
> >> >> > > > > > > > >> >>>> > >> > > > > > CLI
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > but,
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a
> very
> >> >> > > important
> >> >> > > > > > > client
> >> >> > > > > > > > >> of this
> >> >> > > > > > > > >> >>>> > >> > feature.
> >> >> > > > > > > > >> >>>> > >> > > In
> >> >> > > > > > > > >> >>>> > >> > > > > > this
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > case,
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would
> like
> >> to
> >> >> > > provide
> >> >> > > > > > users
> >> >> > > > > > > > >> with
> >> >> > > > > > > > >> >>>> rich
> >> >> > > > > > > > >> >>>> > >> > > experience
> >> >> > > > > > > > >> >>>> > >> > > > > in
> >> >> > > > > > > > >> >>>> > >> > > > > > >> terms
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > of
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > getting results / errors of
> >> the
> >> >> > > > executed
> >> >> > > > > > > > >> commands.
> >> >> > > > > > > > >> >>>> Usually
> >> >> > > > > > > > >> >>>> > >> we
> >> >> > > > > > > > >> >>>> > >> > > > supply
> >> >> > > > > > > > >> >>>> > >> > > > > > >> with
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > responses only errorCode,
> >> which
> >> >> looks
> >> >> > > > > very
> >> >> > > > > > > > >> limiting,
> >> >> > > > > > > > >> >>>> in case
> >> >> > > > > > > > >> >>>> > >> > of
> >> >> > > > > > > > >> >>>> > >> > > > CLI
> >> >> > > > > > > > >> >>>> > >> > > > > we
> >> >> > > > > > > > >> >>>> > >> > > > > > >> may
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human
> readable
> >> >> error
> >> >> > > > > > > description.
> >> >> > > > > > > > >> >>>> > >> > > > > > >> >
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into account
> >> >> previous item
> >> >> > > > > about
> >> >> > > > > > > > >> batching,
> >> >> > > > > > > > >> >>>> what
> >> >> > > > > > > > >> >>>> > >> do
> >> >> > > > > > > > >> >>>> > >> > > you
> >> >> > > > > > > > >> >>>> > >> > > > > > think
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > about
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> >> >> > > > > > > > >> >>>> > >> > > > > > >> >
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support
> >> >> regexp)
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest =>
> >> TopicName
> >> >> > > > > Partitions
> >> >> > > > > > > > >> Replicas
> >> >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse =>
> >> ErrorCode
> >> >> > > > > > > > ErrorDescription
> >> >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> >> >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription =>
> string
> >> >> (empty
> >> >> > > if
> >> >> > > > > > > > >> successful)
> >> >> > > > > > > > >> >>>> > >> > > > > > >> >
> >> >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
> >> >> TopicNameRegexp
> >> >> > > >
> >> >
> >> > ...
> >> >
> >> > [Message clipped]
> >>
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Gwen Shapira <gs...@cloudera.com>.
On Wed, Mar 18, 2015 at 9:34 AM, Jun Rao <ju...@confluent.io> wrote:
> Andri,
>
> Thanks for the summary.
>
> 1. I just realized that in order to start working on KAFKA-1927, we will
> need to merge the changes to OffsetCommitRequest (from 0.8.2) to trunk.
> This is planned to be done as part of KAFKA-1634. So, we will need Guozhang
> and Joel's help to wrap this up.

I mentioned this in a separate thread, but it may be more relevant here:
It looks like the SimpleConsumer API exposes TopicMetadataRequest and
TopicMetadataResponse.
This means that KAFKA-1927 doesn't remove this duplication.

So I'm not sure we actually need KAFKA-1927 before implementing this KIP.
This doesn't mean I'm stopping work on KAFKA-1927, but perhaps it
means we can proceed in parallel?

> 2. Thinking about this a bit more, if the semantic of those "write"
> requests is async (i.e., after the client gets a response, it just means
> that the operation is initiated, but not necessarily completed), we don't
> really need to forward the requests to the controller. Instead, the
> receiving broker can just write the operation to ZK as the admin command
> line tool previously does. This will simplify the implementation.
>
> 8. There is another implementation detail for describe topic. Ideally, we
> want to read the topic config from the broker cache, instead of ZooKeeper.
> Currently, every broker reads the topic-level config for all topics.
> However, it ignores those for topics not hosted on itself. So, we may need
> to change TopicConfigManager a bit so that it caches the configs for all
> topics.
>
> Thanks,
>
> Jun
>
>
> On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
>> Guys,
>>
>> Thanks for a great discussion!
>> Here are the actions points:
>>
>> 1. Q: Get rid of all scala requests objects, use java protocol definitions.
>>     A: Gwen kindly took that (KAFKA-1927). It's important to speed up
>> review procedure
>>          there since this ticket blocks other important changes.
>>
>> 2. Q: Generic re-reroute facility vs client maintaining cluster state.
>>     A: Jay has added pseudo code to KAFKA-1912 - need to consider whether
>> this will be
>>         easy to implement as a server-side feature (comments are
>> welcomed!).
>>
>> 3. Q: Controller field in wire protocol.
>>     A: This might be useful for clients, add this to TopicMetadataResponse
>> (already in KIP).
>>
>> 4. Q: Decoupling topic creation from TMR.
>>     A: I will add proposed by Jun solution (using clientId for that) to the
>> KIP.
>>
>> 5. Q: Bumping new versions of TMR vs grabbing all protocol changes in one
>> version.
>>     A: It was decided to try to gather all changes to protocol (before
>> release).
>>         In case of TMR it worth checking: KAFKA-2020 and KIP-13 (quotas)
>>
>> 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
>>     A: Use jackson for that, /tools project is a separate jar so shouldn't
>> be a big deal.
>>
>> 7.  Q: VerifyReassingPartitions vs generic status check command.
>>      A: For long-running requests like reassign partitions *progress* check
>> request is useful,
>>          it makes sense to introduce it.
>>
>>  Please add, correct me if I missed something.
>>
>> Thanks,
>> Andrii Biletskyi
>>
>> On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
>> andrii.biletskyi@stealth.ly> wrote:
>>
>> > Joel,
>> >
>> > You are right, I removed ClusterMetadata because we have partially
>> > what we need in TopicMetadata. Also, as Jay pointed out earlier, we
>> > would like to have "orthogonal" API, but at the same time we need
>> > to be backward compatible.
>> >
>> > But I like your idea and even have some other arguments for this option:
>> > There is also DescribeTopicRequest which was proposed in this KIP,
>> > it returns topic configs, partitions, replication factor plus partition
>> > ISR, ASR,
>> > leader replica. The later part is really already there in
>> > TopicMetadataRequest.
>> > So again we'll have to add stuff to TMR, not to duplicate some info in
>> > newly added requests. However, this way we'll end up with "monster"
>> > request which returns cluster metadata, topic replication and config info
>> > plus partition replication data. Seems logical to split TMR to
>> > - ClusterMetadata (brokers + controller, maybe smth else)
>> > - TopicMetadata (topic info + partition details)
>> > But since current TMR is involved in lots of places (including network
>> > client,
>> > as I understand) this might be very serious change and it probably makes
>> > sense to stick with current approach.
>> >
>> > Thanks,
>> > Andrii Biletskyi
>> >
>> >
>> > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jj...@gmail.com> wrote:
>> >
>> >> I may be missing some context but hopefully this will also be covered
>> >> today: I thought the earlier proposal where there was an explicit
>> >> ClusterMetadata request was clearer and explicit. During the course of
>> >> this thread I think the conclusion was that the main need was for
>> >> controller information and that can be rolled into the topic metadata
>> >> response but that seems a bit irrelevant to topic metadata. FWIW I
>> >> think the full broker-list is also irrelevant to topic metadata, but
>> >> it is already there and in use. I think there is still room for an
>> >> explicit ClusterMetadata request since there may be other
>> >> cluster-level information that we may want to add over time (and that
>> >> have nothing to do with topic metadata).
>> >>
>> >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
>> >> > Jun,
>> >> >
>> >> > 101. Okay, if you say that such use case is important. I also think
>> >> > using clientId for these purposes is fine - if we already have this
>> >> field
>> >> > as part of all Wire protocol messages, why not use that.
>> >> > I will update KIP-4 page if nobody has other ideas (which may come up
>> >> > during the call today).
>> >> >
>> >> > 102.1 Agree, I'll update the KIP accordingly. I think we can add new,
>> >> > fine-grained error codes if some error code received in specific case
>> >> > won't give enough context to return a descriptive error message for
>> >> user.
>> >> >
>> >> > Look forward to discussing all outstanding issues in detail today
>> during
>> >> > the call.
>> >> >
>> >> > Thanks,
>> >> > Andrii Biletskyi
>> >> >
>> >> >
>> >> >
>> >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io> wrote:
>> >> >
>> >> > > 101. There may be a use case where you only want the topics to be
>> >> created
>> >> > > manually by admins. Currently, you can do that by disabling auto
>> topic
>> >> > > creation and issue topic creation from the TopicCommand. If we
>> >> disable auto
>> >> > > topic creation completely on the broker and don't have a way to
>> >> distinguish
>> >> > > between topic creation requests from the regular clients and the
>> >> admin, we
>> >> > > can't support manual topic creation any more. I was thinking that
>> >> another
>> >> > > way of distinguishing the clients making the topic creation requests
>> >> is
>> >> > > using clientId. For example, the admin tool can set it to something
>> >> like
>> >> > > admin and the broker can treat that clientId specially.
>> >> > >
>> >> > > Also, there is a related discussion in KAFKA-2020. Currently, we do
>> >> the
>> >> > > following in TopicMetadataResponse:
>> >> > >
>> >> > > 1. If leader is not available, we set the partition level error code
>> >> to
>> >> > > LeaderNotAvailable.
>> >> > > 2. If a non-leader replica is not available, we take that replica
>> out
>> >> of
>> >> > > the assigned replica list and isr in the response. As an indication
>> >> for
>> >> > > doing that, we set the partition level error code to
>> >> ReplicaNotAvailable.
>> >> > >
>> >> > > This has a few problems. First, ReplicaNotAvailable probably
>> >> shouldn't be
>> >> > > an error, at least for the normal producer/consumer clients that
>> just
>> >> want
>> >> > > to find out the leader. Second, it can happen that both the leader
>> and
>> >> > > another replica are not available at the same time. There is no
>> error
>> >> code
>> >> > > to indicate both. Third, even if a replica is not available, it's
>> >> still
>> >> > > useful to return its replica id since some clients (e.g. admin tool)
>> >> may
>> >> > > still make use of it.
>> >> > >
>> >> > > One way to address this issue is to always return the replica id for
>> >> > > leader, assigned replicas, and isr regardless of whether the
>> >> corresponding
>> >> > > broker is live or not. Since we also return the list of live
>> brokers,
>> >> the
>> >> > > client can figure out whether a leader or a replica is live or not
>> >> and act
>> >> > > accordingly. This way, we don't need to set the partition level
>> error
>> >> code
>> >> > > when the leader or a replica is not available. This doesn't change
>> >> the wire
>> >> > > protocol, but does change the semantics. Since we are evolving the
>> >> protocol
>> >> > > of TopicMetadataRequest here, we can potentially piggyback the
>> change.
>> >> > >
>> >> > > 102.1 For those types of errors due to invalid input, shouldn't we
>> >> just
>> >> > > guard it at parameter validation time and throw
>> >> InvalidArgumentException
>> >> > > without even sending the request to the broker?
>> >> > >
>> >> > > Thanks,
>> >> > >
>> >> > > Jun
>> >> > >
>> >> > >
>> >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
>> >> > > andrii.biletskyi@stealth.ly> wrote:
>> >> > >
>> >> > > > Jun,
>> >> > > >
>> >> > > > Answering your questions:
>> >> > > >
>> >> > > > 101. If I understand you correctly, you are saying future producer
>> >> > > versions
>> >> > > > (which
>> >> > > > will be ported to TMR_V1) won't be able to automatically create
>> >> topic (if
>> >> > > > we
>> >> > > > unconditionally remove topic creation from there). But we need to
>> >> this
>> >> > > > preserve logic.
>> >> > > > Ok, about your proposal: I'm not a big fan too, when it comes to
>> >> > > > differentiating
>> >> > > > clients directly in protocol schema. And also I'm not sure I
>> >> understand
>> >> > > at
>> >> > > > all why
>> >> > > > auto.create.topics.enable is a server side configuration. Can we
>> >> > > deprecate
>> >> > > > this setting
>> >> > > > in future versions, add this setting to producer and based on that
>> >> upon
>> >> > > > receiving
>> >> > > > UnknownTopic create topic explicitly by a separate producer call
>> via
>> >> > > > adminClient?
>> >> > > >
>> >> > > > 102.1. Hm, yes. It's because we want to support batching and at
>> the
>> >> same
>> >> > > > time we
>> >> > > > want to give descriptive error messages for clients. Since
>> >> AdminClient
>> >> > > > holds the context
>> >> > > > to construct such messages (e.g. AdminClient layer can know that
>> >> > > > InvalidArgumentsCode
>> >> > > > means two cases: either invalid number - e.g. -1; or
>> >> replication-factor
>> >> > > was
>> >> > > > provided while
>> >> > > > partitions argument wasn't) - I wrapped responses in Exceptions.
>> >> But I'm
>> >> > > > open to any
>> >> > > > other ideas, this was just initial version.
>> >> > > > 102.2. Yes, I agree. I'll change that to probably some other dto.
>> >> > > >
>> >> > > > Thanks,
>> >> > > > Andrii Biletskyi
>> >> > > >
>> >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io>
>> wrote:
>> >> > > >
>> >> > > > > Andrii,
>> >> > > > >
>> >> > > > > 101. That's what I was thinking too, but it may not be that
>> >> simple. In
>> >> > > > > TopicMetadataRequest_V1,
>> >> > > > > we can let it not trigger auto topic creation. Then, in the
>> >> producer
>> >> > > > side,
>> >> > > > > if it gets an UnknownTopicException, it can explicitly issue a
>> >> > > > > createTopicRequest for auto topic creation. On the consumer
>> side,
>> >> it
>> >> > > will
>> >> > > > > never issue createTopicRequest. This works when auto topic
>> >> creation is
>> >> > > > > enabled on the broker side. However, I am not sure how things
>> >> will work
>> >> > > > > when auto topic creation is disabled on the broker side. In this
>> >> case,
>> >> > > we
>> >> > > > > want to have a way to manually create a topic, potentially
>> through
>> >> > > admin
>> >> > > > > commands. However, then we need a way to distinguish
>> >> createTopicRequest
>> >> > > > > issued from the producer clients and the admin tools. May be we
>> >> can
>> >> > > add a
>> >> > > > > new field in createTopicRequest and set it differently in the
>> >> producer
>> >> > > > > client and the admin client. However, I am not sure if that's
>> the
>> >> best
>> >> > > > > approach.
>> >> > > > >
>> >> > > > > 2. Yes, refactoring existing requests is a non-trivial amount of
>> >> work.
>> >> > > I
>> >> > > > > posted some comments in KAFKA-1927. We will probably have to fix
>> >> > > > KAFKA-1927
>> >> > > > > first, before adding the new logic in KAFKA-1694. Otherwise, the
>> >> > > changes
>> >> > > > > will be too big.
>> >> > > > >
>> >> > > > > 102. About the AdminClient:
>> >> > > > > 102.1. It's a bit weird that we return exception in the api. It
>> >> seems
>> >> > > > that
>> >> > > > > we should either return error code or throw an exception when
>> >> getting
>> >> > > the
>> >> > > > > response state.
>> >> > > > > 102.2. We probably shouldn't explicitly use the request object
>> in
>> >> the
>> >> > > > api.
>> >> > > > > Not every request evolution requires an api change.
>> >> > > > >
>> >> > > > > Thanks,
>> >> > > > >
>> >> > > > > Jun
>> >> > > > >
>> >> > > > >
>> >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
>> >> > > > > andrii.biletskyi@stealth.ly> wrote:
>> >> > > > >
>> >> > > > > > Jun,
>> >> > > > > >
>> >> > > > > > Thanks for you comments. Answers inline:
>> >> > > > > >
>> >> > > > > > 100. There are a few fields such as ReplicaAssignment,
>> >> > > > > > > ReassignPartitionRequest,
>> >> > > > > > > and PartitionsSerialized that are represented as a string,
>> but
>> >> > > > contain
>> >> > > > > > > composite structures in json. Could we flatten them out
>> >> directly in
>> >> > > > the
>> >> > > > > > > protocol definition as arrays/records?
>> >> > > > > >
>> >> > > > > >
>> >> > > > > > Yes, now with Admin Client this looks a bit weird. My initial
>> >> > > > motivation
>> >> > > > > > was:
>> >> > > > > > ReassignPartitionCommand accepts input in json, we want to
>> >> remain
>> >> > > > tools'
>> >> > > > > > interfaces unchanged, where possible.
>> >> > > > > > If we port it to deserialized format, in CLI (/tools project)
>> >> we will
>> >> > > > > have
>> >> > > > > > to add some
>> >> > > > > > json library since /tools is written in java and we'll need to
>> >> > > > > deserialize
>> >> > > > > > json file
>> >> > > > > > provided by a user. Can we quickly agree on what this library
>> >> should
>> >> > > be
>> >> > > > > > (Jackson, GSON, whatever)?
>> >> > > > > >
>> >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
>> >> creation?
>> >> > > > This
>> >> > > > > > > will be a bit weird now that we have a separate topic
>> >> creation api.
>> >> > > > > Have
>> >> > > > > > > you thought about how the new createTopicRequest and
>> >> > > > > TopicMetadataRequest
>> >> > > > > > > v1 will be used in the producer/consumer client, in addition
>> >> to
>> >> > > admin
>> >> > > > > > > tools? For example, ideally, we don't want
>> >> TopicMetadataRequest
>> >> > > from
>> >> > > > > the
>> >> > > > > > > consumer to trigger auto topic creation.
>> >> > > > > >
>> >> > > > > >
>> >> > > > > > I agree, this strange logic should be fixed. I'm not confident
>> >> in
>> >> > > this
>> >> > > > > > Kafka part so
>> >> > > > > > correct me if I'm wrong, but it doesn't look like a hard thing
>> >> to
>> >> > > do, I
>> >> > > > > > think we can
>> >> > > > > > leverage AdminClient for that in Producer and unconditionally
>> >> remove
>> >> > > > > topic
>> >> > > > > > creation from the TopicMetadataRequest_V1.
>> >> > > > > >
>> >> > > > > > 2. I think Jay meant getting rid of scala classes
>> >> > > > > > > like HeartbeatRequestAndHeader and
>> >> HeartbeatResponseAndHeader. We
>> >> > > did
>> >> > > > > > that
>> >> > > > > > > as a stop-gap thing when adding the new requests for the
>> >> consumers.
>> >> > > > > > > However, the long term plan is to get rid of all those and
>> >> just
>> >> > > reuse
>> >> > > > > the
>> >> > > > > > > java request/response in the client. Since this KIP proposes
>> >> to
>> >> > > add a
>> >> > > > > > > significant number of new requests, perhaps we should bite
>> the
>> >> > > bullet
>> >> > > > > to
>> >> > > > > > > clean up the existing scala requests first before adding new
>> >> ones?
>> >> > > > > > >
>> >> > > > > >
>> >> > > > > > Yes, looks like I misunderstood the point of
>> >> ...RequestAndHeader.
>> >> > > > Okay, I
>> >> > > > > > will
>> >> > > > > > rework that. The only thing is that I don't see any example
>> how
>> >> it
>> >> > > was
>> >> > > > > done
>> >> > > > > > for at
>> >> > > > > > least one existing protocol message. Thus, as I understand, I
>> >> have to
>> >> > > > > think
>> >> > > > > > how we
>> >> > > > > > are going to do it.
>> >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
>> reasonable,
>> >> but
>> >> > > if
>> >> > > > > it's
>> >> > > > > > an *obligatory*
>> >> > > > > > requirement to have Admin KIP done, I'm afraid this can be a
>> >> serious
>> >> > > > > > blocker for us.
>> >> > > > > > There are 13 protocol messages and all that would require not
>> >> only
>> >> > > unit
>> >> > > > > > tests but quite
>> >> > > > > > intensive manual testing, no? I'm afraid I'm not the right guy
>> >> to
>> >> > > cover
>> >> > > > > > pretty much all
>> >> > > > > > Kafka core internals :). Let me know your thoughts on this
>> >> item. Btw
>> >> > > > > there
>> >> > > > > > is a ticket to
>> >> > > > > > follow-up this issue (
>> >> > > https://issues.apache.org/jira/browse/KAFKA-2006
>> >> > > > ).
>> >> > > > > >
>> >> > > > > > Thanks,
>> >> > > > > > Andrii Biletskyi
>> >> > > > > >
>> >> > > > > >
>> >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <ju...@confluent.io>
>> >> wrote:
>> >> > > > > >
>> >> > > > > > > Andrii,
>> >> > > > > > >
>> >> > > > > > >
>> >> > > > > > > A few more comments.
>> >> > > > > > >
>> >> > > > > > > 100. There are a few fields such as ReplicaAssignment,
>> >> > > > > > > ReassignPartitionRequest,
>> >> > > > > > > and PartitionsSerialized that are represented as a string,
>> but
>> >> > > > contain
>> >> > > > > > > composite structures in json. Could we flatten them out
>> >> directly in
>> >> > > > the
>> >> > > > > > > protocol definition as arrays/records?
>> >> > > > > > >
>> >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
>> >> > > creation?
>> >> > > > > This
>> >> > > > > > > will be a bit weird now that we have a separate topic
>> >> creation api.
>> >> > > > > Have
>> >> > > > > > > you thought about how the new createTopicRequest and
>> >> > > > > TopicMetadataRequest
>> >> > > > > > > v1 will be used in the producer/consumer client, in addition
>> >> to
>> >> > > admin
>> >> > > > > > > tools? For example, ideally, we don't want
>> >> TopicMetadataRequest
>> >> > > from
>> >> > > > > the
>> >> > > > > > > consumer to trigger auto topic creation.
>> >> > > > > > >
>> >> > > > > > > 2. I think Jay meant getting rid of scala classes
>> >> > > > > > > like HeartbeatRequestAndHeader and
>> >> HeartbeatResponseAndHeader. We
>> >> > > did
>> >> > > > > > that
>> >> > > > > > > as a stop-gap thing when adding the new requests for the
>> >> consumers.
>> >> > > > > > > However, the long term plan is to get rid of all those and
>> >> just
>> >> > > reuse
>> >> > > > > the
>> >> > > > > > > java request/response in the client. Since this KIP proposes
>> >> to
>> >> > > add a
>> >> > > > > > > significant number of new requests, perhaps we should bite
>> the
>> >> > > bullet
>> >> > > > > to
>> >> > > > > > > clean up the existing scala requests first before adding new
>> >> ones?
>> >> > > > > > >
>> >> > > > > > > Thanks,
>> >> > > > > > >
>> >> > > > > > > Jun
>> >> > > > > > >
>> >> > > > > > >
>> >> > > > > > >
>> >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
>> >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> >> > > > > > >
>> >> > > > > > > > Hi,
>> >> > > > > > > >
>> >> > > > > > > > As said above - I list again all comments from this thread
>> >> so we
>> >> > > > > > > > can see what's left and finalize all pending issues.
>> >> > > > > > > >
>> >> > > > > > > > Comments from Jay:
>> >> > > > > > > > 1. This is much needed functionality, but there are a lot
>> >> of the
>> >> > > so
>> >> > > > > > let's
>> >> > > > > > > > really think these protocols through. We really want to
>> end
>> >> up
>> >> > > > with a
>> >> > > > > > set
>> >> > > > > > > > of well thought-out, orthoganol apis. For this reason I
>> >> think it
>> >> > > is
>> >> > > > > > > really
>> >> > > > > > > > important to think through the end state even if that
>> >> includes
>> >> > > APIs
>> >> > > > > we
>> >> > > > > > > > won't implement in the first phase.
>> >> > > > > > > >
>> >> > > > > > > > A: Definitely behind this. Would appreciate if there are
>> >> concrete
>> >> > > > > > > comments
>> >> > > > > > > > how this can be improved.
>> >> > > > > > > >
>> >> > > > > > > > 2. Let's please please please wait until we have switched
>> >> the
>> >> > > > server
>> >> > > > > > over
>> >> > > > > > > > to the new java protocol definitions. If we add upteen
>> more
>> >> ad
>> >> > > hoc
>> >> > > > > > scala
>> >> > > > > > > > objects that is just generating more work for the
>> >> conversion we
>> >> > > > know
>> >> > > > > we
>> >> > > > > > > > have to do.
>> >> > > > > > > >
>> >> > > > > > > > A: Fixed in the latest patch - removed scala protocol
>> >> classes.
>> >> > > > > > > >
>> >> > > > > > > > 3. This proposal introduces a new type of optional
>> >> parameter.
>> >> > > This
>> >> > > > is
>> >> > > > > > > > inconsistent with everything else in the protocol where we
>> >> use -1
>> >> > > > or
>> >> > > > > > some
>> >> > > > > > > > other marker value. You could argue either way but let's
>> >> stick
>> >> > > with
>> >> > > > > > that
>> >> > > > > > > > for consistency. For clients that implemented the protocol
>> >> in a
>> >> > > > > better
>> >> > > > > > > way
>> >> > > > > > > > than our scala code these basic primitives are hard to
>> >> change.
>> >> > > > > > > >
>> >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf type and
>> >> changed
>> >> > > > > > protocol
>> >> > > > > > > > accordingly.
>> >> > > > > > > >
>> >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
>> >> TopicMetadataRequest
>> >> > > > > which
>> >> > > > > > > has
>> >> > > > > > > > brokers, topics, and partitions. I think we should rename
>> >> that
>> >> > > > > request
>> >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest) and
>> >> include the
>> >> > > id
>> >> > > > > of
>> >> > > > > > > the
>> >> > > > > > > > controller. Or are there other things we could add here?
>> >> > > > > > > >
>> >> > > > > > > > A: I agree. Updated the KIP. Let's extends TopicMetadata
>> to
>> >> > > > version 2
>> >> > > > > > and
>> >> > > > > > > > include controller.
>> >> > > > > > > >
>> >> > > > > > > > 5. We have a tendency to try to make a lot of requests
>> that
>> >> can
>> >> > > > only
>> >> > > > > go
>> >> > > > > > > to
>> >> > > > > > > > particular nodes. This adds a lot of burden for client
>> >> > > > > implementations
>> >> > > > > > > (it
>> >> > > > > > > > sounds easy but each discovery can fail in many parts so
>> it
>> >> ends
>> >> > > up
>> >> > > > > > > being a
>> >> > > > > > > > full state machine to do right). I think we should
>> consider
>> >> > > making
>> >> > > > > > admin
>> >> > > > > > > > commands and ideally as many of the other apis as possible
>> >> > > > available
>> >> > > > > on
>> >> > > > > > > all
>> >> > > > > > > > brokers and just redirect to the controller on the broker
>> >> side.
>> >> > > > > Perhaps
>> >> > > > > > > > there would be a general way to encapsulate this
>> re-routing
>> >> > > > behavior.
>> >> > > > > > > >
>> >> > > > > > > > A: It's a very interesting idea, but seems there are some
>> >> > > concerns
>> >> > > > > > about
>> >> > > > > > > > this
>> >> > > > > > > > feature (like performance considerations, how this will
>> >> > > complicate
>> >> > > > > > server
>> >> > > > > > > > etc).
>> >> > > > > > > > I believe this shouldn't be a blocker. If this feature is
>> >> > > > implemented
>> >> > > > > > at
>> >> > > > > > > > some
>> >> > > > > > > > point it won't affect Admin changes - at least no changes
>> to
>> >> > > public
>> >> > > > > API
>> >> > > > > > > > will be required.
>> >> > > > > > > >
>> >> > > > > > > > 6. We should probably normalize the key value pairs used
>> for
>> >> > > > configs
>> >> > > > > > > rather
>> >> > > > > > > > than embedding a new formatting. So two strings rather
>> than
>> >> one
>> >> > > > with
>> >> > > > > an
>> >> > > > > > > > internal equals sign.
>> >> > > > > > > >
>> >> > > > > > > > A: Fixed in the latest patch - normalized configs and
>> >> changed
>> >> > > > > protocol
>> >> > > > > > > > accordingly.
>> >> > > > > > > >
>> >> > > > > > > > 7. Is the postcondition of these APIs that the command has
>> >> begun
>> >> > > or
>> >> > > > > > that
>> >> > > > > > > > the command has been completed? It is a lot more usable if
>> >> the
>> >> > > > > command
>> >> > > > > > > has
>> >> > > > > > > > been completed so you know that if you create a topic and
>> >> then
>> >> > > > > publish
>> >> > > > > > to
>> >> > > > > > > > it you won't get an exception about there being no such
>> >> topic.
>> >> > > > > > > >
>> >> > > > > > > > A: For long running requests (like reassign partitions) -
>> >> the
>> >> > > post
>> >> > > > > > > > condition is
>> >> > > > > > > > command has begun - so we don't block the client. In case
>> >> of your
>> >> > > > > > > example -
>> >> > > > > > > > topic commands, this will be refactored and topic commands
>> >> will
>> >> > > be
>> >> > > > > > > executed
>> >> > > > > > > > immediately, since the Controller will serve Admin
>> requests
>> >> > > > > > > > (follow-up ticket KAFKA-1777).
>> >> > > > > > > >
>> >> > > > > > > > 8. Describe topic and list topics duplicate a lot of stuff
>> >> in the
>> >> > > > > > > metadata
>> >> > > > > > > > request. Is there a reason to give back topics marked for
>> >> > > > deletion? I
>> >> > > > > > > feel
>> >> > > > > > > > like if we just make the post-condition of the delete
>> >> command be
>> >> > > > that
>> >> > > > > > the
>> >> > > > > > > > topic is deleted that will get rid of the need for this
>> >> right?
>> >> > > And
>> >> > > > it
>> >> > > > > > > will
>> >> > > > > > > > be much more intuitive.
>> >> > > > > > > >
>> >> > > > > > > > A: Fixed in the latest patch - removed topics marked for
>> >> deletion
>> >> > > > in
>> >> > > > > > > > ListTopicsRequest.
>> >> > > > > > > >
>> >> > > > > > > > 9. Should we consider batching these requests? We have
>> >> generally
>> >> > > > > tried
>> >> > > > > > to
>> >> > > > > > > > allow multiple operations to be batched. My suspicion is
>> >> that
>> >> > > > without
>> >> > > > > > > this
>> >> > > > > > > > we will get a lot of code that does something like
>> >> > > > > > > >    for(topic: adminClient.listTopics())
>> >> > > > > > > >       adminClient.describeTopic(topic)
>> >> > > > > > > > this code will work great when you test on 5 topics but
>> not
>> >> do as
>> >> > > > > well
>> >> > > > > > if
>> >> > > > > > > > you have 50k.
>> >> > > > > > > >
>> >> > > > > > > > A: Updated the KIP - please check "Topic Admin Schema"
>> >> section.
>> >> > > > > > > >
>> >> > > > > > > > 10. I think we should also discuss how we want to expose a
>> >> > > > > programmatic
>> >> > > > > > > JVM
>> >> > > > > > > > client api for these operations. Currently people rely on
>> >> > > > AdminUtils
>> >> > > > > > > which
>> >> > > > > > > > is totally sketchy. I think we probably need another
>> client
>> >> under
>> >> > > > > > > clients/
>> >> > > > > > > > that exposes administrative functionality. We will need
>> >> this just
>> >> > > > to
>> >> > > > > > > > properly test the new apis, I suspect. We should figure
>> out
>> >> that
>> >> > > > API.
>> >> > > > > > > >
>> >> > > > > > > > A: Updated the KIP - please check "Admin Client" section
>> >> with an
>> >> > > > > > initial
>> >> > > > > > > > API proposal.
>> >> > > > > > > >
>> >> > > > > > > > 11. The other information that would be really useful to
>> get
>> >> > > would
>> >> > > > be
>> >> > > > > > > > information about partitions--how much data is in the
>> >> partition,
>> >> > > > what
>> >> > > > > > are
>> >> > > > > > > > the segment offsets, what is the log-end offset (i.e. last
>> >> > > offset),
>> >> > > > > > what
>> >> > > > > > > is
>> >> > > > > > > > the compaction point, etc. I think that done right this
>> >> would be
>> >> > > > the
>> >> > > > > > > > successor to the very awkward OffsetRequest we have today.
>> >> > > > > > > >
>> >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest
>> >> patch. I
>> >> > > > > believe
>> >> > > > > > > > this should
>> >> > > > > > > > be resolved in a separate KIP / jira ticket.
>> >> > > > > > > >
>> >> > > > > > > > 12. Generally we can do good error handling without
>> needing
>> >> > > custom
>> >> > > > > > > > server-side
>> >> > > > > > > > messages. I.e. generally the client has the context to
>> know
>> >> that
>> >> > > if
>> >> > > > > it
>> >> > > > > > > got
>> >> > > > > > > > an error that the topic doesn't exist to say "Topic X
>> >> doesn't
>> >> > > > exist"
>> >> > > > > > > rather
>> >> > > > > > > > than "error code 14" (or whatever). Maybe there are
>> specific
>> >> > > cases
>> >> > > > > > where
>> >> > > > > > > > this is hard? If we want to add server-side error messages
>> >> we
>> >> > > > really
>> >> > > > > do
>> >> > > > > > > > need to do this in a consistent way across the protocol.
>> >> > > > > > > >
>> >> > > > > > > > A: Updated the KIP - please check "Protocol Errors"
>> >> section. I
>> >> > > > added
>> >> > > > > > the
>> >> > > > > > > > comprehensive, fine-grained list of error codes.
>> >> > > > > > > >
>> >> > > > > > > > Comments from Guozhang:
>> >> > > > > > > > 13. Describe topic request: it would be great to go beyond
>> >> just
>> >> > > > > > batching
>> >> > > > > > > on
>> >> > > > > > > > topic name regex for this request. For example, a very
>> >> common use
>> >> > > > > case
>> >> > > > > > of
>> >> > > > > > > > the topic command is to list all topics whose config A's
>> >> value is
>> >> > > > B.
>> >> > > > > > With
>> >> > > > > > > > topic name regex then we have to first retrieve __all__
>> >> topics's
>> >> > > > > > > > description info and then filter at the client end, which
>> >> will
>> >> > > be a
>> >> > > > > > huge
>> >> > > > > > > > burden on ZK.
>> >> > > > > > > > AND
>> >> > > > > > > > 14. Config K-Vs in create topic: this is related to the
>> >> previous
>> >> > > > > point;
>> >> > > > > > > > maybe we can add another metadata K-V or just a metadata
>> >> string
>> >> > > > along
>> >> > > > > > > side
>> >> > > > > > > > with config K-V in create topic like we did for offset
>> >> commit
>> >> > > > > request.
>> >> > > > > > > This
>> >> > > > > > > > field can be quite useful in storing information like
>> >> "owner" of
>> >> > > > the
>> >> > > > > > > topic
>> >> > > > > > > > who issue the create command, etc, which is quite
>> important
>> >> for a
>> >> > > > > > > > multi-tenant setting. Then in the describe topic request
>> we
>> >> can
>> >> > > > also
>> >> > > > > > > batch
>> >> > > > > > > > on regex of the metadata field.
>> >> > > > > > > >
>> >> > > > > > > > A: As discussed it is very interesting but can be
>> >> implemented
>> >> > > later
>> >> > > > > > after
>> >> > > > > > > > we have some basic functionality there.
>> >> > > > > > > >
>> >> > > > > > > > 15. Today all the admin operations are async in the sense
>> >> that
>> >> > > > > command
>> >> > > > > > > will
>> >> > > > > > > > return once it is written in ZK, and that is why we need
>> >> extra
>> >> > > > > > > verification
>> >> > > > > > > > like testUtil.waitForTopicCreated() / verify partition
>> >> > > reassignment
>> >> > > > > > > > request, etc. With admin requests we could add a flag to
>> >> enable /
>> >> > > > > > disable
>> >> > > > > > > > synchronous requests; when it is turned on, the response
>> >> will not
>> >> > > > > > return
>> >> > > > > > > > until the request has been completed. And for async
>> >> requests we
>> >> > > can
>> >> > > > > > add a
>> >> > > > > > > > "token" field in the response, and then only need a
>> general
>> >> > > "admin
>> >> > > > > > > > verification request" with the given token to check if the
>> >> async
>> >> > > > > > request
>> >> > > > > > > > has been completed.
>> >> > > > > > > >
>> >> > > > > > > > A: I see your point. My idea was to provide specific
>> >> > > > Verify...Request
>> >> > > > > > per
>> >> > > > > > > > each
>> >> > > > > > > > long running request, where needed. We can do it the way
>> you
>> >> > > > suggest.
>> >> > > > > > The
>> >> > > > > > > > only
>> >> > > > > > > > concern is that introducing a token we again will make
>> >> schema
>> >> > > > > > "dynamic".
>> >> > > > > > > We
>> >> > > > > > > > wanted
>> >> > > > > > > > to do similar thing introducing single AdminRequest for
>> all
>> >> topic
>> >> > > > > > > commands
>> >> > > > > > > > but rejected
>> >> > > > > > > > this idea because we wanted to have schema defined. So
>> this
>> >> is
>> >> > > > more a
>> >> > > > > > > > choice between:
>> >> > > > > > > > a) have fixed schema but introduce each time new
>> >> Verify...Request
>> >> > > > for
>> >> > > > > > > > long-running requests
>> >> > > > > > > > b) use one request for verification but generalize it with
>> >> token
>> >> > > > > > > > I'm fine with whatever decision community come to. Just
>> let
>> >> me
>> >> > > know
>> >> > > > > > your
>> >> > > > > > > > thoughts.
>> >> > > > > > > >
>> >> > > > > > > > Comment from Gwen:
>> >> > > > > > > > 16. Specifically for ownership, I think the plan is to add
>> >> ACL
>> >> > > (it
>> >> > > > > > sounds
>> >> > > > > > > > like you are describing ACL) via an external system
>> (Argus,
>> >> > > > Sentry).
>> >> > > > > > > > I remember KIP-11 described this, but I can't find the KIP
>> >> any
>> >> > > > > longer.
>> >> > > > > > > >
>> >> > > > > > > > A: Okay, no problem. Not sure though how we are going to
>> >> handle
>> >> > > it.
>> >> > > > > > Wait
>> >> > > > > > > > which KIP
>> >> > > > > > > > will be committed first and include changes to
>> >> TopicMetadata from
>> >> > > > the
>> >> > > > > > > later
>> >> > > > > > > > one?
>> >> > > > > > > > Anyway, I added this note to "Open Questions" section so
>> we
>> >> don't
>> >> > > > > miss
>> >> > > > > > > this
>> >> > > > > > > > piece.
>> >> > > > > > > >
>> >> > > > > > > > Thanks,
>> >> > > > > > > > Andrii Biletskyi
>> >> > > > > > > >
>> >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
>> >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> >> > > > > > > >
>> >> > > > > > > > > Hi all,
>> >> > > > > > > > >
>> >> > > > > > > > > Today I uploaded the patch that covers some of the
>> >> discussed
>> >> > > and
>> >> > > > > > agreed
>> >> > > > > > > > > items:
>> >> > > > > > > > > - removed MaybeOf optional type
>> >> > > > > > > > > - switched to java protocol definitions
>> >> > > > > > > > > - simplified messages (normalized configs, removed topic
>> >> marked
>> >> > > > for
>> >> > > > > > > > > deletion)
>> >> > > > > > > > >
>> >> > > > > > > > > I also updated the KIP-4 with respective changes and
>> >> wrote down
>> >> > > > my
>> >> > > > > > > > > proposal for
>> >> > > > > > > > > pending items:
>> >> > > > > > > > > - Batch Admin Operations -> updated Wire Protocol schema
>> >> > > proposal
>> >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
>> >> > > > TopicMetadataRequest
>> >> > > > > > > > > - Admin Client -> updated my initial proposal to reflect
>> >> > > batching
>> >> > > > > > > > > - Error codes -> proposed fine-grained error code
>> instead
>> >> of
>> >> > > > > > > > > AdminRequestFailed
>> >> > > > > > > > >
>> >> > > > > > > > > I will also send a separate email to cover all comments
>> >> from
>> >> > > this
>> >> > > > > > > thread.
>> >> > > > > > > > >
>> >> > > > > > > > > Thanks,
>> >> > > > > > > > > Andrii Biletskyi
>> >> > > > > > > > >
>> >> > > > > > > > >
>> >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
>> >> > > > > gshapira@cloudera.com
>> >> > > > > > >
>> >> > > > > > > > > wrote:
>> >> > > > > > > > >
>> >> > > > > > > > >> Found KIP-11 (
>> >> > > > > > > > >>
>> >> > > > > > > >
>> >> > > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
>> >> > > > > > > > >> )
>> >> > > > > > > > >> It actually specifies changes to the Metadata protocol,
>> >> so
>> >> > > > making
>> >> > > > > > sure
>> >> > > > > > > > >> both KIPs are consistent in this regard will be good.
>> >> > > > > > > > >>
>> >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
>> >> > > > > > gshapira@cloudera.com
>> >> > > > > > > >
>> >> > > > > > > > >> wrote:
>> >> > > > > > > > >> > Specifically for ownership, I think the plan is to
>> add
>> >> ACL
>> >> > > (it
>> >> > > > > > > sounds
>> >> > > > > > > > >> > like you are describing ACL) via an external system
>> >> (Argus,
>> >> > > > > > Sentry).
>> >> > > > > > > > >> > I remember KIP-11 described this, but I can't find
>> the
>> >> KIP
>> >> > > any
>> >> > > > > > > longer.
>> >> > > > > > > > >> >
>> >> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting
>> >> information
>> >> > > that
>> >> > > > > > > already
>> >> > > > > > > > >> > exists from Kafka brokers, not on adding information
>> >> that
>> >> > > > > perhaps
>> >> > > > > > > > >> > should exist but doesn't yet?
>> >> > > > > > > > >> >
>> >> > > > > > > > >> > Gwen
>> >> > > > > > > > >> >
>> >> > > > > > > > >> >
>> >> > > > > > > > >> >
>> >> > > > > > > > >> >
>> >> > > > > > > > >> >
>> >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
>> >> > > > > > wangguoz@gmail.com>
>> >> > > > > > > > >> wrote:
>> >> > > > > > > > >> >> Folks,
>> >> > > > > > > > >> >>
>> >> > > > > > > > >> >> Just want to elaborate a bit more on the
>> create-topic
>> >> > > > metadata
>> >> > > > > > and
>> >> > > > > > > > >> batching
>> >> > > > > > > > >> >> describe-topic based on config / metadata in my
>> >> previous
>> >> > > > email
>> >> > > > > as
>> >> > > > > > > we
>> >> > > > > > > > >> work
>> >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to have some
>> >> sort of
>> >> > > > > topic
>> >> > > > > > > > >> management
>> >> > > > > > > > >> >> mechanisms, which I think is quite important in a
>> >> > > > multi-tenant
>> >> > > > > /
>> >> > > > > > > > cloud
>> >> > > > > > > > >> >> architecture: today anyone can create topics in a
>> >> shared
>> >> > > > Kafka
>> >> > > > > > > > >> cluster, but
>> >> > > > > > > > >> >> there is no concept or "ownership" of topics that
>> are
>> >> > > created
>> >> > > > > by
>> >> > > > > > > > >> different
>> >> > > > > > > > >> >> users. For example, at LinkedIn we basically
>> >> distinguish
>> >> > > > topic
>> >> > > > > > > owners
>> >> > > > > > > > >> via
>> >> > > > > > > > >> >> some casual topic name prefix, which is a bit
>> awkward
>> >> and
>> >> > > > does
>> >> > > > > > not
>> >> > > > > > > > fly
>> >> > > > > > > > >> as
>> >> > > > > > > > >> >> we scale our customers. It would be great to use
>> >> > > > > describe-topics
>> >> > > > > > > such
>> >> > > > > > > > >> as:
>> >> > > > > > > > >> >>
>> >> > > > > > > > >> >> Describe all topics that is created by me.
>> >> > > > > > > > >> >>
>> >> > > > > > > > >> >> Describe all topics whose retention time is
>> overriden
>> >> to X.
>> >> > > > > > > > >> >>
>> >> > > > > > > > >> >> Describe all topics whose writable group include
>> user
>> >> Y
>> >> > > (this
>> >> > > > > is
>> >> > > > > > > > >> related to
>> >> > > > > > > > >> >> authorization), etc..
>> >> > > > > > > > >> >>
>> >> > > > > > > > >> >> One possible way to achieve this is to add a
>> metadata
>> >> file
>> >> > > in
>> >> > > > > the
>> >> > > > > > > > >> >> create-topic request, whose value will also be
>> >> written ZK
>> >> > > as
>> >> > > > we
>> >> > > > > > > > create
>> >> > > > > > > > >> the
>> >> > > > > > > > >> >> topic; then describe-topics can choose to batch
>> topics
>> >> > > based
>> >> > > > on
>> >> > > > > > 1)
>> >> > > > > > > > name
>> >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata regex,
>> etc.
>> >> > > > > > > > >> >>
>> >> > > > > > > > >> >> Thoughts?
>> >> > > > > > > > >> >>
>> >> > > > > > > > >> >> Guozhang
>> >> > > > > > > > >> >>
>> >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
>> >> > > > > > wangguoz@gmail.com>
>> >> > > > > > > > >> wrote:
>> >> > > > > > > > >> >>
>> >> > > > > > > > >> >>> Thanks for the updated wiki. A few comments below:
>> >> > > > > > > > >> >>>
>> >> > > > > > > > >> >>> 1. Error description in response: I think if some
>> >> > > errorCode
>> >> > > > > > could
>> >> > > > > > > > >> indicate
>> >> > > > > > > > >> >>> several different error cases then we should really
>> >> change
>> >> > > > it
>> >> > > > > to
>> >> > > > > > > > >> multiple
>> >> > > > > > > > >> >>> codes. In general the errorCode itself would be
>> >> precise
>> >> > > and
>> >> > > > > > > > >> sufficient for
>> >> > > > > > > > >> >>> describing the server side errors.
>> >> > > > > > > > >> >>>
>> >> > > > > > > > >> >>> 2. Describe topic request: it would be great to go
>> >> beyond
>> >> > > > just
>> >> > > > > > > > >> batching on
>> >> > > > > > > > >> >>> topic name regex for this request. For example, a
>> >> very
>> >> > > > common
>> >> > > > > > use
>> >> > > > > > > > >> case of
>> >> > > > > > > > >> >>> the topic command is to list all topics whose
>> config
>> >> A's
>> >> > > > value
>> >> > > > > > is
>> >> > > > > > > B.
>> >> > > > > > > > >> With
>> >> > > > > > > > >> >>> topic name regex then we have to first retrieve
>> >> __all__
>> >> > > > > topics's
>> >> > > > > > > > >> >>> description info and then filter at the client end,
>> >> which
>> >> > > > will
>> >> > > > > > be
>> >> > > > > > > a
>> >> > > > > > > > >> huge
>> >> > > > > > > > >> >>> burden on ZK.
>> >> > > > > > > > >> >>>
>> >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is related to
>> >> the
>> >> > > > > previous
>> >> > > > > > > > point;
>> >> > > > > > > > >> >>> maybe we can add another metadata K-V or just a
>> >> metadata
>> >> > > > > string
>> >> > > > > > > > along
>> >> > > > > > > > >> side
>> >> > > > > > > > >> >>> with config K-V in create topic like we did for
>> >> offset
>> >> > > > commit
>> >> > > > > > > > >> request. This
>> >> > > > > > > > >> >>> field can be quite useful in storing information
>> like
>> >> > > > "owner"
>> >> > > > > of
>> >> > > > > > > the
>> >> > > > > > > > >> topic
>> >> > > > > > > > >> >>> who issue the create command, etc, which is quite
>> >> > > important
>> >> > > > > for
>> >> > > > > > a
>> >> > > > > > > > >> >>> multi-tenant setting. Then in the describe topic
>> >> request
>> >> > > we
>> >> > > > > can
>> >> > > > > > > also
>> >> > > > > > > > >> batch
>> >> > > > > > > > >> >>> on regex of the metadata field.
>> >> > > > > > > > >> >>>
>> >> > > > > > > > >> >>> 4. Today all the admin operations are async in the
>> >> sense
>> >> > > > that
>> >> > > > > > > > command
>> >> > > > > > > > >> will
>> >> > > > > > > > >> >>> return once it is written in ZK, and that is why we
>> >> need
>> >> > > > extra
>> >> > > > > > > > >> verification
>> >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify
>> >> partition
>> >> > > > > > > reassignment
>> >> > > > > > > > >> >>> request, etc. With admin requests we could add a
>> >> flag to
>> >> > > > > enable
>> >> > > > > > /
>> >> > > > > > > > >> disable
>> >> > > > > > > > >> >>> synchronous requests; when it is turned on, the
>> >> response
>> >> > > > will
>> >> > > > > > not
>> >> > > > > > > > >> return
>> >> > > > > > > > >> >>> until the request has been completed. And for async
>> >> > > requests
>> >> > > > > we
>> >> > > > > > > can
>> >> > > > > > > > >> add a
>> >> > > > > > > > >> >>> "token" field in the response, and then only need a
>> >> > > general
>> >> > > > > > "admin
>> >> > > > > > > > >> >>> verification request" with the given token to check
>> >> if the
>> >> > > > > async
>> >> > > > > > > > >> request
>> >> > > > > > > > >> >>> has been completed.
>> >> > > > > > > > >> >>>
>> >> > > > > > > > >> >>> 5. +1 for extending Metadata request to include
>> >> > > controller /
>> >> > > > > > > > >> coordinator
>> >> > > > > > > > >> >>> information, and then we can remove the
>> >> ConsumerMetadata /
>> >> > > > > > > > >> ClusterMetadata
>> >> > > > > > > > >> >>> requests.
>> >> > > > > > > > >> >>>
>> >> > > > > > > > >> >>> Guozhang
>> >> > > > > > > > >> >>>
>> >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
>> >> > > > > > jjkoshy.w@gmail.com>
>> >> > > > > > > > >> wrote:
>> >> > > > > > > > >> >>>
>> >> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't think I
>> >> will be
>> >> > > > > able
>> >> > > > > > to
>> >> > > > > > > > >> make
>> >> > > > > > > > >> >>>> it today, so if notes can be sent out afterward
>> that
>> >> > > would
>> >> > > > be
>> >> > > > > > > > great.
>> >> > > > > > > > >> >>>>
>> >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen
>> >> Shapira
>> >> > > > wrote:
>> >> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking forward
>> >> to
>> >> > > > > chatting
>> >> > > > > > > with
>> >> > > > > > > > >> >>>> everyone :)
>> >> > > > > > > > >> >>>> >
>> >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
>> >> > > > > > > joe.stein@stealth.ly>
>> >> > > > > > > > >> wrote:
>> >> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout invite
>> to
>> >> all
>> >> > > > pmc,
>> >> > > > > > > > >> committers
>> >> > > > > > > > >> >>>> and
>> >> > > > > > > > >> >>>> > > everyone I found working on a KIP. If I missed
>> >> anyone
>> >> > > > in
>> >> > > > > > the
>> >> > > > > > > > >> invite
>> >> > > > > > > > >> >>>> please
>> >> > > > > > > > >> >>>> > > let me know and can update it, np.
>> >> > > > > > > > >> >>>> > >
>> >> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm Eastern
>> >> Time.
>> >> > > > Maybe
>> >> > > > > > we
>> >> > > > > > > > can
>> >> > > > > > > > >> get
>> >> > > > > > > > >> >>>> INFRA
>> >> > > > > > > > >> >>>> > > help to make a google account so we can manage
>> >> > > better?
>> >> > > > > > > > >> >>>> > >
>> >> > > > > > > > >> >>>> > > To discuss
>> >> > > > > > > > >> >>>> > >
>> >> > > > > > > > >> >>>>
>> >> > > > > > > > >>
>> >> > > > > > > >
>> >> > > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>> >> > > > > > > > >> >>>> > > in progress and related JIRA that are
>> >> interdependent
>> >> > > > and
>> >> > > > > > > common
>> >> > > > > > > > >> work.
>> >> > > > > > > > >> >>>> > >
>> >> > > > > > > > >> >>>> > > ~ Joe Stein
>> >> > > > > > > > >> >>>> > >
>> >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
>> >> > > > > > > > jay.kreps@gmail.com>
>> >> > > > > > > > >> >>>> wrote:
>> >> > > > > > > > >> >>>> > >
>> >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that will also
>> >> record
>> >> > > > and
>> >> > > > > > make
>> >> > > > > > > > the
>> >> > > > > > > > >> >>>> sessions
>> >> > > > > > > > >> >>>> > >> available on youtube.
>> >> > > > > > > > >> >>>> > >>
>> >> > > > > > > > >> >>>> > >> -Jay
>> >> > > > > > > > >> >>>> > >>
>> >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff
>> Holoman
>> >> <
>> >> > > > > > > > >> >>>> jholoman@cloudera.com>
>> >> > > > > > > > >> >>>> > >> wrote:
>> >> > > > > > > > >> >>>> > >>
>> >> > > > > > > > >> >>>> > >> > Jay / Joe
>> >> > > > > > > > >> >>>> > >> >
>> >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for this
>> >> purpose.
>> >> > > We
>> >> > > > > > could
>> >> > > > > > > > >> record
>> >> > > > > > > > >> >>>> the
>> >> > > > > > > > >> >>>> > >> > sessions if there is interest and publish
>> >> them
>> >> > > out.
>> >> > > > > > > > >> >>>> > >> >
>> >> > > > > > > > >> >>>> > >> > Thanks
>> >> > > > > > > > >> >>>> > >> >
>> >> > > > > > > > >> >>>> > >> > Jeff
>> >> > > > > > > > >> >>>> > >> >
>> >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay
>> Kreps <
>> >> > > > > > > > >> jay.kreps@gmail.com>
>> >> > > > > > > > >> >>>> wrote:
>> >> > > > > > > > >> >>>> > >> >
>> >> > > > > > > > >> >>>> > >> > > Let's try to get the technical hang-ups
>> >> sorted
>> >> > > > out,
>> >> > > > > > > > though.
>> >> > > > > > > > >> I
>> >> > > > > > > > >> >>>> really
>> >> > > > > > > > >> >>>> > >> > think
>> >> > > > > > > > >> >>>> > >> > > there is some benefit to live discussion
>> vs
>> >> > > > > writing. I
>> >> > > > > > > am
>> >> > > > > > > > >> >>>> hopeful that
>> >> > > > > > > > >> >>>> > >> if
>> >> > > > > > > > >> >>>> > >> > > we post instructions and give ourselves a
>> >> few
>> >> > > > > attempts
>> >> > > > > > > we
>> >> > > > > > > > >> can
>> >> > > > > > > > >> >>>> get it
>> >> > > > > > > > >> >>>> > >> > > working.
>> >> > > > > > > > >> >>>> > >> > >
>> >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work for
>> >> me...any
>> >> > > > > > objections?
>> >> > > > > > > > >> >>>> > >> > >
>> >> > > > > > > > >> >>>> > >> > > -Jay
>> >> > > > > > > > >> >>>> > >> > >
>> >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe
>> Stein
>> >> <
>> >> > > > > > > > >> joe.stein@stealth.ly
>> >> > > > > > > > >> >>>> >
>> >> > > > > > > > >> >>>> > >> wrote:
>> >> > > > > > > > >> >>>> > >> > >
>> >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe like every
>> >> > > Tuesday ~
>> >> > > > > 1pm
>> >> > > > > > > ET
>> >> > > > > > > > /
>> >> > > > > > > > >> 10am
>> >> > > > > > > > >> >>>> PT
>> >> > > > > > > > >> >>>> > >> ????
>> >> > > > > > > > >> >>>> > >> > > >
>> >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but there
>> is
>> >> > > always
>> >> > > > > some
>> >> > > > > > > > >> issue or
>> >> > > > > > > > >> >>>> > >> whatever
>> >> > > > > > > > >> >>>> > >> > so
>> >> > > > > > > > >> >>>> > >> > > > we know the apache irc channel works.
>> We
>> >> can
>> >> > > > start
>> >> > > > > > > there
>> >> > > > > > > > >> and
>> >> > > > > > > > >> >>>> see how
>> >> > > > > > > > >> >>>> > >> it
>> >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too and
>> >> > > associate
>> >> > > > to
>> >> > > > > > > > >> tickets if
>> >> > > > > > > > >> >>>> need be
>> >> > > > > > > > >> >>>> > >> > > makes
>> >> > > > > > > > >> >>>> > >> > > > it helpful for things.
>> >> > > > > > > > >> >>>> > >> > > >
>> >> > > > > > > > >> >>>> > >> > > > ~ Joestein
>> >> > > > > > > > >> >>>> > >> > > >
>> >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay
>> >> Kreps <
>> >> > > > > > > > >> >>>> jay.kreps@gmail.com>
>> >> > > > > > > > >> >>>> > >> > wrote:
>> >> > > > > > > > >> >>>> > >> > > >
>> >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a Google
>> >> Hangout to
>> >> > > > chat
>> >> > > > > > > about
>> >> > > > > > > > >> this.
>> >> > > > > > > > >> >>>> What
>> >> > > > > > > > >> >>>> > >> > about
>> >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
>> further...I
>> >> > > > actually
>> >> > > > > > > think
>> >> > > > > > > > it
>> >> > > > > > > > >> >>>> would be
>> >> > > > > > > > >> >>>> > >> > good
>> >> > > > > > > > >> >>>> > >> > > > for
>> >> > > > > > > > >> >>>> > >> > > > > everyone spending a reasonable chunk
>> of
>> >> > > their
>> >> > > > > week
>> >> > > > > > > on
>> >> > > > > > > > >> Kafka
>> >> > > > > > > > >> >>>> stuff
>> >> > > > > > > > >> >>>> > >> to
>> >> > > > > > > > >> >>>> > >> > > > maybe
>> >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think we could
>> >> use
>> >> > > time
>> >> > > > > to
>> >> > > > > > > talk
>> >> > > > > > > > >> >>>> through
>> >> > > > > > > > >> >>>> > >> design
>> >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of
>> code
>> >> > > > reviews,
>> >> > > > > > talk
>> >> > > > > > > > >> through
>> >> > > > > > > > >> >>>> any
>> >> > > > > > > > >> >>>> > >> > tricky
>> >> > > > > > > > >> >>>> > >> > > > > issues, etc.
>> >> > > > > > > > >> >>>> > >> > > > >
>> >> > > > > > > > >> >>>> > >> > > > > We can make it publicly available so
>> >> that
>> >> > > any
>> >> > > > > one
>> >> > > > > > > can
>> >> > > > > > > > >> follow
>> >> > > > > > > > >> >>>> along
>> >> > > > > > > > >> >>>> > >> > who
>> >> > > > > > > > >> >>>> > >> > > > > likes.
>> >> > > > > > > > >> >>>> > >> > > > >
>> >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this? If so
>> I'll
>> >> try
>> >> > > to
>> >> > > > > set
>> >> > > > > > it
>> >> > > > > > > > up
>> >> > > > > > > > >> >>>> starting
>> >> > > > > > > > >> >>>> > >> next
>> >> > > > > > > > >> >>>> > >> > > > week.
>> >> > > > > > > > >> >>>> > >> > > > >
>> >> > > > > > > > >> >>>> > >> > > > > -Jay
>> >> > > > > > > > >> >>>> > >> > > > >
>> >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM,
>> Andrii
>> >> > > > > Biletskyi
>> >> > > > > > <
>> >> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
>> >> > > > > > > > >> >>>> > >> > > > >
>> >> > > > > > > > >> >>>> > >> > > > > > Hi all,
>> >> > > > > > > > >> >>>> > >> > > > > >
>> >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed /
>> >> aligned
>> >> > > > > document
>> >> > > > > > > > >> structure.
>> >> > > > > > > > >> >>>> Also I
>> >> > > > > > > > >> >>>> > >> > > added
>> >> > > > > > > > >> >>>> > >> > > > > > some
>> >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
>> >> AdminClient so
>> >> > > we
>> >> > > > > have
>> >> > > > > > > > >> something
>> >> > > > > > > > >> >>>> to
>> >> > > > > > > > >> >>>> > >> start
>> >> > > > > > > > >> >>>> > >> > > > from
>> >> > > > > > > > >> >>>> > >> > > > > > while
>> >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
>> >> > > > > > > > >> >>>> > >> > > > > >
>> >> > > > > > > > >> >>>> > >> > > > > >
>> >> > > > > > > > >> >>>> > >> > > > > >
>> >> > > > > > > > >> >>>> > >> > > > >
>> >> > > > > > > > >> >>>> > >> > > >
>> >> > > > > > > > >> >>>> > >> > >
>> >> > > > > > > > >> >>>> > >> >
>> >> > > > > > > > >> >>>> > >>
>> >> > > > > > > > >> >>>>
>> >> > > > > > > > >>
>> >> > > > > > > >
>> >> > > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>> >> > > > > > > > >> >>>> > >> > > > > >
>> >> > > > > > > > >> >>>> > >> > > > > > Thanks,
>> >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
>> >> > > > > > > > >> >>>> > >> > > > > >
>> >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM,
>> >> Andrii
>> >> > > > > > Biletskyi
>> >> > > > > > > <
>> >> > > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly>
>> wrote:
>> >> > > > > > > > >> >>>> > >> > > > > >
>> >> > > > > > > > >> >>>> > >> > > > > > > Jay,
>> >> > > > > > > > >> >>>> > >> > > > > > >
>> >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you are right,
>> >> in
>> >> > > most
>> >> > > > > > cases
>> >> > > > > > > > >> client
>> >> > > > > > > > >> >>>> will
>> >> > > > > > > > >> >>>> > >> have
>> >> > > > > > > > >> >>>> > >> > > > enough
>> >> > > > > > > > >> >>>> > >> > > > > > > context to show descriptive error
>> >> > > message.
>> >> > > > > My
>> >> > > > > > > > >> concern is
>> >> > > > > > > > >> >>>> that
>> >> > > > > > > > >> >>>> > >> we
>> >> > > > > > > > >> >>>> > >> > > will
>> >> > > > > > > > >> >>>> > >> > > > > > have
>> >> > > > > > > > >> >>>> > >> > > > > > > to
>> >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error codes for
>> >> each
>> >> > > > > possible
>> >> > > > > > > > >> error. Of
>> >> > > > > > > > >> >>>> course,
>> >> > > > > > > > >> >>>> > >> > we
>> >> > > > > > > > >> >>>> > >> > > > > could
>> >> > > > > > > > >> >>>> > >> > > > > > > reuse
>> >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
>> >> > > > > > > UknownTopicOrPartitionCode,
>> >> > > > > > > > >> but we
>> >> > > > > > > > >> >>>> will
>> >> > > > > > > > >> >>>> > >> > also
>> >> > > > > > > > >> >>>> > >> > > > need
>> >> > > > > > > > >> >>>> > >> > > > > > to
>> >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
>> >> TopicAlreadyExistsCode,
>> >> > > > > > > > >> >>>> TopicConfigInvalid (both
>> >> > > > > > > > >> >>>> > >> > for
>> >> > > > > > > > >> >>>> > >> > > > > topic
>> >> > > > > > > > >> >>>> > >> > > > > > > name and config, and probably
>> user
>> >> would
>> >> > > > > like
>> >> > > > > > to
>> >> > > > > > > > >> know
>> >> > > > > > > > >> >>>> what
>> >> > > > > > > > >> >>>> > >> > exactly
>> >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
>> >> > > > > > > InvalidReplicaAssignment,
>> >> > > > > > > > >> >>>> > >> InternalError
>> >> > > > > > > > >> >>>> > >> > > > (e.g.
>> >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
>> >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
>> TopicCommand,
>> >> we
>> >> > > will
>> >> > > > > > also
>> >> > > > > > > > >> need to
>> >> > > > > > > > >> >>>> add
>> >> > > > > > > > >> >>>> > >> > similar
>> >> > > > > > > > >> >>>> > >> > > > > stuff
>> >> > > > > > > > >> >>>> > >> > > > > > > for
>> >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
>> >> PreferredReplica. So
>> >> > > > > we'll
>> >> > > > > > > end
>> >> > > > > > > > >> up
>> >> > > > > > > > >> >>>> with a
>> >> > > > > > > > >> >>>> > >> > large
>> >> > > > > > > > >> >>>> > >> > > > list
>> >> > > > > > > > >> >>>> > >> > > > > > of
>> >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in Admin
>> >> > > protocol.
>> >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree my
>> >> proposal is
>> >> > > > not
>> >> > > > > > > > >> consistent
>> >> > > > > > > > >> >>>> with
>> >> > > > > > > > >> >>>> > >> > other
>> >> > > > > > > > >> >>>> > >> > > > > cases.
>> >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better solution
>> >> or
>> >> > > > > something
>> >> > > > > > > > >> >>>> in-between.
>> >> > > > > > > > >> >>>> > >> > > > > > >
>> >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it is a
>> >> great
>> >> > > > idea.
>> >> > > > > > > This
>> >> > > > > > > > >> way we
>> >> > > > > > > > >> >>>> can
>> >> > > > > > > > >> >>>> > >> move
>> >> > > > > > > > >> >>>> > >> > > on
>> >> > > > > > > > >> >>>> > >> > > > > > > faster.
>> >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on date/time
>> so
>> >> > > people
>> >> > > > > can
>> >> > > > > > > > join.
>> >> > > > > > > > >> >>>> Will work
>> >> > > > > > > > >> >>>> > >> > for
>> >> > > > > > > > >> >>>> > >> > > me
>> >> > > > > > > > >> >>>> > >> > > > > > this
>> >> > > > > > > > >> >>>> > >> > > > > > > and
>> >> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime if
>> agreed
>> >> in
>> >> > > > > advance.
>> >> > > > > > > > >> >>>> > >> > > > > > >
>> >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
>> >> > > > > > > > >> >>>> > >> > > > > > > Andrii
>> >> > > > > > > > >> >>>> > >> > > > > > >
>> >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM,
>> >> Jay
>> >> > > > Kreps <
>> >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
>> >> > > > > > > > >> >>>> > >> > > > > wrote:
>> >> > > > > > > > >> >>>> > >> > > > > > >
>> >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
>> >> > > > > > > > >> >>>> > >> > > > > > >>
>> >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good error
>> >> handling
>> >> > > > > > without
>> >> > > > > > > > >> needing
>> >> > > > > > > > >> >>>> custom
>> >> > > > > > > > >> >>>> > >> > > > > > server-side
>> >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the
>> >> client has
>> >> > > > the
>> >> > > > > > > > >> context to
>> >> > > > > > > > >> >>>> know
>> >> > > > > > > > >> >>>> > >> that
>> >> > > > > > > > >> >>>> > >> > > if
>> >> > > > > > > > >> >>>> > >> > > > it
>> >> > > > > > > > >> >>>> > >> > > > > > got
>> >> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic doesn't
>> >> exist
>> >> > > to
>> >> > > > > say
>> >> > > > > > > > >> "Topic X
>> >> > > > > > > > >> >>>> doesn't
>> >> > > > > > > > >> >>>> > >> > > exist"
>> >> > > > > > > > >> >>>> > >> > > > > > >> rather
>> >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or
>> >> whatever).
>> >> > > Maybe
>> >> > > > > > there
>> >> > > > > > > > are
>> >> > > > > > > > >> >>>> specific
>> >> > > > > > > > >> >>>> > >> > cases
>> >> > > > > > > > >> >>>> > >> > > > > where
>> >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to add
>> >> > > > server-side
>> >> > > > > > > error
>> >> > > > > > > > >> >>>> messages we
>> >> > > > > > > > >> >>>> > >> > > really
>> >> > > > > > > > >> >>>> > >> > > > > do
>> >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a consistent
>> >> way
>> >> > > > across
>> >> > > > > > the
>> >> > > > > > > > >> protocol.
>> >> > > > > > > > >> >>>> > >> > > > > > >>
>> >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open
>> >> questions
>> >> > > > here
>> >> > > > > > > from
>> >> > > > > > > > my
>> >> > > > > > > > >> >>>> previous
>> >> > > > > > > > >> >>>> > >> > > list. I
>> >> > > > > > > > >> >>>> > >> > > > > > will
>> >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few days for
>> >> Strata
>> >> > > > > > though.
>> >> > > > > > > > >> Maybe
>> >> > > > > > > > >> >>>> we could
>> >> > > > > > > > >> >>>> > >> > do
>> >> > > > > > > > >> >>>> > >> > > a
>> >> > > > > > > > >> >>>> > >> > > > > > Google
>> >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open issues
>> >> some
>> >> > > time
>> >> > > > > > > towards
>> >> > > > > > > > >> the
>> >> > > > > > > > >> >>>> end of
>> >> > > > > > > > >> >>>> > >> > next
>> >> > > > > > > > >> >>>> > >> > > > week
>> >> > > > > > > > >> >>>> > >> > > > > > for
>> >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in this
>> ticket?
>> >> I
>> >> > > have
>> >> > > > a
>> >> > > > > > > > feeling
>> >> > > > > > > > >> that
>> >> > > > > > > > >> >>>> might
>> >> > > > > > > > >> >>>> > >> > > > progress
>> >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster than
>> >> email--I
>> >> > > > think
>> >> > > > > we
>> >> > > > > > > > >> could talk
>> >> > > > > > > > >> >>>> > >> through
>> >> > > > > > > > >> >>>> > >> > > > those
>> >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly
>> >> quickly...
>> >> > > > > > > > >> >>>> > >> > > > > > >>
>> >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
>> >> > > > > > > > >> >>>> > >> > > > > > >>
>> >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM,
>> >> Andrii
>> >> > > > > > > > Biletskyi <
>> >> > > > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly>
>> >> wrote:
>> >> > > > > > > > >> >>>> > >> > > > > > >>
>> >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
>> >> > > > > > > > >> >>>> > >> > > > > > >> >
>> >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address some of
>> >> the
>> >> > > > issues
>> >> > > > > > > which
>> >> > > > > > > > >> were
>> >> > > > > > > > >> >>>> > >> mentioned
>> >> > > > > > > > >> >>>> > >> > > > > earlier
>> >> > > > > > > > >> >>>> > >> > > > > > >> about
>> >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of
>> >> those was
>> >> > > > > about
>> >> > > > > > > > >> batching
>> >> > > > > > > > >> >>>> > >> > operations.
>> >> > > > > > > > >> >>>> > >> > > > What
>> >> > > > > > > > >> >>>> > >> > > > > > if
>> >> > > > > > > > >> >>>> > >> > > > > > >> we
>> >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand approach
>> >> and let
>> >> > > > > people
>> >> > > > > > > > >> specify
>> >> > > > > > > > >> >>>> > >> topic-name
>> >> > > > > > > > >> >>>> > >> > > by
>> >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
>> >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most of the
>> use
>> >> > > cases?
>> >> > > > > > > > >> >>>> > >> > > > > > >> >
>> >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what information
>> >> should
>> >> > > we
>> >> > > > > > > > generally
>> >> > > > > > > > >> >>>> provide in
>> >> > > > > > > > >> >>>> > >> > > Admin
>> >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
>> >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin commands
>> >> don't
>> >> > > > imply
>> >> > > > > > > they
>> >> > > > > > > > >> will
>> >> > > > > > > > >> >>>> be used
>> >> > > > > > > > >> >>>> > >> > only
>> >> > > > > > > > >> >>>> > >> > > > in
>> >> > > > > > > > >> >>>> > >> > > > > > CLI
>> >> > > > > > > > >> >>>> > >> > > > > > >> > but,
>> >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a very
>> >> > > important
>> >> > > > > > > client
>> >> > > > > > > > >> of this
>> >> > > > > > > > >> >>>> > >> > feature.
>> >> > > > > > > > >> >>>> > >> > > In
>> >> > > > > > > > >> >>>> > >> > > > > > this
>> >> > > > > > > > >> >>>> > >> > > > > > >> > case,
>> >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would like
>> to
>> >> > > provide
>> >> > > > > > users
>> >> > > > > > > > >> with
>> >> > > > > > > > >> >>>> rich
>> >> > > > > > > > >> >>>> > >> > > experience
>> >> > > > > > > > >> >>>> > >> > > > > in
>> >> > > > > > > > >> >>>> > >> > > > > > >> terms
>> >> > > > > > > > >> >>>> > >> > > > > > >> > of
>> >> > > > > > > > >> >>>> > >> > > > > > >> > getting results / errors of
>> the
>> >> > > > executed
>> >> > > > > > > > >> commands.
>> >> > > > > > > > >> >>>> Usually
>> >> > > > > > > > >> >>>> > >> we
>> >> > > > > > > > >> >>>> > >> > > > supply
>> >> > > > > > > > >> >>>> > >> > > > > > >> with
>> >> > > > > > > > >> >>>> > >> > > > > > >> > responses only errorCode,
>> which
>> >> looks
>> >> > > > > very
>> >> > > > > > > > >> limiting,
>> >> > > > > > > > >> >>>> in case
>> >> > > > > > > > >> >>>> > >> > of
>> >> > > > > > > > >> >>>> > >> > > > CLI
>> >> > > > > > > > >> >>>> > >> > > > > we
>> >> > > > > > > > >> >>>> > >> > > > > > >> may
>> >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human readable
>> >> error
>> >> > > > > > > description.
>> >> > > > > > > > >> >>>> > >> > > > > > >> >
>> >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into account
>> >> previous item
>> >> > > > > about
>> >> > > > > > > > >> batching,
>> >> > > > > > > > >> >>>> what
>> >> > > > > > > > >> >>>> > >> do
>> >> > > > > > > > >> >>>> > >> > > you
>> >> > > > > > > > >> >>>> > >> > > > > > think
>> >> > > > > > > > >> >>>> > >> > > > > > >> > about
>> >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
>> >> > > > > > > > >> >>>> > >> > > > > > >> >
>> >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support
>> >> regexp)
>> >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest =>
>> TopicName
>> >> > > > > Partitions
>> >> > > > > > > > >> Replicas
>> >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
>> >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
>> >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse =>
>> ErrorCode
>> >> > > > > > > > ErrorDescription
>> >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
>> >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription => string
>> >> (empty
>> >> > > if
>> >> > > > > > > > >> successful)
>> >> > > > > > > > >> >>>> > >> > > > > > >> >
>> >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
>> >> TopicNameRegexp
>> >> > > >
>> >
>> > ...
>> >
>> > [Message clipped]
>>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
Andri,

Thanks for the summary.

1. I just realized that in order to start working on KAFKA-1927, we will
need to merge the changes to OffsetCommitRequest (from 0.8.2) to trunk.
This is planned to be done as part of KAFKA-1634. So, we will need Guozhang
and Joel's help to wrap this up.

2. Thinking about this a bit more, if the semantic of those "write"
requests is async (i.e., after the client gets a response, it just means
that the operation is initiated, but not necessarily completed), we don't
really need to forward the requests to the controller. Instead, the
receiving broker can just write the operation to ZK as the admin command
line tool previously does. This will simplify the implementation.

8. There is another implementation detail for describe topic. Ideally, we
want to read the topic config from the broker cache, instead of ZooKeeper.
Currently, every broker reads the topic-level config for all topics.
However, it ignores those for topics not hosted on itself. So, we may need
to change TopicConfigManager a bit so that it caches the configs for all
topics.

Thanks,

Jun


On Tue, Mar 17, 2015 at 1:13 PM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Guys,
>
> Thanks for a great discussion!
> Here are the actions points:
>
> 1. Q: Get rid of all scala requests objects, use java protocol definitions.
>     A: Gwen kindly took that (KAFKA-1927). It's important to speed up
> review procedure
>          there since this ticket blocks other important changes.
>
> 2. Q: Generic re-reroute facility vs client maintaining cluster state.
>     A: Jay has added pseudo code to KAFKA-1912 - need to consider whether
> this will be
>         easy to implement as a server-side feature (comments are
> welcomed!).
>
> 3. Q: Controller field in wire protocol.
>     A: This might be useful for clients, add this to TopicMetadataResponse
> (already in KIP).
>
> 4. Q: Decoupling topic creation from TMR.
>     A: I will add proposed by Jun solution (using clientId for that) to the
> KIP.
>
> 5. Q: Bumping new versions of TMR vs grabbing all protocol changes in one
> version.
>     A: It was decided to try to gather all changes to protocol (before
> release).
>         In case of TMR it worth checking: KAFKA-2020 and KIP-13 (quotas)
>
> 6. Q: JSON lib is needed to deserialize user's input in CLI tool.
>     A: Use jackson for that, /tools project is a separate jar so shouldn't
> be a big deal.
>
> 7.  Q: VerifyReassingPartitions vs generic status check command.
>      A: For long-running requests like reassign partitions *progress* check
> request is useful,
>          it makes sense to introduce it.
>
>  Please add, correct me if I missed something.
>
> Thanks,
> Andrii Biletskyi
>
> On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Joel,
> >
> > You are right, I removed ClusterMetadata because we have partially
> > what we need in TopicMetadata. Also, as Jay pointed out earlier, we
> > would like to have "orthogonal" API, but at the same time we need
> > to be backward compatible.
> >
> > But I like your idea and even have some other arguments for this option:
> > There is also DescribeTopicRequest which was proposed in this KIP,
> > it returns topic configs, partitions, replication factor plus partition
> > ISR, ASR,
> > leader replica. The later part is really already there in
> > TopicMetadataRequest.
> > So again we'll have to add stuff to TMR, not to duplicate some info in
> > newly added requests. However, this way we'll end up with "monster"
> > request which returns cluster metadata, topic replication and config info
> > plus partition replication data. Seems logical to split TMR to
> > - ClusterMetadata (brokers + controller, maybe smth else)
> > - TopicMetadata (topic info + partition details)
> > But since current TMR is involved in lots of places (including network
> > client,
> > as I understand) this might be very serious change and it probably makes
> > sense to stick with current approach.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> >
> > On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jj...@gmail.com> wrote:
> >
> >> I may be missing some context but hopefully this will also be covered
> >> today: I thought the earlier proposal where there was an explicit
> >> ClusterMetadata request was clearer and explicit. During the course of
> >> this thread I think the conclusion was that the main need was for
> >> controller information and that can be rolled into the topic metadata
> >> response but that seems a bit irrelevant to topic metadata. FWIW I
> >> think the full broker-list is also irrelevant to topic metadata, but
> >> it is already there and in use. I think there is still room for an
> >> explicit ClusterMetadata request since there may be other
> >> cluster-level information that we may want to add over time (and that
> >> have nothing to do with topic metadata).
> >>
> >> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
> >> > Jun,
> >> >
> >> > 101. Okay, if you say that such use case is important. I also think
> >> > using clientId for these purposes is fine - if we already have this
> >> field
> >> > as part of all Wire protocol messages, why not use that.
> >> > I will update KIP-4 page if nobody has other ideas (which may come up
> >> > during the call today).
> >> >
> >> > 102.1 Agree, I'll update the KIP accordingly. I think we can add new,
> >> > fine-grained error codes if some error code received in specific case
> >> > won't give enough context to return a descriptive error message for
> >> user.
> >> >
> >> > Look forward to discussing all outstanding issues in detail today
> during
> >> > the call.
> >> >
> >> > Thanks,
> >> > Andrii Biletskyi
> >> >
> >> >
> >> >
> >> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io> wrote:
> >> >
> >> > > 101. There may be a use case where you only want the topics to be
> >> created
> >> > > manually by admins. Currently, you can do that by disabling auto
> topic
> >> > > creation and issue topic creation from the TopicCommand. If we
> >> disable auto
> >> > > topic creation completely on the broker and don't have a way to
> >> distinguish
> >> > > between topic creation requests from the regular clients and the
> >> admin, we
> >> > > can't support manual topic creation any more. I was thinking that
> >> another
> >> > > way of distinguishing the clients making the topic creation requests
> >> is
> >> > > using clientId. For example, the admin tool can set it to something
> >> like
> >> > > admin and the broker can treat that clientId specially.
> >> > >
> >> > > Also, there is a related discussion in KAFKA-2020. Currently, we do
> >> the
> >> > > following in TopicMetadataResponse:
> >> > >
> >> > > 1. If leader is not available, we set the partition level error code
> >> to
> >> > > LeaderNotAvailable.
> >> > > 2. If a non-leader replica is not available, we take that replica
> out
> >> of
> >> > > the assigned replica list and isr in the response. As an indication
> >> for
> >> > > doing that, we set the partition level error code to
> >> ReplicaNotAvailable.
> >> > >
> >> > > This has a few problems. First, ReplicaNotAvailable probably
> >> shouldn't be
> >> > > an error, at least for the normal producer/consumer clients that
> just
> >> want
> >> > > to find out the leader. Second, it can happen that both the leader
> and
> >> > > another replica are not available at the same time. There is no
> error
> >> code
> >> > > to indicate both. Third, even if a replica is not available, it's
> >> still
> >> > > useful to return its replica id since some clients (e.g. admin tool)
> >> may
> >> > > still make use of it.
> >> > >
> >> > > One way to address this issue is to always return the replica id for
> >> > > leader, assigned replicas, and isr regardless of whether the
> >> corresponding
> >> > > broker is live or not. Since we also return the list of live
> brokers,
> >> the
> >> > > client can figure out whether a leader or a replica is live or not
> >> and act
> >> > > accordingly. This way, we don't need to set the partition level
> error
> >> code
> >> > > when the leader or a replica is not available. This doesn't change
> >> the wire
> >> > > protocol, but does change the semantics. Since we are evolving the
> >> protocol
> >> > > of TopicMetadataRequest here, we can potentially piggyback the
> change.
> >> > >
> >> > > 102.1 For those types of errors due to invalid input, shouldn't we
> >> just
> >> > > guard it at parameter validation time and throw
> >> InvalidArgumentException
> >> > > without even sending the request to the broker?
> >> > >
> >> > > Thanks,
> >> > >
> >> > > Jun
> >> > >
> >> > >
> >> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> >> > > andrii.biletskyi@stealth.ly> wrote:
> >> > >
> >> > > > Jun,
> >> > > >
> >> > > > Answering your questions:
> >> > > >
> >> > > > 101. If I understand you correctly, you are saying future producer
> >> > > versions
> >> > > > (which
> >> > > > will be ported to TMR_V1) won't be able to automatically create
> >> topic (if
> >> > > > we
> >> > > > unconditionally remove topic creation from there). But we need to
> >> this
> >> > > > preserve logic.
> >> > > > Ok, about your proposal: I'm not a big fan too, when it comes to
> >> > > > differentiating
> >> > > > clients directly in protocol schema. And also I'm not sure I
> >> understand
> >> > > at
> >> > > > all why
> >> > > > auto.create.topics.enable is a server side configuration. Can we
> >> > > deprecate
> >> > > > this setting
> >> > > > in future versions, add this setting to producer and based on that
> >> upon
> >> > > > receiving
> >> > > > UnknownTopic create topic explicitly by a separate producer call
> via
> >> > > > adminClient?
> >> > > >
> >> > > > 102.1. Hm, yes. It's because we want to support batching and at
> the
> >> same
> >> > > > time we
> >> > > > want to give descriptive error messages for clients. Since
> >> AdminClient
> >> > > > holds the context
> >> > > > to construct such messages (e.g. AdminClient layer can know that
> >> > > > InvalidArgumentsCode
> >> > > > means two cases: either invalid number - e.g. -1; or
> >> replication-factor
> >> > > was
> >> > > > provided while
> >> > > > partitions argument wasn't) - I wrapped responses in Exceptions.
> >> But I'm
> >> > > > open to any
> >> > > > other ideas, this was just initial version.
> >> > > > 102.2. Yes, I agree. I'll change that to probably some other dto.
> >> > > >
> >> > > > Thanks,
> >> > > > Andrii Biletskyi
> >> > > >
> >> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io>
> wrote:
> >> > > >
> >> > > > > Andrii,
> >> > > > >
> >> > > > > 101. That's what I was thinking too, but it may not be that
> >> simple. In
> >> > > > > TopicMetadataRequest_V1,
> >> > > > > we can let it not trigger auto topic creation. Then, in the
> >> producer
> >> > > > side,
> >> > > > > if it gets an UnknownTopicException, it can explicitly issue a
> >> > > > > createTopicRequest for auto topic creation. On the consumer
> side,
> >> it
> >> > > will
> >> > > > > never issue createTopicRequest. This works when auto topic
> >> creation is
> >> > > > > enabled on the broker side. However, I am not sure how things
> >> will work
> >> > > > > when auto topic creation is disabled on the broker side. In this
> >> case,
> >> > > we
> >> > > > > want to have a way to manually create a topic, potentially
> through
> >> > > admin
> >> > > > > commands. However, then we need a way to distinguish
> >> createTopicRequest
> >> > > > > issued from the producer clients and the admin tools. May be we
> >> can
> >> > > add a
> >> > > > > new field in createTopicRequest and set it differently in the
> >> producer
> >> > > > > client and the admin client. However, I am not sure if that's
> the
> >> best
> >> > > > > approach.
> >> > > > >
> >> > > > > 2. Yes, refactoring existing requests is a non-trivial amount of
> >> work.
> >> > > I
> >> > > > > posted some comments in KAFKA-1927. We will probably have to fix
> >> > > > KAFKA-1927
> >> > > > > first, before adding the new logic in KAFKA-1694. Otherwise, the
> >> > > changes
> >> > > > > will be too big.
> >> > > > >
> >> > > > > 102. About the AdminClient:
> >> > > > > 102.1. It's a bit weird that we return exception in the api. It
> >> seems
> >> > > > that
> >> > > > > we should either return error code or throw an exception when
> >> getting
> >> > > the
> >> > > > > response state.
> >> > > > > 102.2. We probably shouldn't explicitly use the request object
> in
> >> the
> >> > > > api.
> >> > > > > Not every request evolution requires an api change.
> >> > > > >
> >> > > > > Thanks,
> >> > > > >
> >> > > > > Jun
> >> > > > >
> >> > > > >
> >> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> >> > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > >
> >> > > > > > Jun,
> >> > > > > >
> >> > > > > > Thanks for you comments. Answers inline:
> >> > > > > >
> >> > > > > > 100. There are a few fields such as ReplicaAssignment,
> >> > > > > > > ReassignPartitionRequest,
> >> > > > > > > and PartitionsSerialized that are represented as a string,
> but
> >> > > > contain
> >> > > > > > > composite structures in json. Could we flatten them out
> >> directly in
> >> > > > the
> >> > > > > > > protocol definition as arrays/records?
> >> > > > > >
> >> > > > > >
> >> > > > > > Yes, now with Admin Client this looks a bit weird. My initial
> >> > > > motivation
> >> > > > > > was:
> >> > > > > > ReassignPartitionCommand accepts input in json, we want to
> >> remain
> >> > > > tools'
> >> > > > > > interfaces unchanged, where possible.
> >> > > > > > If we port it to deserialized format, in CLI (/tools project)
> >> we will
> >> > > > > have
> >> > > > > > to add some
> >> > > > > > json library since /tools is written in java and we'll need to
> >> > > > > deserialize
> >> > > > > > json file
> >> > > > > > provided by a user. Can we quickly agree on what this library
> >> should
> >> > > be
> >> > > > > > (Jackson, GSON, whatever)?
> >> > > > > >
> >> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> >> creation?
> >> > > > This
> >> > > > > > > will be a bit weird now that we have a separate topic
> >> creation api.
> >> > > > > Have
> >> > > > > > > you thought about how the new createTopicRequest and
> >> > > > > TopicMetadataRequest
> >> > > > > > > v1 will be used in the producer/consumer client, in addition
> >> to
> >> > > admin
> >> > > > > > > tools? For example, ideally, we don't want
> >> TopicMetadataRequest
> >> > > from
> >> > > > > the
> >> > > > > > > consumer to trigger auto topic creation.
> >> > > > > >
> >> > > > > >
> >> > > > > > I agree, this strange logic should be fixed. I'm not confident
> >> in
> >> > > this
> >> > > > > > Kafka part so
> >> > > > > > correct me if I'm wrong, but it doesn't look like a hard thing
> >> to
> >> > > do, I
> >> > > > > > think we can
> >> > > > > > leverage AdminClient for that in Producer and unconditionally
> >> remove
> >> > > > > topic
> >> > > > > > creation from the TopicMetadataRequest_V1.
> >> > > > > >
> >> > > > > > 2. I think Jay meant getting rid of scala classes
> >> > > > > > > like HeartbeatRequestAndHeader and
> >> HeartbeatResponseAndHeader. We
> >> > > did
> >> > > > > > that
> >> > > > > > > as a stop-gap thing when adding the new requests for the
> >> consumers.
> >> > > > > > > However, the long term plan is to get rid of all those and
> >> just
> >> > > reuse
> >> > > > > the
> >> > > > > > > java request/response in the client. Since this KIP proposes
> >> to
> >> > > add a
> >> > > > > > > significant number of new requests, perhaps we should bite
> the
> >> > > bullet
> >> > > > > to
> >> > > > > > > clean up the existing scala requests first before adding new
> >> ones?
> >> > > > > > >
> >> > > > > >
> >> > > > > > Yes, looks like I misunderstood the point of
> >> ...RequestAndHeader.
> >> > > > Okay, I
> >> > > > > > will
> >> > > > > > rework that. The only thing is that I don't see any example
> how
> >> it
> >> > > was
> >> > > > > done
> >> > > > > > for at
> >> > > > > > least one existing protocol message. Thus, as I understand, I
> >> have to
> >> > > > > think
> >> > > > > > how we
> >> > > > > > are going to do it.
> >> > > > > > Re porting all existing RQ/RP in this patch. Sounds
> reasonable,
> >> but
> >> > > if
> >> > > > > it's
> >> > > > > > an *obligatory*
> >> > > > > > requirement to have Admin KIP done, I'm afraid this can be a
> >> serious
> >> > > > > > blocker for us.
> >> > > > > > There are 13 protocol messages and all that would require not
> >> only
> >> > > unit
> >> > > > > > tests but quite
> >> > > > > > intensive manual testing, no? I'm afraid I'm not the right guy
> >> to
> >> > > cover
> >> > > > > > pretty much all
> >> > > > > > Kafka core internals :). Let me know your thoughts on this
> >> item. Btw
> >> > > > > there
> >> > > > > > is a ticket to
> >> > > > > > follow-up this issue (
> >> > > https://issues.apache.org/jira/browse/KAFKA-2006
> >> > > > ).
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > > Andrii Biletskyi
> >> > > > > >
> >> > > > > >
> >> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <ju...@confluent.io>
> >> wrote:
> >> > > > > >
> >> > > > > > > Andrii,
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > A few more comments.
> >> > > > > > >
> >> > > > > > > 100. There are a few fields such as ReplicaAssignment,
> >> > > > > > > ReassignPartitionRequest,
> >> > > > > > > and PartitionsSerialized that are represented as a string,
> but
> >> > > > contain
> >> > > > > > > composite structures in json. Could we flatten them out
> >> directly in
> >> > > > the
> >> > > > > > > protocol definition as arrays/records?
> >> > > > > > >
> >> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> >> > > creation?
> >> > > > > This
> >> > > > > > > will be a bit weird now that we have a separate topic
> >> creation api.
> >> > > > > Have
> >> > > > > > > you thought about how the new createTopicRequest and
> >> > > > > TopicMetadataRequest
> >> > > > > > > v1 will be used in the producer/consumer client, in addition
> >> to
> >> > > admin
> >> > > > > > > tools? For example, ideally, we don't want
> >> TopicMetadataRequest
> >> > > from
> >> > > > > the
> >> > > > > > > consumer to trigger auto topic creation.
> >> > > > > > >
> >> > > > > > > 2. I think Jay meant getting rid of scala classes
> >> > > > > > > like HeartbeatRequestAndHeader and
> >> HeartbeatResponseAndHeader. We
> >> > > did
> >> > > > > > that
> >> > > > > > > as a stop-gap thing when adding the new requests for the
> >> consumers.
> >> > > > > > > However, the long term plan is to get rid of all those and
> >> just
> >> > > reuse
> >> > > > > the
> >> > > > > > > java request/response in the client. Since this KIP proposes
> >> to
> >> > > add a
> >> > > > > > > significant number of new requests, perhaps we should bite
> the
> >> > > bullet
> >> > > > > to
> >> > > > > > > clean up the existing scala requests first before adding new
> >> ones?
> >> > > > > > >
> >> > > > > > > Thanks,
> >> > > > > > >
> >> > > > > > > Jun
> >> > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> >> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > >
> >> > > > > > > > Hi,
> >> > > > > > > >
> >> > > > > > > > As said above - I list again all comments from this thread
> >> so we
> >> > > > > > > > can see what's left and finalize all pending issues.
> >> > > > > > > >
> >> > > > > > > > Comments from Jay:
> >> > > > > > > > 1. This is much needed functionality, but there are a lot
> >> of the
> >> > > so
> >> > > > > > let's
> >> > > > > > > > really think these protocols through. We really want to
> end
> >> up
> >> > > > with a
> >> > > > > > set
> >> > > > > > > > of well thought-out, orthoganol apis. For this reason I
> >> think it
> >> > > is
> >> > > > > > > really
> >> > > > > > > > important to think through the end state even if that
> >> includes
> >> > > APIs
> >> > > > > we
> >> > > > > > > > won't implement in the first phase.
> >> > > > > > > >
> >> > > > > > > > A: Definitely behind this. Would appreciate if there are
> >> concrete
> >> > > > > > > comments
> >> > > > > > > > how this can be improved.
> >> > > > > > > >
> >> > > > > > > > 2. Let's please please please wait until we have switched
> >> the
> >> > > > server
> >> > > > > > over
> >> > > > > > > > to the new java protocol definitions. If we add upteen
> more
> >> ad
> >> > > hoc
> >> > > > > > scala
> >> > > > > > > > objects that is just generating more work for the
> >> conversion we
> >> > > > know
> >> > > > > we
> >> > > > > > > > have to do.
> >> > > > > > > >
> >> > > > > > > > A: Fixed in the latest patch - removed scala protocol
> >> classes.
> >> > > > > > > >
> >> > > > > > > > 3. This proposal introduces a new type of optional
> >> parameter.
> >> > > This
> >> > > > is
> >> > > > > > > > inconsistent with everything else in the protocol where we
> >> use -1
> >> > > > or
> >> > > > > > some
> >> > > > > > > > other marker value. You could argue either way but let's
> >> stick
> >> > > with
> >> > > > > > that
> >> > > > > > > > for consistency. For clients that implemented the protocol
> >> in a
> >> > > > > better
> >> > > > > > > way
> >> > > > > > > > than our scala code these basic primitives are hard to
> >> change.
> >> > > > > > > >
> >> > > > > > > > A: Fixed in the latest patch - removed MaybeOf type and
> >> changed
> >> > > > > > protocol
> >> > > > > > > > accordingly.
> >> > > > > > > >
> >> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> >> TopicMetadataRequest
> >> > > > > which
> >> > > > > > > has
> >> > > > > > > > brokers, topics, and partitions. I think we should rename
> >> that
> >> > > > > request
> >> > > > > > > > ClusterMetadataRequest (or just MetadataRequest) and
> >> include the
> >> > > id
> >> > > > > of
> >> > > > > > > the
> >> > > > > > > > controller. Or are there other things we could add here?
> >> > > > > > > >
> >> > > > > > > > A: I agree. Updated the KIP. Let's extends TopicMetadata
> to
> >> > > > version 2
> >> > > > > > and
> >> > > > > > > > include controller.
> >> > > > > > > >
> >> > > > > > > > 5. We have a tendency to try to make a lot of requests
> that
> >> can
> >> > > > only
> >> > > > > go
> >> > > > > > > to
> >> > > > > > > > particular nodes. This adds a lot of burden for client
> >> > > > > implementations
> >> > > > > > > (it
> >> > > > > > > > sounds easy but each discovery can fail in many parts so
> it
> >> ends
> >> > > up
> >> > > > > > > being a
> >> > > > > > > > full state machine to do right). I think we should
> consider
> >> > > making
> >> > > > > > admin
> >> > > > > > > > commands and ideally as many of the other apis as possible
> >> > > > available
> >> > > > > on
> >> > > > > > > all
> >> > > > > > > > brokers and just redirect to the controller on the broker
> >> side.
> >> > > > > Perhaps
> >> > > > > > > > there would be a general way to encapsulate this
> re-routing
> >> > > > behavior.
> >> > > > > > > >
> >> > > > > > > > A: It's a very interesting idea, but seems there are some
> >> > > concerns
> >> > > > > > about
> >> > > > > > > > this
> >> > > > > > > > feature (like performance considerations, how this will
> >> > > complicate
> >> > > > > > server
> >> > > > > > > > etc).
> >> > > > > > > > I believe this shouldn't be a blocker. If this feature is
> >> > > > implemented
> >> > > > > > at
> >> > > > > > > > some
> >> > > > > > > > point it won't affect Admin changes - at least no changes
> to
> >> > > public
> >> > > > > API
> >> > > > > > > > will be required.
> >> > > > > > > >
> >> > > > > > > > 6. We should probably normalize the key value pairs used
> for
> >> > > > configs
> >> > > > > > > rather
> >> > > > > > > > than embedding a new formatting. So two strings rather
> than
> >> one
> >> > > > with
> >> > > > > an
> >> > > > > > > > internal equals sign.
> >> > > > > > > >
> >> > > > > > > > A: Fixed in the latest patch - normalized configs and
> >> changed
> >> > > > > protocol
> >> > > > > > > > accordingly.
> >> > > > > > > >
> >> > > > > > > > 7. Is the postcondition of these APIs that the command has
> >> begun
> >> > > or
> >> > > > > > that
> >> > > > > > > > the command has been completed? It is a lot more usable if
> >> the
> >> > > > > command
> >> > > > > > > has
> >> > > > > > > > been completed so you know that if you create a topic and
> >> then
> >> > > > > publish
> >> > > > > > to
> >> > > > > > > > it you won't get an exception about there being no such
> >> topic.
> >> > > > > > > >
> >> > > > > > > > A: For long running requests (like reassign partitions) -
> >> the
> >> > > post
> >> > > > > > > > condition is
> >> > > > > > > > command has begun - so we don't block the client. In case
> >> of your
> >> > > > > > > example -
> >> > > > > > > > topic commands, this will be refactored and topic commands
> >> will
> >> > > be
> >> > > > > > > executed
> >> > > > > > > > immediately, since the Controller will serve Admin
> requests
> >> > > > > > > > (follow-up ticket KAFKA-1777).
> >> > > > > > > >
> >> > > > > > > > 8. Describe topic and list topics duplicate a lot of stuff
> >> in the
> >> > > > > > > metadata
> >> > > > > > > > request. Is there a reason to give back topics marked for
> >> > > > deletion? I
> >> > > > > > > feel
> >> > > > > > > > like if we just make the post-condition of the delete
> >> command be
> >> > > > that
> >> > > > > > the
> >> > > > > > > > topic is deleted that will get rid of the need for this
> >> right?
> >> > > And
> >> > > > it
> >> > > > > > > will
> >> > > > > > > > be much more intuitive.
> >> > > > > > > >
> >> > > > > > > > A: Fixed in the latest patch - removed topics marked for
> >> deletion
> >> > > > in
> >> > > > > > > > ListTopicsRequest.
> >> > > > > > > >
> >> > > > > > > > 9. Should we consider batching these requests? We have
> >> generally
> >> > > > > tried
> >> > > > > > to
> >> > > > > > > > allow multiple operations to be batched. My suspicion is
> >> that
> >> > > > without
> >> > > > > > > this
> >> > > > > > > > we will get a lot of code that does something like
> >> > > > > > > >    for(topic: adminClient.listTopics())
> >> > > > > > > >       adminClient.describeTopic(topic)
> >> > > > > > > > this code will work great when you test on 5 topics but
> not
> >> do as
> >> > > > > well
> >> > > > > > if
> >> > > > > > > > you have 50k.
> >> > > > > > > >
> >> > > > > > > > A: Updated the KIP - please check "Topic Admin Schema"
> >> section.
> >> > > > > > > >
> >> > > > > > > > 10. I think we should also discuss how we want to expose a
> >> > > > > programmatic
> >> > > > > > > JVM
> >> > > > > > > > client api for these operations. Currently people rely on
> >> > > > AdminUtils
> >> > > > > > > which
> >> > > > > > > > is totally sketchy. I think we probably need another
> client
> >> under
> >> > > > > > > clients/
> >> > > > > > > > that exposes administrative functionality. We will need
> >> this just
> >> > > > to
> >> > > > > > > > properly test the new apis, I suspect. We should figure
> out
> >> that
> >> > > > API.
> >> > > > > > > >
> >> > > > > > > > A: Updated the KIP - please check "Admin Client" section
> >> with an
> >> > > > > > initial
> >> > > > > > > > API proposal.
> >> > > > > > > >
> >> > > > > > > > 11. The other information that would be really useful to
> get
> >> > > would
> >> > > > be
> >> > > > > > > > information about partitions--how much data is in the
> >> partition,
> >> > > > what
> >> > > > > > are
> >> > > > > > > > the segment offsets, what is the log-end offset (i.e. last
> >> > > offset),
> >> > > > > > what
> >> > > > > > > is
> >> > > > > > > > the compaction point, etc. I think that done right this
> >> would be
> >> > > > the
> >> > > > > > > > successor to the very awkward OffsetRequest we have today.
> >> > > > > > > >
> >> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest
> >> patch. I
> >> > > > > believe
> >> > > > > > > > this should
> >> > > > > > > > be resolved in a separate KIP / jira ticket.
> >> > > > > > > >
> >> > > > > > > > 12. Generally we can do good error handling without
> needing
> >> > > custom
> >> > > > > > > > server-side
> >> > > > > > > > messages. I.e. generally the client has the context to
> know
> >> that
> >> > > if
> >> > > > > it
> >> > > > > > > got
> >> > > > > > > > an error that the topic doesn't exist to say "Topic X
> >> doesn't
> >> > > > exist"
> >> > > > > > > rather
> >> > > > > > > > than "error code 14" (or whatever). Maybe there are
> specific
> >> > > cases
> >> > > > > > where
> >> > > > > > > > this is hard? If we want to add server-side error messages
> >> we
> >> > > > really
> >> > > > > do
> >> > > > > > > > need to do this in a consistent way across the protocol.
> >> > > > > > > >
> >> > > > > > > > A: Updated the KIP - please check "Protocol Errors"
> >> section. I
> >> > > > added
> >> > > > > > the
> >> > > > > > > > comprehensive, fine-grained list of error codes.
> >> > > > > > > >
> >> > > > > > > > Comments from Guozhang:
> >> > > > > > > > 13. Describe topic request: it would be great to go beyond
> >> just
> >> > > > > > batching
> >> > > > > > > on
> >> > > > > > > > topic name regex for this request. For example, a very
> >> common use
> >> > > > > case
> >> > > > > > of
> >> > > > > > > > the topic command is to list all topics whose config A's
> >> value is
> >> > > > B.
> >> > > > > > With
> >> > > > > > > > topic name regex then we have to first retrieve __all__
> >> topics's
> >> > > > > > > > description info and then filter at the client end, which
> >> will
> >> > > be a
> >> > > > > > huge
> >> > > > > > > > burden on ZK.
> >> > > > > > > > AND
> >> > > > > > > > 14. Config K-Vs in create topic: this is related to the
> >> previous
> >> > > > > point;
> >> > > > > > > > maybe we can add another metadata K-V or just a metadata
> >> string
> >> > > > along
> >> > > > > > > side
> >> > > > > > > > with config K-V in create topic like we did for offset
> >> commit
> >> > > > > request.
> >> > > > > > > This
> >> > > > > > > > field can be quite useful in storing information like
> >> "owner" of
> >> > > > the
> >> > > > > > > topic
> >> > > > > > > > who issue the create command, etc, which is quite
> important
> >> for a
> >> > > > > > > > multi-tenant setting. Then in the describe topic request
> we
> >> can
> >> > > > also
> >> > > > > > > batch
> >> > > > > > > > on regex of the metadata field.
> >> > > > > > > >
> >> > > > > > > > A: As discussed it is very interesting but can be
> >> implemented
> >> > > later
> >> > > > > > after
> >> > > > > > > > we have some basic functionality there.
> >> > > > > > > >
> >> > > > > > > > 15. Today all the admin operations are async in the sense
> >> that
> >> > > > > command
> >> > > > > > > will
> >> > > > > > > > return once it is written in ZK, and that is why we need
> >> extra
> >> > > > > > > verification
> >> > > > > > > > like testUtil.waitForTopicCreated() / verify partition
> >> > > reassignment
> >> > > > > > > > request, etc. With admin requests we could add a flag to
> >> enable /
> >> > > > > > disable
> >> > > > > > > > synchronous requests; when it is turned on, the response
> >> will not
> >> > > > > > return
> >> > > > > > > > until the request has been completed. And for async
> >> requests we
> >> > > can
> >> > > > > > add a
> >> > > > > > > > "token" field in the response, and then only need a
> general
> >> > > "admin
> >> > > > > > > > verification request" with the given token to check if the
> >> async
> >> > > > > > request
> >> > > > > > > > has been completed.
> >> > > > > > > >
> >> > > > > > > > A: I see your point. My idea was to provide specific
> >> > > > Verify...Request
> >> > > > > > per
> >> > > > > > > > each
> >> > > > > > > > long running request, where needed. We can do it the way
> you
> >> > > > suggest.
> >> > > > > > The
> >> > > > > > > > only
> >> > > > > > > > concern is that introducing a token we again will make
> >> schema
> >> > > > > > "dynamic".
> >> > > > > > > We
> >> > > > > > > > wanted
> >> > > > > > > > to do similar thing introducing single AdminRequest for
> all
> >> topic
> >> > > > > > > commands
> >> > > > > > > > but rejected
> >> > > > > > > > this idea because we wanted to have schema defined. So
> this
> >> is
> >> > > > more a
> >> > > > > > > > choice between:
> >> > > > > > > > a) have fixed schema but introduce each time new
> >> Verify...Request
> >> > > > for
> >> > > > > > > > long-running requests
> >> > > > > > > > b) use one request for verification but generalize it with
> >> token
> >> > > > > > > > I'm fine with whatever decision community come to. Just
> let
> >> me
> >> > > know
> >> > > > > > your
> >> > > > > > > > thoughts.
> >> > > > > > > >
> >> > > > > > > > Comment from Gwen:
> >> > > > > > > > 16. Specifically for ownership, I think the plan is to add
> >> ACL
> >> > > (it
> >> > > > > > sounds
> >> > > > > > > > like you are describing ACL) via an external system
> (Argus,
> >> > > > Sentry).
> >> > > > > > > > I remember KIP-11 described this, but I can't find the KIP
> >> any
> >> > > > > longer.
> >> > > > > > > >
> >> > > > > > > > A: Okay, no problem. Not sure though how we are going to
> >> handle
> >> > > it.
> >> > > > > > Wait
> >> > > > > > > > which KIP
> >> > > > > > > > will be committed first and include changes to
> >> TopicMetadata from
> >> > > > the
> >> > > > > > > later
> >> > > > > > > > one?
> >> > > > > > > > Anyway, I added this note to "Open Questions" section so
> we
> >> don't
> >> > > > > miss
> >> > > > > > > this
> >> > > > > > > > piece.
> >> > > > > > > >
> >> > > > > > > > Thanks,
> >> > > > > > > > Andrii Biletskyi
> >> > > > > > > >
> >> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> >> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > > >
> >> > > > > > > > > Hi all,
> >> > > > > > > > >
> >> > > > > > > > > Today I uploaded the patch that covers some of the
> >> discussed
> >> > > and
> >> > > > > > agreed
> >> > > > > > > > > items:
> >> > > > > > > > > - removed MaybeOf optional type
> >> > > > > > > > > - switched to java protocol definitions
> >> > > > > > > > > - simplified messages (normalized configs, removed topic
> >> marked
> >> > > > for
> >> > > > > > > > > deletion)
> >> > > > > > > > >
> >> > > > > > > > > I also updated the KIP-4 with respective changes and
> >> wrote down
> >> > > > my
> >> > > > > > > > > proposal for
> >> > > > > > > > > pending items:
> >> > > > > > > > > - Batch Admin Operations -> updated Wire Protocol schema
> >> > > proposal
> >> > > > > > > > > - Remove ClusterMetadata -> changed to extend
> >> > > > TopicMetadataRequest
> >> > > > > > > > > - Admin Client -> updated my initial proposal to reflect
> >> > > batching
> >> > > > > > > > > - Error codes -> proposed fine-grained error code
> instead
> >> of
> >> > > > > > > > > AdminRequestFailed
> >> > > > > > > > >
> >> > > > > > > > > I will also send a separate email to cover all comments
> >> from
> >> > > this
> >> > > > > > > thread.
> >> > > > > > > > >
> >> > > > > > > > > Thanks,
> >> > > > > > > > > Andrii Biletskyi
> >> > > > > > > > >
> >> > > > > > > > >
> >> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> >> > > > > gshapira@cloudera.com
> >> > > > > > >
> >> > > > > > > > > wrote:
> >> > > > > > > > >
> >> > > > > > > > >> Found KIP-11 (
> >> > > > > > > > >>
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> >> > > > > > > > >> )
> >> > > > > > > > >> It actually specifies changes to the Metadata protocol,
> >> so
> >> > > > making
> >> > > > > > sure
> >> > > > > > > > >> both KIPs are consistent in this regard will be good.
> >> > > > > > > > >>
> >> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> >> > > > > > gshapira@cloudera.com
> >> > > > > > > >
> >> > > > > > > > >> wrote:
> >> > > > > > > > >> > Specifically for ownership, I think the plan is to
> add
> >> ACL
> >> > > (it
> >> > > > > > > sounds
> >> > > > > > > > >> > like you are describing ACL) via an external system
> >> (Argus,
> >> > > > > > Sentry).
> >> > > > > > > > >> > I remember KIP-11 described this, but I can't find
> the
> >> KIP
> >> > > any
> >> > > > > > > longer.
> >> > > > > > > > >> >
> >> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting
> >> information
> >> > > that
> >> > > > > > > already
> >> > > > > > > > >> > exists from Kafka brokers, not on adding information
> >> that
> >> > > > > perhaps
> >> > > > > > > > >> > should exist but doesn't yet?
> >> > > > > > > > >> >
> >> > > > > > > > >> > Gwen
> >> > > > > > > > >> >
> >> > > > > > > > >> >
> >> > > > > > > > >> >
> >> > > > > > > > >> >
> >> > > > > > > > >> >
> >> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> >> > > > > > wangguoz@gmail.com>
> >> > > > > > > > >> wrote:
> >> > > > > > > > >> >> Folks,
> >> > > > > > > > >> >>
> >> > > > > > > > >> >> Just want to elaborate a bit more on the
> create-topic
> >> > > > metadata
> >> > > > > > and
> >> > > > > > > > >> batching
> >> > > > > > > > >> >> describe-topic based on config / metadata in my
> >> previous
> >> > > > email
> >> > > > > as
> >> > > > > > > we
> >> > > > > > > > >> work
> >> > > > > > > > >> >> on KAFKA-1694. The main motivation is to have some
> >> sort of
> >> > > > > topic
> >> > > > > > > > >> management
> >> > > > > > > > >> >> mechanisms, which I think is quite important in a
> >> > > > multi-tenant
> >> > > > > /
> >> > > > > > > > cloud
> >> > > > > > > > >> >> architecture: today anyone can create topics in a
> >> shared
> >> > > > Kafka
> >> > > > > > > > >> cluster, but
> >> > > > > > > > >> >> there is no concept or "ownership" of topics that
> are
> >> > > created
> >> > > > > by
> >> > > > > > > > >> different
> >> > > > > > > > >> >> users. For example, at LinkedIn we basically
> >> distinguish
> >> > > > topic
> >> > > > > > > owners
> >> > > > > > > > >> via
> >> > > > > > > > >> >> some casual topic name prefix, which is a bit
> awkward
> >> and
> >> > > > does
> >> > > > > > not
> >> > > > > > > > fly
> >> > > > > > > > >> as
> >> > > > > > > > >> >> we scale our customers. It would be great to use
> >> > > > > describe-topics
> >> > > > > > > such
> >> > > > > > > > >> as:
> >> > > > > > > > >> >>
> >> > > > > > > > >> >> Describe all topics that is created by me.
> >> > > > > > > > >> >>
> >> > > > > > > > >> >> Describe all topics whose retention time is
> overriden
> >> to X.
> >> > > > > > > > >> >>
> >> > > > > > > > >> >> Describe all topics whose writable group include
> user
> >> Y
> >> > > (this
> >> > > > > is
> >> > > > > > > > >> related to
> >> > > > > > > > >> >> authorization), etc..
> >> > > > > > > > >> >>
> >> > > > > > > > >> >> One possible way to achieve this is to add a
> metadata
> >> file
> >> > > in
> >> > > > > the
> >> > > > > > > > >> >> create-topic request, whose value will also be
> >> written ZK
> >> > > as
> >> > > > we
> >> > > > > > > > create
> >> > > > > > > > >> the
> >> > > > > > > > >> >> topic; then describe-topics can choose to batch
> topics
> >> > > based
> >> > > > on
> >> > > > > > 1)
> >> > > > > > > > name
> >> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata regex,
> etc.
> >> > > > > > > > >> >>
> >> > > > > > > > >> >> Thoughts?
> >> > > > > > > > >> >>
> >> > > > > > > > >> >> Guozhang
> >> > > > > > > > >> >>
> >> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> >> > > > > > wangguoz@gmail.com>
> >> > > > > > > > >> wrote:
> >> > > > > > > > >> >>
> >> > > > > > > > >> >>> Thanks for the updated wiki. A few comments below:
> >> > > > > > > > >> >>>
> >> > > > > > > > >> >>> 1. Error description in response: I think if some
> >> > > errorCode
> >> > > > > > could
> >> > > > > > > > >> indicate
> >> > > > > > > > >> >>> several different error cases then we should really
> >> change
> >> > > > it
> >> > > > > to
> >> > > > > > > > >> multiple
> >> > > > > > > > >> >>> codes. In general the errorCode itself would be
> >> precise
> >> > > and
> >> > > > > > > > >> sufficient for
> >> > > > > > > > >> >>> describing the server side errors.
> >> > > > > > > > >> >>>
> >> > > > > > > > >> >>> 2. Describe topic request: it would be great to go
> >> beyond
> >> > > > just
> >> > > > > > > > >> batching on
> >> > > > > > > > >> >>> topic name regex for this request. For example, a
> >> very
> >> > > > common
> >> > > > > > use
> >> > > > > > > > >> case of
> >> > > > > > > > >> >>> the topic command is to list all topics whose
> config
> >> A's
> >> > > > value
> >> > > > > > is
> >> > > > > > > B.
> >> > > > > > > > >> With
> >> > > > > > > > >> >>> topic name regex then we have to first retrieve
> >> __all__
> >> > > > > topics's
> >> > > > > > > > >> >>> description info and then filter at the client end,
> >> which
> >> > > > will
> >> > > > > > be
> >> > > > > > > a
> >> > > > > > > > >> huge
> >> > > > > > > > >> >>> burden on ZK.
> >> > > > > > > > >> >>>
> >> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is related to
> >> the
> >> > > > > previous
> >> > > > > > > > point;
> >> > > > > > > > >> >>> maybe we can add another metadata K-V or just a
> >> metadata
> >> > > > > string
> >> > > > > > > > along
> >> > > > > > > > >> side
> >> > > > > > > > >> >>> with config K-V in create topic like we did for
> >> offset
> >> > > > commit
> >> > > > > > > > >> request. This
> >> > > > > > > > >> >>> field can be quite useful in storing information
> like
> >> > > > "owner"
> >> > > > > of
> >> > > > > > > the
> >> > > > > > > > >> topic
> >> > > > > > > > >> >>> who issue the create command, etc, which is quite
> >> > > important
> >> > > > > for
> >> > > > > > a
> >> > > > > > > > >> >>> multi-tenant setting. Then in the describe topic
> >> request
> >> > > we
> >> > > > > can
> >> > > > > > > also
> >> > > > > > > > >> batch
> >> > > > > > > > >> >>> on regex of the metadata field.
> >> > > > > > > > >> >>>
> >> > > > > > > > >> >>> 4. Today all the admin operations are async in the
> >> sense
> >> > > > that
> >> > > > > > > > command
> >> > > > > > > > >> will
> >> > > > > > > > >> >>> return once it is written in ZK, and that is why we
> >> need
> >> > > > extra
> >> > > > > > > > >> verification
> >> > > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify
> >> partition
> >> > > > > > > reassignment
> >> > > > > > > > >> >>> request, etc. With admin requests we could add a
> >> flag to
> >> > > > > enable
> >> > > > > > /
> >> > > > > > > > >> disable
> >> > > > > > > > >> >>> synchronous requests; when it is turned on, the
> >> response
> >> > > > will
> >> > > > > > not
> >> > > > > > > > >> return
> >> > > > > > > > >> >>> until the request has been completed. And for async
> >> > > requests
> >> > > > > we
> >> > > > > > > can
> >> > > > > > > > >> add a
> >> > > > > > > > >> >>> "token" field in the response, and then only need a
> >> > > general
> >> > > > > > "admin
> >> > > > > > > > >> >>> verification request" with the given token to check
> >> if the
> >> > > > > async
> >> > > > > > > > >> request
> >> > > > > > > > >> >>> has been completed.
> >> > > > > > > > >> >>>
> >> > > > > > > > >> >>> 5. +1 for extending Metadata request to include
> >> > > controller /
> >> > > > > > > > >> coordinator
> >> > > > > > > > >> >>> information, and then we can remove the
> >> ConsumerMetadata /
> >> > > > > > > > >> ClusterMetadata
> >> > > > > > > > >> >>> requests.
> >> > > > > > > > >> >>>
> >> > > > > > > > >> >>> Guozhang
> >> > > > > > > > >> >>>
> >> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> >> > > > > > jjkoshy.w@gmail.com>
> >> > > > > > > > >> wrote:
> >> > > > > > > > >> >>>
> >> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't think I
> >> will be
> >> > > > > able
> >> > > > > > to
> >> > > > > > > > >> make
> >> > > > > > > > >> >>>> it today, so if notes can be sent out afterward
> that
> >> > > would
> >> > > > be
> >> > > > > > > > great.
> >> > > > > > > > >> >>>>
> >> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen
> >> Shapira
> >> > > > wrote:
> >> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking forward
> >> to
> >> > > > > chatting
> >> > > > > > > with
> >> > > > > > > > >> >>>> everyone :)
> >> > > > > > > > >> >>>> >
> >> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> >> > > > > > > joe.stein@stealth.ly>
> >> > > > > > > > >> wrote:
> >> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout invite
> to
> >> all
> >> > > > pmc,
> >> > > > > > > > >> committers
> >> > > > > > > > >> >>>> and
> >> > > > > > > > >> >>>> > > everyone I found working on a KIP. If I missed
> >> anyone
> >> > > > in
> >> > > > > > the
> >> > > > > > > > >> invite
> >> > > > > > > > >> >>>> please
> >> > > > > > > > >> >>>> > > let me know and can update it, np.
> >> > > > > > > > >> >>>> > >
> >> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm Eastern
> >> Time.
> >> > > > Maybe
> >> > > > > > we
> >> > > > > > > > can
> >> > > > > > > > >> get
> >> > > > > > > > >> >>>> INFRA
> >> > > > > > > > >> >>>> > > help to make a google account so we can manage
> >> > > better?
> >> > > > > > > > >> >>>> > >
> >> > > > > > > > >> >>>> > > To discuss
> >> > > > > > > > >> >>>> > >
> >> > > > > > > > >> >>>>
> >> > > > > > > > >>
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >> > > > > > > > >> >>>> > > in progress and related JIRA that are
> >> interdependent
> >> > > > and
> >> > > > > > > common
> >> > > > > > > > >> work.
> >> > > > > > > > >> >>>> > >
> >> > > > > > > > >> >>>> > > ~ Joe Stein
> >> > > > > > > > >> >>>> > >
> >> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
> >> > > > > > > > jay.kreps@gmail.com>
> >> > > > > > > > >> >>>> wrote:
> >> > > > > > > > >> >>>> > >
> >> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that will also
> >> record
> >> > > > and
> >> > > > > > make
> >> > > > > > > > the
> >> > > > > > > > >> >>>> sessions
> >> > > > > > > > >> >>>> > >> available on youtube.
> >> > > > > > > > >> >>>> > >>
> >> > > > > > > > >> >>>> > >> -Jay
> >> > > > > > > > >> >>>> > >>
> >> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff
> Holoman
> >> <
> >> > > > > > > > >> >>>> jholoman@cloudera.com>
> >> > > > > > > > >> >>>> > >> wrote:
> >> > > > > > > > >> >>>> > >>
> >> > > > > > > > >> >>>> > >> > Jay / Joe
> >> > > > > > > > >> >>>> > >> >
> >> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for this
> >> purpose.
> >> > > We
> >> > > > > > could
> >> > > > > > > > >> record
> >> > > > > > > > >> >>>> the
> >> > > > > > > > >> >>>> > >> > sessions if there is interest and publish
> >> them
> >> > > out.
> >> > > > > > > > >> >>>> > >> >
> >> > > > > > > > >> >>>> > >> > Thanks
> >> > > > > > > > >> >>>> > >> >
> >> > > > > > > > >> >>>> > >> > Jeff
> >> > > > > > > > >> >>>> > >> >
> >> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay
> Kreps <
> >> > > > > > > > >> jay.kreps@gmail.com>
> >> > > > > > > > >> >>>> wrote:
> >> > > > > > > > >> >>>> > >> >
> >> > > > > > > > >> >>>> > >> > > Let's try to get the technical hang-ups
> >> sorted
> >> > > > out,
> >> > > > > > > > though.
> >> > > > > > > > >> I
> >> > > > > > > > >> >>>> really
> >> > > > > > > > >> >>>> > >> > think
> >> > > > > > > > >> >>>> > >> > > there is some benefit to live discussion
> vs
> >> > > > > writing. I
> >> > > > > > > am
> >> > > > > > > > >> >>>> hopeful that
> >> > > > > > > > >> >>>> > >> if
> >> > > > > > > > >> >>>> > >> > > we post instructions and give ourselves a
> >> few
> >> > > > > attempts
> >> > > > > > > we
> >> > > > > > > > >> can
> >> > > > > > > > >> >>>> get it
> >> > > > > > > > >> >>>> > >> > > working.
> >> > > > > > > > >> >>>> > >> > >
> >> > > > > > > > >> >>>> > >> > > Tuesday at that time would work for
> >> me...any
> >> > > > > > objections?
> >> > > > > > > > >> >>>> > >> > >
> >> > > > > > > > >> >>>> > >> > > -Jay
> >> > > > > > > > >> >>>> > >> > >
> >> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe
> Stein
> >> <
> >> > > > > > > > >> joe.stein@stealth.ly
> >> > > > > > > > >> >>>> >
> >> > > > > > > > >> >>>> > >> wrote:
> >> > > > > > > > >> >>>> > >> > >
> >> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe like every
> >> > > Tuesday ~
> >> > > > > 1pm
> >> > > > > > > ET
> >> > > > > > > > /
> >> > > > > > > > >> 10am
> >> > > > > > > > >> >>>> PT
> >> > > > > > > > >> >>>> > >> ????
> >> > > > > > > > >> >>>> > >> > > >
> >> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but there
> is
> >> > > always
> >> > > > > some
> >> > > > > > > > >> issue or
> >> > > > > > > > >> >>>> > >> whatever
> >> > > > > > > > >> >>>> > >> > so
> >> > > > > > > > >> >>>> > >> > > > we know the apache irc channel works.
> We
> >> can
> >> > > > start
> >> > > > > > > there
> >> > > > > > > > >> and
> >> > > > > > > > >> >>>> see how
> >> > > > > > > > >> >>>> > >> it
> >> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too and
> >> > > associate
> >> > > > to
> >> > > > > > > > >> tickets if
> >> > > > > > > > >> >>>> need be
> >> > > > > > > > >> >>>> > >> > > makes
> >> > > > > > > > >> >>>> > >> > > > it helpful for things.
> >> > > > > > > > >> >>>> > >> > > >
> >> > > > > > > > >> >>>> > >> > > > ~ Joestein
> >> > > > > > > > >> >>>> > >> > > >
> >> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay
> >> Kreps <
> >> > > > > > > > >> >>>> jay.kreps@gmail.com>
> >> > > > > > > > >> >>>> > >> > wrote:
> >> > > > > > > > >> >>>> > >> > > >
> >> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a Google
> >> Hangout to
> >> > > > chat
> >> > > > > > > about
> >> > > > > > > > >> this.
> >> > > > > > > > >> >>>> What
> >> > > > > > > > >> >>>> > >> > about
> >> > > > > > > > >> >>>> > >> > > > > generalizing that a little
> further...I
> >> > > > actually
> >> > > > > > > think
> >> > > > > > > > it
> >> > > > > > > > >> >>>> would be
> >> > > > > > > > >> >>>> > >> > good
> >> > > > > > > > >> >>>> > >> > > > for
> >> > > > > > > > >> >>>> > >> > > > > everyone spending a reasonable chunk
> of
> >> > > their
> >> > > > > week
> >> > > > > > > on
> >> > > > > > > > >> Kafka
> >> > > > > > > > >> >>>> stuff
> >> > > > > > > > >> >>>> > >> to
> >> > > > > > > > >> >>>> > >> > > > maybe
> >> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think we could
> >> use
> >> > > time
> >> > > > > to
> >> > > > > > > talk
> >> > > > > > > > >> >>>> through
> >> > > > > > > > >> >>>> > >> design
> >> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of
> code
> >> > > > reviews,
> >> > > > > > talk
> >> > > > > > > > >> through
> >> > > > > > > > >> >>>> any
> >> > > > > > > > >> >>>> > >> > tricky
> >> > > > > > > > >> >>>> > >> > > > > issues, etc.
> >> > > > > > > > >> >>>> > >> > > > >
> >> > > > > > > > >> >>>> > >> > > > > We can make it publicly available so
> >> that
> >> > > any
> >> > > > > one
> >> > > > > > > can
> >> > > > > > > > >> follow
> >> > > > > > > > >> >>>> along
> >> > > > > > > > >> >>>> > >> > who
> >> > > > > > > > >> >>>> > >> > > > > likes.
> >> > > > > > > > >> >>>> > >> > > > >
> >> > > > > > > > >> >>>> > >> > > > > Any interest in doing this? If so
> I'll
> >> try
> >> > > to
> >> > > > > set
> >> > > > > > it
> >> > > > > > > > up
> >> > > > > > > > >> >>>> starting
> >> > > > > > > > >> >>>> > >> next
> >> > > > > > > > >> >>>> > >> > > > week.
> >> > > > > > > > >> >>>> > >> > > > >
> >> > > > > > > > >> >>>> > >> > > > > -Jay
> >> > > > > > > > >> >>>> > >> > > > >
> >> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM,
> Andrii
> >> > > > > Biletskyi
> >> > > > > > <
> >> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > > > >> >>>> > >> > > > >
> >> > > > > > > > >> >>>> > >> > > > > > Hi all,
> >> > > > > > > > >> >>>> > >> > > > > >
> >> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed /
> >> aligned
> >> > > > > document
> >> > > > > > > > >> structure.
> >> > > > > > > > >> >>>> Also I
> >> > > > > > > > >> >>>> > >> > > added
> >> > > > > > > > >> >>>> > >> > > > > > some
> >> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
> >> AdminClient so
> >> > > we
> >> > > > > have
> >> > > > > > > > >> something
> >> > > > > > > > >> >>>> to
> >> > > > > > > > >> >>>> > >> start
> >> > > > > > > > >> >>>> > >> > > > from
> >> > > > > > > > >> >>>> > >> > > > > > while
> >> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> >> > > > > > > > >> >>>> > >> > > > > >
> >> > > > > > > > >> >>>> > >> > > > > >
> >> > > > > > > > >> >>>> > >> > > > > >
> >> > > > > > > > >> >>>> > >> > > > >
> >> > > > > > > > >> >>>> > >> > > >
> >> > > > > > > > >> >>>> > >> > >
> >> > > > > > > > >> >>>> > >> >
> >> > > > > > > > >> >>>> > >>
> >> > > > > > > > >> >>>>
> >> > > > > > > > >>
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> > > > > > > > >> >>>> > >> > > > > >
> >> > > > > > > > >> >>>> > >> > > > > > Thanks,
> >> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> >> > > > > > > > >> >>>> > >> > > > > >
> >> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM,
> >> Andrii
> >> > > > > > Biletskyi
> >> > > > > > > <
> >> > > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly>
> wrote:
> >> > > > > > > > >> >>>> > >> > > > > >
> >> > > > > > > > >> >>>> > >> > > > > > > Jay,
> >> > > > > > > > >> >>>> > >> > > > > > >
> >> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you are right,
> >> in
> >> > > most
> >> > > > > > cases
> >> > > > > > > > >> client
> >> > > > > > > > >> >>>> will
> >> > > > > > > > >> >>>> > >> have
> >> > > > > > > > >> >>>> > >> > > > enough
> >> > > > > > > > >> >>>> > >> > > > > > > context to show descriptive error
> >> > > message.
> >> > > > > My
> >> > > > > > > > >> concern is
> >> > > > > > > > >> >>>> that
> >> > > > > > > > >> >>>> > >> we
> >> > > > > > > > >> >>>> > >> > > will
> >> > > > > > > > >> >>>> > >> > > > > > have
> >> > > > > > > > >> >>>> > >> > > > > > > to
> >> > > > > > > > >> >>>> > >> > > > > > > add lots of new error codes for
> >> each
> >> > > > > possible
> >> > > > > > > > >> error. Of
> >> > > > > > > > >> >>>> course,
> >> > > > > > > > >> >>>> > >> > we
> >> > > > > > > > >> >>>> > >> > > > > could
> >> > > > > > > > >> >>>> > >> > > > > > > reuse
> >> > > > > > > > >> >>>> > >> > > > > > > some of existing like
> >> > > > > > > UknownTopicOrPartitionCode,
> >> > > > > > > > >> but we
> >> > > > > > > > >> >>>> will
> >> > > > > > > > >> >>>> > >> > also
> >> > > > > > > > >> >>>> > >> > > > need
> >> > > > > > > > >> >>>> > >> > > > > > to
> >> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> >> TopicAlreadyExistsCode,
> >> > > > > > > > >> >>>> TopicConfigInvalid (both
> >> > > > > > > > >> >>>> > >> > for
> >> > > > > > > > >> >>>> > >> > > > > topic
> >> > > > > > > > >> >>>> > >> > > > > > > name and config, and probably
> user
> >> would
> >> > > > > like
> >> > > > > > to
> >> > > > > > > > >> know
> >> > > > > > > > >> >>>> what
> >> > > > > > > > >> >>>> > >> > exactly
> >> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
> >> > > > > > > InvalidReplicaAssignment,
> >> > > > > > > > >> >>>> > >> InternalError
> >> > > > > > > > >> >>>> > >> > > > (e.g.
> >> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> >> > > > > > > > >> >>>> > >> > > > > > > And this is only for
> TopicCommand,
> >> we
> >> > > will
> >> > > > > > also
> >> > > > > > > > >> need to
> >> > > > > > > > >> >>>> add
> >> > > > > > > > >> >>>> > >> > similar
> >> > > > > > > > >> >>>> > >> > > > > stuff
> >> > > > > > > > >> >>>> > >> > > > > > > for
> >> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> >> PreferredReplica. So
> >> > > > > we'll
> >> > > > > > > end
> >> > > > > > > > >> up
> >> > > > > > > > >> >>>> with a
> >> > > > > > > > >> >>>> > >> > large
> >> > > > > > > > >> >>>> > >> > > > list
> >> > > > > > > > >> >>>> > >> > > > > > of
> >> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in Admin
> >> > > protocol.
> >> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree my
> >> proposal is
> >> > > > not
> >> > > > > > > > >> consistent
> >> > > > > > > > >> >>>> with
> >> > > > > > > > >> >>>> > >> > other
> >> > > > > > > > >> >>>> > >> > > > > cases.
> >> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better solution
> >> or
> >> > > > > something
> >> > > > > > > > >> >>>> in-between.
> >> > > > > > > > >> >>>> > >> > > > > > >
> >> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it is a
> >> great
> >> > > > idea.
> >> > > > > > > This
> >> > > > > > > > >> way we
> >> > > > > > > > >> >>>> can
> >> > > > > > > > >> >>>> > >> move
> >> > > > > > > > >> >>>> > >> > > on
> >> > > > > > > > >> >>>> > >> > > > > > > faster.
> >> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on date/time
> so
> >> > > people
> >> > > > > can
> >> > > > > > > > join.
> >> > > > > > > > >> >>>> Will work
> >> > > > > > > > >> >>>> > >> > for
> >> > > > > > > > >> >>>> > >> > > me
> >> > > > > > > > >> >>>> > >> > > > > > this
> >> > > > > > > > >> >>>> > >> > > > > > > and
> >> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime if
> agreed
> >> in
> >> > > > > advance.
> >> > > > > > > > >> >>>> > >> > > > > > >
> >> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> >> > > > > > > > >> >>>> > >> > > > > > > Andrii
> >> > > > > > > > >> >>>> > >> > > > > > >
> >> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM,
> >> Jay
> >> > > > Kreps <
> >> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> >> > > > > > > > >> >>>> > >> > > > > wrote:
> >> > > > > > > > >> >>>> > >> > > > > > >
> >> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> >> > > > > > > > >> >>>> > >> > > > > > >>
> >> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good error
> >> handling
> >> > > > > > without
> >> > > > > > > > >> needing
> >> > > > > > > > >> >>>> custom
> >> > > > > > > > >> >>>> > >> > > > > > server-side
> >> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the
> >> client has
> >> > > > the
> >> > > > > > > > >> context to
> >> > > > > > > > >> >>>> know
> >> > > > > > > > >> >>>> > >> that
> >> > > > > > > > >> >>>> > >> > > if
> >> > > > > > > > >> >>>> > >> > > > it
> >> > > > > > > > >> >>>> > >> > > > > > got
> >> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic doesn't
> >> exist
> >> > > to
> >> > > > > say
> >> > > > > > > > >> "Topic X
> >> > > > > > > > >> >>>> doesn't
> >> > > > > > > > >> >>>> > >> > > exist"
> >> > > > > > > > >> >>>> > >> > > > > > >> rather
> >> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or
> >> whatever).
> >> > > Maybe
> >> > > > > > there
> >> > > > > > > > are
> >> > > > > > > > >> >>>> specific
> >> > > > > > > > >> >>>> > >> > cases
> >> > > > > > > > >> >>>> > >> > > > > where
> >> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to add
> >> > > > server-side
> >> > > > > > > error
> >> > > > > > > > >> >>>> messages we
> >> > > > > > > > >> >>>> > >> > > really
> >> > > > > > > > >> >>>> > >> > > > > do
> >> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a consistent
> >> way
> >> > > > across
> >> > > > > > the
> >> > > > > > > > >> protocol.
> >> > > > > > > > >> >>>> > >> > > > > > >>
> >> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open
> >> questions
> >> > > > here
> >> > > > > > > from
> >> > > > > > > > my
> >> > > > > > > > >> >>>> previous
> >> > > > > > > > >> >>>> > >> > > list. I
> >> > > > > > > > >> >>>> > >> > > > > > will
> >> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few days for
> >> Strata
> >> > > > > > though.
> >> > > > > > > > >> Maybe
> >> > > > > > > > >> >>>> we could
> >> > > > > > > > >> >>>> > >> > do
> >> > > > > > > > >> >>>> > >> > > a
> >> > > > > > > > >> >>>> > >> > > > > > Google
> >> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open issues
> >> some
> >> > > time
> >> > > > > > > towards
> >> > > > > > > > >> the
> >> > > > > > > > >> >>>> end of
> >> > > > > > > > >> >>>> > >> > next
> >> > > > > > > > >> >>>> > >> > > > week
> >> > > > > > > > >> >>>> > >> > > > > > for
> >> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in this
> ticket?
> >> I
> >> > > have
> >> > > > a
> >> > > > > > > > feeling
> >> > > > > > > > >> that
> >> > > > > > > > >> >>>> might
> >> > > > > > > > >> >>>> > >> > > > progress
> >> > > > > > > > >> >>>> > >> > > > > > >> things a little faster than
> >> email--I
> >> > > > think
> >> > > > > we
> >> > > > > > > > >> could talk
> >> > > > > > > > >> >>>> > >> through
> >> > > > > > > > >> >>>> > >> > > > those
> >> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly
> >> quickly...
> >> > > > > > > > >> >>>> > >> > > > > > >>
> >> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> >> > > > > > > > >> >>>> > >> > > > > > >>
> >> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM,
> >> Andrii
> >> > > > > > > > Biletskyi <
> >> > > > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly>
> >> wrote:
> >> > > > > > > > >> >>>> > >> > > > > > >>
> >> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> >> > > > > > > > >> >>>> > >> > > > > > >> >
> >> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address some of
> >> the
> >> > > > issues
> >> > > > > > > which
> >> > > > > > > > >> were
> >> > > > > > > > >> >>>> > >> mentioned
> >> > > > > > > > >> >>>> > >> > > > > earlier
> >> > > > > > > > >> >>>> > >> > > > > > >> about
> >> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of
> >> those was
> >> > > > > about
> >> > > > > > > > >> batching
> >> > > > > > > > >> >>>> > >> > operations.
> >> > > > > > > > >> >>>> > >> > > > What
> >> > > > > > > > >> >>>> > >> > > > > > if
> >> > > > > > > > >> >>>> > >> > > > > > >> we
> >> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand approach
> >> and let
> >> > > > > people
> >> > > > > > > > >> specify
> >> > > > > > > > >> >>>> > >> topic-name
> >> > > > > > > > >> >>>> > >> > > by
> >> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> >> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most of the
> use
> >> > > cases?
> >> > > > > > > > >> >>>> > >> > > > > > >> >
> >> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what information
> >> should
> >> > > we
> >> > > > > > > > generally
> >> > > > > > > > >> >>>> provide in
> >> > > > > > > > >> >>>> > >> > > Admin
> >> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> >> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin commands
> >> don't
> >> > > > imply
> >> > > > > > > they
> >> > > > > > > > >> will
> >> > > > > > > > >> >>>> be used
> >> > > > > > > > >> >>>> > >> > only
> >> > > > > > > > >> >>>> > >> > > > in
> >> > > > > > > > >> >>>> > >> > > > > > CLI
> >> > > > > > > > >> >>>> > >> > > > > > >> > but,
> >> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a very
> >> > > important
> >> > > > > > > client
> >> > > > > > > > >> of this
> >> > > > > > > > >> >>>> > >> > feature.
> >> > > > > > > > >> >>>> > >> > > In
> >> > > > > > > > >> >>>> > >> > > > > > this
> >> > > > > > > > >> >>>> > >> > > > > > >> > case,
> >> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would like
> to
> >> > > provide
> >> > > > > > users
> >> > > > > > > > >> with
> >> > > > > > > > >> >>>> rich
> >> > > > > > > > >> >>>> > >> > > experience
> >> > > > > > > > >> >>>> > >> > > > > in
> >> > > > > > > > >> >>>> > >> > > > > > >> terms
> >> > > > > > > > >> >>>> > >> > > > > > >> > of
> >> > > > > > > > >> >>>> > >> > > > > > >> > getting results / errors of
> the
> >> > > > executed
> >> > > > > > > > >> commands.
> >> > > > > > > > >> >>>> Usually
> >> > > > > > > > >> >>>> > >> we
> >> > > > > > > > >> >>>> > >> > > > supply
> >> > > > > > > > >> >>>> > >> > > > > > >> with
> >> > > > > > > > >> >>>> > >> > > > > > >> > responses only errorCode,
> which
> >> looks
> >> > > > > very
> >> > > > > > > > >> limiting,
> >> > > > > > > > >> >>>> in case
> >> > > > > > > > >> >>>> > >> > of
> >> > > > > > > > >> >>>> > >> > > > CLI
> >> > > > > > > > >> >>>> > >> > > > > we
> >> > > > > > > > >> >>>> > >> > > > > > >> may
> >> > > > > > > > >> >>>> > >> > > > > > >> > want to print human readable
> >> error
> >> > > > > > > description.
> >> > > > > > > > >> >>>> > >> > > > > > >> >
> >> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into account
> >> previous item
> >> > > > > about
> >> > > > > > > > >> batching,
> >> > > > > > > > >> >>>> what
> >> > > > > > > > >> >>>> > >> do
> >> > > > > > > > >> >>>> > >> > > you
> >> > > > > > > > >> >>>> > >> > > > > > think
> >> > > > > > > > >> >>>> > >> > > > > > >> > about
> >> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> >> > > > > > > > >> >>>> > >> > > > > > >> >
> >> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support
> >> regexp)
> >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest =>
> TopicName
> >> > > > > Partitions
> >> > > > > > > > >> Replicas
> >> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> >> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> >> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse =>
> ErrorCode
> >> > > > > > > > ErrorDescription
> >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> >> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription => string
> >> (empty
> >> > > if
> >> > > > > > > > >> successful)
> >> > > > > > > > >> >>>> > >> > > > > > >> >
> >> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
> >> TopicNameRegexp
> >> > > >
> >
> > ...
> >
> > [Message clipped]
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Guys,

Thanks for a great discussion!
Here are the actions points:

1. Q: Get rid of all scala requests objects, use java protocol definitions.
    A: Gwen kindly took that (KAFKA-1927). It's important to speed up
review procedure
         there since this ticket blocks other important changes.

2. Q: Generic re-reroute facility vs client maintaining cluster state.
    A: Jay has added pseudo code to KAFKA-1912 - need to consider whether
this will be
        easy to implement as a server-side feature (comments are welcomed!).

3. Q: Controller field in wire protocol.
    A: This might be useful for clients, add this to TopicMetadataResponse
(already in KIP).

4. Q: Decoupling topic creation from TMR.
    A: I will add proposed by Jun solution (using clientId for that) to the
KIP.

5. Q: Bumping new versions of TMR vs grabbing all protocol changes in one
version.
    A: It was decided to try to gather all changes to protocol (before
release).
        In case of TMR it worth checking: KAFKA-2020 and KIP-13 (quotas)

6. Q: JSON lib is needed to deserialize user's input in CLI tool.
    A: Use jackson for that, /tools project is a separate jar so shouldn't
be a big deal.

7.  Q: VerifyReassingPartitions vs generic status check command.
     A: For long-running requests like reassign partitions *progress* check
request is useful,
         it makes sense to introduce it.

 Please add, correct me if I missed something.

Thanks,
Andrii Biletskyi

On Tue, Mar 17, 2015 at 6:20 PM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Joel,
>
> You are right, I removed ClusterMetadata because we have partially
> what we need in TopicMetadata. Also, as Jay pointed out earlier, we
> would like to have "orthogonal" API, but at the same time we need
> to be backward compatible.
>
> But I like your idea and even have some other arguments for this option:
> There is also DescribeTopicRequest which was proposed in this KIP,
> it returns topic configs, partitions, replication factor plus partition
> ISR, ASR,
> leader replica. The later part is really already there in
> TopicMetadataRequest.
> So again we'll have to add stuff to TMR, not to duplicate some info in
> newly added requests. However, this way we'll end up with "monster"
> request which returns cluster metadata, topic replication and config info
> plus partition replication data. Seems logical to split TMR to
> - ClusterMetadata (brokers + controller, maybe smth else)
> - TopicMetadata (topic info + partition details)
> But since current TMR is involved in lots of places (including network
> client,
> as I understand) this might be very serious change and it probably makes
> sense to stick with current approach.
>
> Thanks,
> Andrii Biletskyi
>
>
> On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jj...@gmail.com> wrote:
>
>> I may be missing some context but hopefully this will also be covered
>> today: I thought the earlier proposal where there was an explicit
>> ClusterMetadata request was clearer and explicit. During the course of
>> this thread I think the conclusion was that the main need was for
>> controller information and that can be rolled into the topic metadata
>> response but that seems a bit irrelevant to topic metadata. FWIW I
>> think the full broker-list is also irrelevant to topic metadata, but
>> it is already there and in use. I think there is still room for an
>> explicit ClusterMetadata request since there may be other
>> cluster-level information that we may want to add over time (and that
>> have nothing to do with topic metadata).
>>
>> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
>> > Jun,
>> >
>> > 101. Okay, if you say that such use case is important. I also think
>> > using clientId for these purposes is fine - if we already have this
>> field
>> > as part of all Wire protocol messages, why not use that.
>> > I will update KIP-4 page if nobody has other ideas (which may come up
>> > during the call today).
>> >
>> > 102.1 Agree, I'll update the KIP accordingly. I think we can add new,
>> > fine-grained error codes if some error code received in specific case
>> > won't give enough context to return a descriptive error message for
>> user.
>> >
>> > Look forward to discussing all outstanding issues in detail today during
>> > the call.
>> >
>> > Thanks,
>> > Andrii Biletskyi
>> >
>> >
>> >
>> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io> wrote:
>> >
>> > > 101. There may be a use case where you only want the topics to be
>> created
>> > > manually by admins. Currently, you can do that by disabling auto topic
>> > > creation and issue topic creation from the TopicCommand. If we
>> disable auto
>> > > topic creation completely on the broker and don't have a way to
>> distinguish
>> > > between topic creation requests from the regular clients and the
>> admin, we
>> > > can't support manual topic creation any more. I was thinking that
>> another
>> > > way of distinguishing the clients making the topic creation requests
>> is
>> > > using clientId. For example, the admin tool can set it to something
>> like
>> > > admin and the broker can treat that clientId specially.
>> > >
>> > > Also, there is a related discussion in KAFKA-2020. Currently, we do
>> the
>> > > following in TopicMetadataResponse:
>> > >
>> > > 1. If leader is not available, we set the partition level error code
>> to
>> > > LeaderNotAvailable.
>> > > 2. If a non-leader replica is not available, we take that replica out
>> of
>> > > the assigned replica list and isr in the response. As an indication
>> for
>> > > doing that, we set the partition level error code to
>> ReplicaNotAvailable.
>> > >
>> > > This has a few problems. First, ReplicaNotAvailable probably
>> shouldn't be
>> > > an error, at least for the normal producer/consumer clients that just
>> want
>> > > to find out the leader. Second, it can happen that both the leader and
>> > > another replica are not available at the same time. There is no error
>> code
>> > > to indicate both. Third, even if a replica is not available, it's
>> still
>> > > useful to return its replica id since some clients (e.g. admin tool)
>> may
>> > > still make use of it.
>> > >
>> > > One way to address this issue is to always return the replica id for
>> > > leader, assigned replicas, and isr regardless of whether the
>> corresponding
>> > > broker is live or not. Since we also return the list of live brokers,
>> the
>> > > client can figure out whether a leader or a replica is live or not
>> and act
>> > > accordingly. This way, we don't need to set the partition level error
>> code
>> > > when the leader or a replica is not available. This doesn't change
>> the wire
>> > > protocol, but does change the semantics. Since we are evolving the
>> protocol
>> > > of TopicMetadataRequest here, we can potentially piggyback the change.
>> > >
>> > > 102.1 For those types of errors due to invalid input, shouldn't we
>> just
>> > > guard it at parameter validation time and throw
>> InvalidArgumentException
>> > > without even sending the request to the broker?
>> > >
>> > > Thanks,
>> > >
>> > > Jun
>> > >
>> > >
>> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
>> > > andrii.biletskyi@stealth.ly> wrote:
>> > >
>> > > > Jun,
>> > > >
>> > > > Answering your questions:
>> > > >
>> > > > 101. If I understand you correctly, you are saying future producer
>> > > versions
>> > > > (which
>> > > > will be ported to TMR_V1) won't be able to automatically create
>> topic (if
>> > > > we
>> > > > unconditionally remove topic creation from there). But we need to
>> this
>> > > > preserve logic.
>> > > > Ok, about your proposal: I'm not a big fan too, when it comes to
>> > > > differentiating
>> > > > clients directly in protocol schema. And also I'm not sure I
>> understand
>> > > at
>> > > > all why
>> > > > auto.create.topics.enable is a server side configuration. Can we
>> > > deprecate
>> > > > this setting
>> > > > in future versions, add this setting to producer and based on that
>> upon
>> > > > receiving
>> > > > UnknownTopic create topic explicitly by a separate producer call via
>> > > > adminClient?
>> > > >
>> > > > 102.1. Hm, yes. It's because we want to support batching and at the
>> same
>> > > > time we
>> > > > want to give descriptive error messages for clients. Since
>> AdminClient
>> > > > holds the context
>> > > > to construct such messages (e.g. AdminClient layer can know that
>> > > > InvalidArgumentsCode
>> > > > means two cases: either invalid number - e.g. -1; or
>> replication-factor
>> > > was
>> > > > provided while
>> > > > partitions argument wasn't) - I wrapped responses in Exceptions.
>> But I'm
>> > > > open to any
>> > > > other ideas, this was just initial version.
>> > > > 102.2. Yes, I agree. I'll change that to probably some other dto.
>> > > >
>> > > > Thanks,
>> > > > Andrii Biletskyi
>> > > >
>> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io> wrote:
>> > > >
>> > > > > Andrii,
>> > > > >
>> > > > > 101. That's what I was thinking too, but it may not be that
>> simple. In
>> > > > > TopicMetadataRequest_V1,
>> > > > > we can let it not trigger auto topic creation. Then, in the
>> producer
>> > > > side,
>> > > > > if it gets an UnknownTopicException, it can explicitly issue a
>> > > > > createTopicRequest for auto topic creation. On the consumer side,
>> it
>> > > will
>> > > > > never issue createTopicRequest. This works when auto topic
>> creation is
>> > > > > enabled on the broker side. However, I am not sure how things
>> will work
>> > > > > when auto topic creation is disabled on the broker side. In this
>> case,
>> > > we
>> > > > > want to have a way to manually create a topic, potentially through
>> > > admin
>> > > > > commands. However, then we need a way to distinguish
>> createTopicRequest
>> > > > > issued from the producer clients and the admin tools. May be we
>> can
>> > > add a
>> > > > > new field in createTopicRequest and set it differently in the
>> producer
>> > > > > client and the admin client. However, I am not sure if that's the
>> best
>> > > > > approach.
>> > > > >
>> > > > > 2. Yes, refactoring existing requests is a non-trivial amount of
>> work.
>> > > I
>> > > > > posted some comments in KAFKA-1927. We will probably have to fix
>> > > > KAFKA-1927
>> > > > > first, before adding the new logic in KAFKA-1694. Otherwise, the
>> > > changes
>> > > > > will be too big.
>> > > > >
>> > > > > 102. About the AdminClient:
>> > > > > 102.1. It's a bit weird that we return exception in the api. It
>> seems
>> > > > that
>> > > > > we should either return error code or throw an exception when
>> getting
>> > > the
>> > > > > response state.
>> > > > > 102.2. We probably shouldn't explicitly use the request object in
>> the
>> > > > api.
>> > > > > Not every request evolution requires an api change.
>> > > > >
>> > > > > Thanks,
>> > > > >
>> > > > > Jun
>> > > > >
>> > > > >
>> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
>> > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > >
>> > > > > > Jun,
>> > > > > >
>> > > > > > Thanks for you comments. Answers inline:
>> > > > > >
>> > > > > > 100. There are a few fields such as ReplicaAssignment,
>> > > > > > > ReassignPartitionRequest,
>> > > > > > > and PartitionsSerialized that are represented as a string, but
>> > > > contain
>> > > > > > > composite structures in json. Could we flatten them out
>> directly in
>> > > > the
>> > > > > > > protocol definition as arrays/records?
>> > > > > >
>> > > > > >
>> > > > > > Yes, now with Admin Client this looks a bit weird. My initial
>> > > > motivation
>> > > > > > was:
>> > > > > > ReassignPartitionCommand accepts input in json, we want to
>> remain
>> > > > tools'
>> > > > > > interfaces unchanged, where possible.
>> > > > > > If we port it to deserialized format, in CLI (/tools project)
>> we will
>> > > > > have
>> > > > > > to add some
>> > > > > > json library since /tools is written in java and we'll need to
>> > > > > deserialize
>> > > > > > json file
>> > > > > > provided by a user. Can we quickly agree on what this library
>> should
>> > > be
>> > > > > > (Jackson, GSON, whatever)?
>> > > > > >
>> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
>> creation?
>> > > > This
>> > > > > > > will be a bit weird now that we have a separate topic
>> creation api.
>> > > > > Have
>> > > > > > > you thought about how the new createTopicRequest and
>> > > > > TopicMetadataRequest
>> > > > > > > v1 will be used in the producer/consumer client, in addition
>> to
>> > > admin
>> > > > > > > tools? For example, ideally, we don't want
>> TopicMetadataRequest
>> > > from
>> > > > > the
>> > > > > > > consumer to trigger auto topic creation.
>> > > > > >
>> > > > > >
>> > > > > > I agree, this strange logic should be fixed. I'm not confident
>> in
>> > > this
>> > > > > > Kafka part so
>> > > > > > correct me if I'm wrong, but it doesn't look like a hard thing
>> to
>> > > do, I
>> > > > > > think we can
>> > > > > > leverage AdminClient for that in Producer and unconditionally
>> remove
>> > > > > topic
>> > > > > > creation from the TopicMetadataRequest_V1.
>> > > > > >
>> > > > > > 2. I think Jay meant getting rid of scala classes
>> > > > > > > like HeartbeatRequestAndHeader and
>> HeartbeatResponseAndHeader. We
>> > > did
>> > > > > > that
>> > > > > > > as a stop-gap thing when adding the new requests for the
>> consumers.
>> > > > > > > However, the long term plan is to get rid of all those and
>> just
>> > > reuse
>> > > > > the
>> > > > > > > java request/response in the client. Since this KIP proposes
>> to
>> > > add a
>> > > > > > > significant number of new requests, perhaps we should bite the
>> > > bullet
>> > > > > to
>> > > > > > > clean up the existing scala requests first before adding new
>> ones?
>> > > > > > >
>> > > > > >
>> > > > > > Yes, looks like I misunderstood the point of
>> ...RequestAndHeader.
>> > > > Okay, I
>> > > > > > will
>> > > > > > rework that. The only thing is that I don't see any example how
>> it
>> > > was
>> > > > > done
>> > > > > > for at
>> > > > > > least one existing protocol message. Thus, as I understand, I
>> have to
>> > > > > think
>> > > > > > how we
>> > > > > > are going to do it.
>> > > > > > Re porting all existing RQ/RP in this patch. Sounds reasonable,
>> but
>> > > if
>> > > > > it's
>> > > > > > an *obligatory*
>> > > > > > requirement to have Admin KIP done, I'm afraid this can be a
>> serious
>> > > > > > blocker for us.
>> > > > > > There are 13 protocol messages and all that would require not
>> only
>> > > unit
>> > > > > > tests but quite
>> > > > > > intensive manual testing, no? I'm afraid I'm not the right guy
>> to
>> > > cover
>> > > > > > pretty much all
>> > > > > > Kafka core internals :). Let me know your thoughts on this
>> item. Btw
>> > > > > there
>> > > > > > is a ticket to
>> > > > > > follow-up this issue (
>> > > https://issues.apache.org/jira/browse/KAFKA-2006
>> > > > ).
>> > > > > >
>> > > > > > Thanks,
>> > > > > > Andrii Biletskyi
>> > > > > >
>> > > > > >
>> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <ju...@confluent.io>
>> wrote:
>> > > > > >
>> > > > > > > Andrii,
>> > > > > > >
>> > > > > > >
>> > > > > > > A few more comments.
>> > > > > > >
>> > > > > > > 100. There are a few fields such as ReplicaAssignment,
>> > > > > > > ReassignPartitionRequest,
>> > > > > > > and PartitionsSerialized that are represented as a string, but
>> > > > contain
>> > > > > > > composite structures in json. Could we flatten them out
>> directly in
>> > > > the
>> > > > > > > protocol definition as arrays/records?
>> > > > > > >
>> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
>> > > creation?
>> > > > > This
>> > > > > > > will be a bit weird now that we have a separate topic
>> creation api.
>> > > > > Have
>> > > > > > > you thought about how the new createTopicRequest and
>> > > > > TopicMetadataRequest
>> > > > > > > v1 will be used in the producer/consumer client, in addition
>> to
>> > > admin
>> > > > > > > tools? For example, ideally, we don't want
>> TopicMetadataRequest
>> > > from
>> > > > > the
>> > > > > > > consumer to trigger auto topic creation.
>> > > > > > >
>> > > > > > > 2. I think Jay meant getting rid of scala classes
>> > > > > > > like HeartbeatRequestAndHeader and
>> HeartbeatResponseAndHeader. We
>> > > did
>> > > > > > that
>> > > > > > > as a stop-gap thing when adding the new requests for the
>> consumers.
>> > > > > > > However, the long term plan is to get rid of all those and
>> just
>> > > reuse
>> > > > > the
>> > > > > > > java request/response in the client. Since this KIP proposes
>> to
>> > > add a
>> > > > > > > significant number of new requests, perhaps we should bite the
>> > > bullet
>> > > > > to
>> > > > > > > clean up the existing scala requests first before adding new
>> ones?
>> > > > > > >
>> > > > > > > Thanks,
>> > > > > > >
>> > > > > > > Jun
>> > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
>> > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > >
>> > > > > > > > Hi,
>> > > > > > > >
>> > > > > > > > As said above - I list again all comments from this thread
>> so we
>> > > > > > > > can see what's left and finalize all pending issues.
>> > > > > > > >
>> > > > > > > > Comments from Jay:
>> > > > > > > > 1. This is much needed functionality, but there are a lot
>> of the
>> > > so
>> > > > > > let's
>> > > > > > > > really think these protocols through. We really want to end
>> up
>> > > > with a
>> > > > > > set
>> > > > > > > > of well thought-out, orthoganol apis. For this reason I
>> think it
>> > > is
>> > > > > > > really
>> > > > > > > > important to think through the end state even if that
>> includes
>> > > APIs
>> > > > > we
>> > > > > > > > won't implement in the first phase.
>> > > > > > > >
>> > > > > > > > A: Definitely behind this. Would appreciate if there are
>> concrete
>> > > > > > > comments
>> > > > > > > > how this can be improved.
>> > > > > > > >
>> > > > > > > > 2. Let's please please please wait until we have switched
>> the
>> > > > server
>> > > > > > over
>> > > > > > > > to the new java protocol definitions. If we add upteen more
>> ad
>> > > hoc
>> > > > > > scala
>> > > > > > > > objects that is just generating more work for the
>> conversion we
>> > > > know
>> > > > > we
>> > > > > > > > have to do.
>> > > > > > > >
>> > > > > > > > A: Fixed in the latest patch - removed scala protocol
>> classes.
>> > > > > > > >
>> > > > > > > > 3. This proposal introduces a new type of optional
>> parameter.
>> > > This
>> > > > is
>> > > > > > > > inconsistent with everything else in the protocol where we
>> use -1
>> > > > or
>> > > > > > some
>> > > > > > > > other marker value. You could argue either way but let's
>> stick
>> > > with
>> > > > > > that
>> > > > > > > > for consistency. For clients that implemented the protocol
>> in a
>> > > > > better
>> > > > > > > way
>> > > > > > > > than our scala code these basic primitives are hard to
>> change.
>> > > > > > > >
>> > > > > > > > A: Fixed in the latest patch - removed MaybeOf type and
>> changed
>> > > > > > protocol
>> > > > > > > > accordingly.
>> > > > > > > >
>> > > > > > > > 4. ClusterMetadata: This seems to duplicate
>> TopicMetadataRequest
>> > > > > which
>> > > > > > > has
>> > > > > > > > brokers, topics, and partitions. I think we should rename
>> that
>> > > > > request
>> > > > > > > > ClusterMetadataRequest (or just MetadataRequest) and
>> include the
>> > > id
>> > > > > of
>> > > > > > > the
>> > > > > > > > controller. Or are there other things we could add here?
>> > > > > > > >
>> > > > > > > > A: I agree. Updated the KIP. Let's extends TopicMetadata to
>> > > > version 2
>> > > > > > and
>> > > > > > > > include controller.
>> > > > > > > >
>> > > > > > > > 5. We have a tendency to try to make a lot of requests that
>> can
>> > > > only
>> > > > > go
>> > > > > > > to
>> > > > > > > > particular nodes. This adds a lot of burden for client
>> > > > > implementations
>> > > > > > > (it
>> > > > > > > > sounds easy but each discovery can fail in many parts so it
>> ends
>> > > up
>> > > > > > > being a
>> > > > > > > > full state machine to do right). I think we should consider
>> > > making
>> > > > > > admin
>> > > > > > > > commands and ideally as many of the other apis as possible
>> > > > available
>> > > > > on
>> > > > > > > all
>> > > > > > > > brokers and just redirect to the controller on the broker
>> side.
>> > > > > Perhaps
>> > > > > > > > there would be a general way to encapsulate this re-routing
>> > > > behavior.
>> > > > > > > >
>> > > > > > > > A: It's a very interesting idea, but seems there are some
>> > > concerns
>> > > > > > about
>> > > > > > > > this
>> > > > > > > > feature (like performance considerations, how this will
>> > > complicate
>> > > > > > server
>> > > > > > > > etc).
>> > > > > > > > I believe this shouldn't be a blocker. If this feature is
>> > > > implemented
>> > > > > > at
>> > > > > > > > some
>> > > > > > > > point it won't affect Admin changes - at least no changes to
>> > > public
>> > > > > API
>> > > > > > > > will be required.
>> > > > > > > >
>> > > > > > > > 6. We should probably normalize the key value pairs used for
>> > > > configs
>> > > > > > > rather
>> > > > > > > > than embedding a new formatting. So two strings rather than
>> one
>> > > > with
>> > > > > an
>> > > > > > > > internal equals sign.
>> > > > > > > >
>> > > > > > > > A: Fixed in the latest patch - normalized configs and
>> changed
>> > > > > protocol
>> > > > > > > > accordingly.
>> > > > > > > >
>> > > > > > > > 7. Is the postcondition of these APIs that the command has
>> begun
>> > > or
>> > > > > > that
>> > > > > > > > the command has been completed? It is a lot more usable if
>> the
>> > > > > command
>> > > > > > > has
>> > > > > > > > been completed so you know that if you create a topic and
>> then
>> > > > > publish
>> > > > > > to
>> > > > > > > > it you won't get an exception about there being no such
>> topic.
>> > > > > > > >
>> > > > > > > > A: For long running requests (like reassign partitions) -
>> the
>> > > post
>> > > > > > > > condition is
>> > > > > > > > command has begun - so we don't block the client. In case
>> of your
>> > > > > > > example -
>> > > > > > > > topic commands, this will be refactored and topic commands
>> will
>> > > be
>> > > > > > > executed
>> > > > > > > > immediately, since the Controller will serve Admin requests
>> > > > > > > > (follow-up ticket KAFKA-1777).
>> > > > > > > >
>> > > > > > > > 8. Describe topic and list topics duplicate a lot of stuff
>> in the
>> > > > > > > metadata
>> > > > > > > > request. Is there a reason to give back topics marked for
>> > > > deletion? I
>> > > > > > > feel
>> > > > > > > > like if we just make the post-condition of the delete
>> command be
>> > > > that
>> > > > > > the
>> > > > > > > > topic is deleted that will get rid of the need for this
>> right?
>> > > And
>> > > > it
>> > > > > > > will
>> > > > > > > > be much more intuitive.
>> > > > > > > >
>> > > > > > > > A: Fixed in the latest patch - removed topics marked for
>> deletion
>> > > > in
>> > > > > > > > ListTopicsRequest.
>> > > > > > > >
>> > > > > > > > 9. Should we consider batching these requests? We have
>> generally
>> > > > > tried
>> > > > > > to
>> > > > > > > > allow multiple operations to be batched. My suspicion is
>> that
>> > > > without
>> > > > > > > this
>> > > > > > > > we will get a lot of code that does something like
>> > > > > > > >    for(topic: adminClient.listTopics())
>> > > > > > > >       adminClient.describeTopic(topic)
>> > > > > > > > this code will work great when you test on 5 topics but not
>> do as
>> > > > > well
>> > > > > > if
>> > > > > > > > you have 50k.
>> > > > > > > >
>> > > > > > > > A: Updated the KIP - please check "Topic Admin Schema"
>> section.
>> > > > > > > >
>> > > > > > > > 10. I think we should also discuss how we want to expose a
>> > > > > programmatic
>> > > > > > > JVM
>> > > > > > > > client api for these operations. Currently people rely on
>> > > > AdminUtils
>> > > > > > > which
>> > > > > > > > is totally sketchy. I think we probably need another client
>> under
>> > > > > > > clients/
>> > > > > > > > that exposes administrative functionality. We will need
>> this just
>> > > > to
>> > > > > > > > properly test the new apis, I suspect. We should figure out
>> that
>> > > > API.
>> > > > > > > >
>> > > > > > > > A: Updated the KIP - please check "Admin Client" section
>> with an
>> > > > > > initial
>> > > > > > > > API proposal.
>> > > > > > > >
>> > > > > > > > 11. The other information that would be really useful to get
>> > > would
>> > > > be
>> > > > > > > > information about partitions--how much data is in the
>> partition,
>> > > > what
>> > > > > > are
>> > > > > > > > the segment offsets, what is the log-end offset (i.e. last
>> > > offset),
>> > > > > > what
>> > > > > > > is
>> > > > > > > > the compaction point, etc. I think that done right this
>> would be
>> > > > the
>> > > > > > > > successor to the very awkward OffsetRequest we have today.
>> > > > > > > >
>> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest
>> patch. I
>> > > > > believe
>> > > > > > > > this should
>> > > > > > > > be resolved in a separate KIP / jira ticket.
>> > > > > > > >
>> > > > > > > > 12. Generally we can do good error handling without needing
>> > > custom
>> > > > > > > > server-side
>> > > > > > > > messages. I.e. generally the client has the context to know
>> that
>> > > if
>> > > > > it
>> > > > > > > got
>> > > > > > > > an error that the topic doesn't exist to say "Topic X
>> doesn't
>> > > > exist"
>> > > > > > > rather
>> > > > > > > > than "error code 14" (or whatever). Maybe there are specific
>> > > cases
>> > > > > > where
>> > > > > > > > this is hard? If we want to add server-side error messages
>> we
>> > > > really
>> > > > > do
>> > > > > > > > need to do this in a consistent way across the protocol.
>> > > > > > > >
>> > > > > > > > A: Updated the KIP - please check "Protocol Errors"
>> section. I
>> > > > added
>> > > > > > the
>> > > > > > > > comprehensive, fine-grained list of error codes.
>> > > > > > > >
>> > > > > > > > Comments from Guozhang:
>> > > > > > > > 13. Describe topic request: it would be great to go beyond
>> just
>> > > > > > batching
>> > > > > > > on
>> > > > > > > > topic name regex for this request. For example, a very
>> common use
>> > > > > case
>> > > > > > of
>> > > > > > > > the topic command is to list all topics whose config A's
>> value is
>> > > > B.
>> > > > > > With
>> > > > > > > > topic name regex then we have to first retrieve __all__
>> topics's
>> > > > > > > > description info and then filter at the client end, which
>> will
>> > > be a
>> > > > > > huge
>> > > > > > > > burden on ZK.
>> > > > > > > > AND
>> > > > > > > > 14. Config K-Vs in create topic: this is related to the
>> previous
>> > > > > point;
>> > > > > > > > maybe we can add another metadata K-V or just a metadata
>> string
>> > > > along
>> > > > > > > side
>> > > > > > > > with config K-V in create topic like we did for offset
>> commit
>> > > > > request.
>> > > > > > > This
>> > > > > > > > field can be quite useful in storing information like
>> "owner" of
>> > > > the
>> > > > > > > topic
>> > > > > > > > who issue the create command, etc, which is quite important
>> for a
>> > > > > > > > multi-tenant setting. Then in the describe topic request we
>> can
>> > > > also
>> > > > > > > batch
>> > > > > > > > on regex of the metadata field.
>> > > > > > > >
>> > > > > > > > A: As discussed it is very interesting but can be
>> implemented
>> > > later
>> > > > > > after
>> > > > > > > > we have some basic functionality there.
>> > > > > > > >
>> > > > > > > > 15. Today all the admin operations are async in the sense
>> that
>> > > > > command
>> > > > > > > will
>> > > > > > > > return once it is written in ZK, and that is why we need
>> extra
>> > > > > > > verification
>> > > > > > > > like testUtil.waitForTopicCreated() / verify partition
>> > > reassignment
>> > > > > > > > request, etc. With admin requests we could add a flag to
>> enable /
>> > > > > > disable
>> > > > > > > > synchronous requests; when it is turned on, the response
>> will not
>> > > > > > return
>> > > > > > > > until the request has been completed. And for async
>> requests we
>> > > can
>> > > > > > add a
>> > > > > > > > "token" field in the response, and then only need a general
>> > > "admin
>> > > > > > > > verification request" with the given token to check if the
>> async
>> > > > > > request
>> > > > > > > > has been completed.
>> > > > > > > >
>> > > > > > > > A: I see your point. My idea was to provide specific
>> > > > Verify...Request
>> > > > > > per
>> > > > > > > > each
>> > > > > > > > long running request, where needed. We can do it the way you
>> > > > suggest.
>> > > > > > The
>> > > > > > > > only
>> > > > > > > > concern is that introducing a token we again will make
>> schema
>> > > > > > "dynamic".
>> > > > > > > We
>> > > > > > > > wanted
>> > > > > > > > to do similar thing introducing single AdminRequest for all
>> topic
>> > > > > > > commands
>> > > > > > > > but rejected
>> > > > > > > > this idea because we wanted to have schema defined. So this
>> is
>> > > > more a
>> > > > > > > > choice between:
>> > > > > > > > a) have fixed schema but introduce each time new
>> Verify...Request
>> > > > for
>> > > > > > > > long-running requests
>> > > > > > > > b) use one request for verification but generalize it with
>> token
>> > > > > > > > I'm fine with whatever decision community come to. Just let
>> me
>> > > know
>> > > > > > your
>> > > > > > > > thoughts.
>> > > > > > > >
>> > > > > > > > Comment from Gwen:
>> > > > > > > > 16. Specifically for ownership, I think the plan is to add
>> ACL
>> > > (it
>> > > > > > sounds
>> > > > > > > > like you are describing ACL) via an external system (Argus,
>> > > > Sentry).
>> > > > > > > > I remember KIP-11 described this, but I can't find the KIP
>> any
>> > > > > longer.
>> > > > > > > >
>> > > > > > > > A: Okay, no problem. Not sure though how we are going to
>> handle
>> > > it.
>> > > > > > Wait
>> > > > > > > > which KIP
>> > > > > > > > will be committed first and include changes to
>> TopicMetadata from
>> > > > the
>> > > > > > > later
>> > > > > > > > one?
>> > > > > > > > Anyway, I added this note to "Open Questions" section so we
>> don't
>> > > > > miss
>> > > > > > > this
>> > > > > > > > piece.
>> > > > > > > >
>> > > > > > > > Thanks,
>> > > > > > > > Andrii Biletskyi
>> > > > > > > >
>> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
>> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > > >
>> > > > > > > > > Hi all,
>> > > > > > > > >
>> > > > > > > > > Today I uploaded the patch that covers some of the
>> discussed
>> > > and
>> > > > > > agreed
>> > > > > > > > > items:
>> > > > > > > > > - removed MaybeOf optional type
>> > > > > > > > > - switched to java protocol definitions
>> > > > > > > > > - simplified messages (normalized configs, removed topic
>> marked
>> > > > for
>> > > > > > > > > deletion)
>> > > > > > > > >
>> > > > > > > > > I also updated the KIP-4 with respective changes and
>> wrote down
>> > > > my
>> > > > > > > > > proposal for
>> > > > > > > > > pending items:
>> > > > > > > > > - Batch Admin Operations -> updated Wire Protocol schema
>> > > proposal
>> > > > > > > > > - Remove ClusterMetadata -> changed to extend
>> > > > TopicMetadataRequest
>> > > > > > > > > - Admin Client -> updated my initial proposal to reflect
>> > > batching
>> > > > > > > > > - Error codes -> proposed fine-grained error code instead
>> of
>> > > > > > > > > AdminRequestFailed
>> > > > > > > > >
>> > > > > > > > > I will also send a separate email to cover all comments
>> from
>> > > this
>> > > > > > > thread.
>> > > > > > > > >
>> > > > > > > > > Thanks,
>> > > > > > > > > Andrii Biletskyi
>> > > > > > > > >
>> > > > > > > > >
>> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
>> > > > > gshapira@cloudera.com
>> > > > > > >
>> > > > > > > > > wrote:
>> > > > > > > > >
>> > > > > > > > >> Found KIP-11 (
>> > > > > > > > >>
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
>> > > > > > > > >> )
>> > > > > > > > >> It actually specifies changes to the Metadata protocol,
>> so
>> > > > making
>> > > > > > sure
>> > > > > > > > >> both KIPs are consistent in this regard will be good.
>> > > > > > > > >>
>> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
>> > > > > > gshapira@cloudera.com
>> > > > > > > >
>> > > > > > > > >> wrote:
>> > > > > > > > >> > Specifically for ownership, I think the plan is to add
>> ACL
>> > > (it
>> > > > > > > sounds
>> > > > > > > > >> > like you are describing ACL) via an external system
>> (Argus,
>> > > > > > Sentry).
>> > > > > > > > >> > I remember KIP-11 described this, but I can't find the
>> KIP
>> > > any
>> > > > > > > longer.
>> > > > > > > > >> >
>> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting
>> information
>> > > that
>> > > > > > > already
>> > > > > > > > >> > exists from Kafka brokers, not on adding information
>> that
>> > > > > perhaps
>> > > > > > > > >> > should exist but doesn't yet?
>> > > > > > > > >> >
>> > > > > > > > >> > Gwen
>> > > > > > > > >> >
>> > > > > > > > >> >
>> > > > > > > > >> >
>> > > > > > > > >> >
>> > > > > > > > >> >
>> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
>> > > > > > wangguoz@gmail.com>
>> > > > > > > > >> wrote:
>> > > > > > > > >> >> Folks,
>> > > > > > > > >> >>
>> > > > > > > > >> >> Just want to elaborate a bit more on the create-topic
>> > > > metadata
>> > > > > > and
>> > > > > > > > >> batching
>> > > > > > > > >> >> describe-topic based on config / metadata in my
>> previous
>> > > > email
>> > > > > as
>> > > > > > > we
>> > > > > > > > >> work
>> > > > > > > > >> >> on KAFKA-1694. The main motivation is to have some
>> sort of
>> > > > > topic
>> > > > > > > > >> management
>> > > > > > > > >> >> mechanisms, which I think is quite important in a
>> > > > multi-tenant
>> > > > > /
>> > > > > > > > cloud
>> > > > > > > > >> >> architecture: today anyone can create topics in a
>> shared
>> > > > Kafka
>> > > > > > > > >> cluster, but
>> > > > > > > > >> >> there is no concept or "ownership" of topics that are
>> > > created
>> > > > > by
>> > > > > > > > >> different
>> > > > > > > > >> >> users. For example, at LinkedIn we basically
>> distinguish
>> > > > topic
>> > > > > > > owners
>> > > > > > > > >> via
>> > > > > > > > >> >> some casual topic name prefix, which is a bit awkward
>> and
>> > > > does
>> > > > > > not
>> > > > > > > > fly
>> > > > > > > > >> as
>> > > > > > > > >> >> we scale our customers. It would be great to use
>> > > > > describe-topics
>> > > > > > > such
>> > > > > > > > >> as:
>> > > > > > > > >> >>
>> > > > > > > > >> >> Describe all topics that is created by me.
>> > > > > > > > >> >>
>> > > > > > > > >> >> Describe all topics whose retention time is overriden
>> to X.
>> > > > > > > > >> >>
>> > > > > > > > >> >> Describe all topics whose writable group include user
>> Y
>> > > (this
>> > > > > is
>> > > > > > > > >> related to
>> > > > > > > > >> >> authorization), etc..
>> > > > > > > > >> >>
>> > > > > > > > >> >> One possible way to achieve this is to add a metadata
>> file
>> > > in
>> > > > > the
>> > > > > > > > >> >> create-topic request, whose value will also be
>> written ZK
>> > > as
>> > > > we
>> > > > > > > > create
>> > > > > > > > >> the
>> > > > > > > > >> >> topic; then describe-topics can choose to batch topics
>> > > based
>> > > > on
>> > > > > > 1)
>> > > > > > > > name
>> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata regex, etc.
>> > > > > > > > >> >>
>> > > > > > > > >> >> Thoughts?
>> > > > > > > > >> >>
>> > > > > > > > >> >> Guozhang
>> > > > > > > > >> >>
>> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
>> > > > > > wangguoz@gmail.com>
>> > > > > > > > >> wrote:
>> > > > > > > > >> >>
>> > > > > > > > >> >>> Thanks for the updated wiki. A few comments below:
>> > > > > > > > >> >>>
>> > > > > > > > >> >>> 1. Error description in response: I think if some
>> > > errorCode
>> > > > > > could
>> > > > > > > > >> indicate
>> > > > > > > > >> >>> several different error cases then we should really
>> change
>> > > > it
>> > > > > to
>> > > > > > > > >> multiple
>> > > > > > > > >> >>> codes. In general the errorCode itself would be
>> precise
>> > > and
>> > > > > > > > >> sufficient for
>> > > > > > > > >> >>> describing the server side errors.
>> > > > > > > > >> >>>
>> > > > > > > > >> >>> 2. Describe topic request: it would be great to go
>> beyond
>> > > > just
>> > > > > > > > >> batching on
>> > > > > > > > >> >>> topic name regex for this request. For example, a
>> very
>> > > > common
>> > > > > > use
>> > > > > > > > >> case of
>> > > > > > > > >> >>> the topic command is to list all topics whose config
>> A's
>> > > > value
>> > > > > > is
>> > > > > > > B.
>> > > > > > > > >> With
>> > > > > > > > >> >>> topic name regex then we have to first retrieve
>> __all__
>> > > > > topics's
>> > > > > > > > >> >>> description info and then filter at the client end,
>> which
>> > > > will
>> > > > > > be
>> > > > > > > a
>> > > > > > > > >> huge
>> > > > > > > > >> >>> burden on ZK.
>> > > > > > > > >> >>>
>> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is related to
>> the
>> > > > > previous
>> > > > > > > > point;
>> > > > > > > > >> >>> maybe we can add another metadata K-V or just a
>> metadata
>> > > > > string
>> > > > > > > > along
>> > > > > > > > >> side
>> > > > > > > > >> >>> with config K-V in create topic like we did for
>> offset
>> > > > commit
>> > > > > > > > >> request. This
>> > > > > > > > >> >>> field can be quite useful in storing information like
>> > > > "owner"
>> > > > > of
>> > > > > > > the
>> > > > > > > > >> topic
>> > > > > > > > >> >>> who issue the create command, etc, which is quite
>> > > important
>> > > > > for
>> > > > > > a
>> > > > > > > > >> >>> multi-tenant setting. Then in the describe topic
>> request
>> > > we
>> > > > > can
>> > > > > > > also
>> > > > > > > > >> batch
>> > > > > > > > >> >>> on regex of the metadata field.
>> > > > > > > > >> >>>
>> > > > > > > > >> >>> 4. Today all the admin operations are async in the
>> sense
>> > > > that
>> > > > > > > > command
>> > > > > > > > >> will
>> > > > > > > > >> >>> return once it is written in ZK, and that is why we
>> need
>> > > > extra
>> > > > > > > > >> verification
>> > > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify
>> partition
>> > > > > > > reassignment
>> > > > > > > > >> >>> request, etc. With admin requests we could add a
>> flag to
>> > > > > enable
>> > > > > > /
>> > > > > > > > >> disable
>> > > > > > > > >> >>> synchronous requests; when it is turned on, the
>> response
>> > > > will
>> > > > > > not
>> > > > > > > > >> return
>> > > > > > > > >> >>> until the request has been completed. And for async
>> > > requests
>> > > > > we
>> > > > > > > can
>> > > > > > > > >> add a
>> > > > > > > > >> >>> "token" field in the response, and then only need a
>> > > general
>> > > > > > "admin
>> > > > > > > > >> >>> verification request" with the given token to check
>> if the
>> > > > > async
>> > > > > > > > >> request
>> > > > > > > > >> >>> has been completed.
>> > > > > > > > >> >>>
>> > > > > > > > >> >>> 5. +1 for extending Metadata request to include
>> > > controller /
>> > > > > > > > >> coordinator
>> > > > > > > > >> >>> information, and then we can remove the
>> ConsumerMetadata /
>> > > > > > > > >> ClusterMetadata
>> > > > > > > > >> >>> requests.
>> > > > > > > > >> >>>
>> > > > > > > > >> >>> Guozhang
>> > > > > > > > >> >>>
>> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
>> > > > > > jjkoshy.w@gmail.com>
>> > > > > > > > >> wrote:
>> > > > > > > > >> >>>
>> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't think I
>> will be
>> > > > > able
>> > > > > > to
>> > > > > > > > >> make
>> > > > > > > > >> >>>> it today, so if notes can be sent out afterward that
>> > > would
>> > > > be
>> > > > > > > > great.
>> > > > > > > > >> >>>>
>> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen
>> Shapira
>> > > > wrote:
>> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking forward
>> to
>> > > > > chatting
>> > > > > > > with
>> > > > > > > > >> >>>> everyone :)
>> > > > > > > > >> >>>> >
>> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
>> > > > > > > joe.stein@stealth.ly>
>> > > > > > > > >> wrote:
>> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout invite to
>> all
>> > > > pmc,
>> > > > > > > > >> committers
>> > > > > > > > >> >>>> and
>> > > > > > > > >> >>>> > > everyone I found working on a KIP. If I missed
>> anyone
>> > > > in
>> > > > > > the
>> > > > > > > > >> invite
>> > > > > > > > >> >>>> please
>> > > > > > > > >> >>>> > > let me know and can update it, np.
>> > > > > > > > >> >>>> > >
>> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm Eastern
>> Time.
>> > > > Maybe
>> > > > > > we
>> > > > > > > > can
>> > > > > > > > >> get
>> > > > > > > > >> >>>> INFRA
>> > > > > > > > >> >>>> > > help to make a google account so we can manage
>> > > better?
>> > > > > > > > >> >>>> > >
>> > > > > > > > >> >>>> > > To discuss
>> > > > > > > > >> >>>> > >
>> > > > > > > > >> >>>>
>> > > > > > > > >>
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>> > > > > > > > >> >>>> > > in progress and related JIRA that are
>> interdependent
>> > > > and
>> > > > > > > common
>> > > > > > > > >> work.
>> > > > > > > > >> >>>> > >
>> > > > > > > > >> >>>> > > ~ Joe Stein
>> > > > > > > > >> >>>> > >
>> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
>> > > > > > > > jay.kreps@gmail.com>
>> > > > > > > > >> >>>> wrote:
>> > > > > > > > >> >>>> > >
>> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that will also
>> record
>> > > > and
>> > > > > > make
>> > > > > > > > the
>> > > > > > > > >> >>>> sessions
>> > > > > > > > >> >>>> > >> available on youtube.
>> > > > > > > > >> >>>> > >>
>> > > > > > > > >> >>>> > >> -Jay
>> > > > > > > > >> >>>> > >>
>> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman
>> <
>> > > > > > > > >> >>>> jholoman@cloudera.com>
>> > > > > > > > >> >>>> > >> wrote:
>> > > > > > > > >> >>>> > >>
>> > > > > > > > >> >>>> > >> > Jay / Joe
>> > > > > > > > >> >>>> > >> >
>> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for this
>> purpose.
>> > > We
>> > > > > > could
>> > > > > > > > >> record
>> > > > > > > > >> >>>> the
>> > > > > > > > >> >>>> > >> > sessions if there is interest and publish
>> them
>> > > out.
>> > > > > > > > >> >>>> > >> >
>> > > > > > > > >> >>>> > >> > Thanks
>> > > > > > > > >> >>>> > >> >
>> > > > > > > > >> >>>> > >> > Jeff
>> > > > > > > > >> >>>> > >> >
>> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
>> > > > > > > > >> jay.kreps@gmail.com>
>> > > > > > > > >> >>>> wrote:
>> > > > > > > > >> >>>> > >> >
>> > > > > > > > >> >>>> > >> > > Let's try to get the technical hang-ups
>> sorted
>> > > > out,
>> > > > > > > > though.
>> > > > > > > > >> I
>> > > > > > > > >> >>>> really
>> > > > > > > > >> >>>> > >> > think
>> > > > > > > > >> >>>> > >> > > there is some benefit to live discussion vs
>> > > > > writing. I
>> > > > > > > am
>> > > > > > > > >> >>>> hopeful that
>> > > > > > > > >> >>>> > >> if
>> > > > > > > > >> >>>> > >> > > we post instructions and give ourselves a
>> few
>> > > > > attempts
>> > > > > > > we
>> > > > > > > > >> can
>> > > > > > > > >> >>>> get it
>> > > > > > > > >> >>>> > >> > > working.
>> > > > > > > > >> >>>> > >> > >
>> > > > > > > > >> >>>> > >> > > Tuesday at that time would work for
>> me...any
>> > > > > > objections?
>> > > > > > > > >> >>>> > >> > >
>> > > > > > > > >> >>>> > >> > > -Jay
>> > > > > > > > >> >>>> > >> > >
>> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein
>> <
>> > > > > > > > >> joe.stein@stealth.ly
>> > > > > > > > >> >>>> >
>> > > > > > > > >> >>>> > >> wrote:
>> > > > > > > > >> >>>> > >> > >
>> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe like every
>> > > Tuesday ~
>> > > > > 1pm
>> > > > > > > ET
>> > > > > > > > /
>> > > > > > > > >> 10am
>> > > > > > > > >> >>>> PT
>> > > > > > > > >> >>>> > >> ????
>> > > > > > > > >> >>>> > >> > > >
>> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but there is
>> > > always
>> > > > > some
>> > > > > > > > >> issue or
>> > > > > > > > >> >>>> > >> whatever
>> > > > > > > > >> >>>> > >> > so
>> > > > > > > > >> >>>> > >> > > > we know the apache irc channel works. We
>> can
>> > > > start
>> > > > > > > there
>> > > > > > > > >> and
>> > > > > > > > >> >>>> see how
>> > > > > > > > >> >>>> > >> it
>> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too and
>> > > associate
>> > > > to
>> > > > > > > > >> tickets if
>> > > > > > > > >> >>>> need be
>> > > > > > > > >> >>>> > >> > > makes
>> > > > > > > > >> >>>> > >> > > > it helpful for things.
>> > > > > > > > >> >>>> > >> > > >
>> > > > > > > > >> >>>> > >> > > > ~ Joestein
>> > > > > > > > >> >>>> > >> > > >
>> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay
>> Kreps <
>> > > > > > > > >> >>>> jay.kreps@gmail.com>
>> > > > > > > > >> >>>> > >> > wrote:
>> > > > > > > > >> >>>> > >> > > >
>> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a Google
>> Hangout to
>> > > > chat
>> > > > > > > about
>> > > > > > > > >> this.
>> > > > > > > > >> >>>> What
>> > > > > > > > >> >>>> > >> > about
>> > > > > > > > >> >>>> > >> > > > > generalizing that a little further...I
>> > > > actually
>> > > > > > > think
>> > > > > > > > it
>> > > > > > > > >> >>>> would be
>> > > > > > > > >> >>>> > >> > good
>> > > > > > > > >> >>>> > >> > > > for
>> > > > > > > > >> >>>> > >> > > > > everyone spending a reasonable chunk of
>> > > their
>> > > > > week
>> > > > > > > on
>> > > > > > > > >> Kafka
>> > > > > > > > >> >>>> stuff
>> > > > > > > > >> >>>> > >> to
>> > > > > > > > >> >>>> > >> > > > maybe
>> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think we could
>> use
>> > > time
>> > > > > to
>> > > > > > > talk
>> > > > > > > > >> >>>> through
>> > > > > > > > >> >>>> > >> design
>> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of code
>> > > > reviews,
>> > > > > > talk
>> > > > > > > > >> through
>> > > > > > > > >> >>>> any
>> > > > > > > > >> >>>> > >> > tricky
>> > > > > > > > >> >>>> > >> > > > > issues, etc.
>> > > > > > > > >> >>>> > >> > > > >
>> > > > > > > > >> >>>> > >> > > > > We can make it publicly available so
>> that
>> > > any
>> > > > > one
>> > > > > > > can
>> > > > > > > > >> follow
>> > > > > > > > >> >>>> along
>> > > > > > > > >> >>>> > >> > who
>> > > > > > > > >> >>>> > >> > > > > likes.
>> > > > > > > > >> >>>> > >> > > > >
>> > > > > > > > >> >>>> > >> > > > > Any interest in doing this? If so I'll
>> try
>> > > to
>> > > > > set
>> > > > > > it
>> > > > > > > > up
>> > > > > > > > >> >>>> starting
>> > > > > > > > >> >>>> > >> next
>> > > > > > > > >> >>>> > >> > > > week.
>> > > > > > > > >> >>>> > >> > > > >
>> > > > > > > > >> >>>> > >> > > > > -Jay
>> > > > > > > > >> >>>> > >> > > > >
>> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii
>> > > > > Biletskyi
>> > > > > > <
>> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > > > >> >>>> > >> > > > >
>> > > > > > > > >> >>>> > >> > > > > > Hi all,
>> > > > > > > > >> >>>> > >> > > > > >
>> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed /
>> aligned
>> > > > > document
>> > > > > > > > >> structure.
>> > > > > > > > >> >>>> Also I
>> > > > > > > > >> >>>> > >> > > added
>> > > > > > > > >> >>>> > >> > > > > > some
>> > > > > > > > >> >>>> > >> > > > > > very initial proposal for
>> AdminClient so
>> > > we
>> > > > > have
>> > > > > > > > >> something
>> > > > > > > > >> >>>> to
>> > > > > > > > >> >>>> > >> start
>> > > > > > > > >> >>>> > >> > > > from
>> > > > > > > > >> >>>> > >> > > > > > while
>> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
>> > > > > > > > >> >>>> > >> > > > > >
>> > > > > > > > >> >>>> > >> > > > > >
>> > > > > > > > >> >>>> > >> > > > > >
>> > > > > > > > >> >>>> > >> > > > >
>> > > > > > > > >> >>>> > >> > > >
>> > > > > > > > >> >>>> > >> > >
>> > > > > > > > >> >>>> > >> >
>> > > > > > > > >> >>>> > >>
>> > > > > > > > >> >>>>
>> > > > > > > > >>
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>> > > > > > > > >> >>>> > >> > > > > >
>> > > > > > > > >> >>>> > >> > > > > > Thanks,
>> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
>> > > > > > > > >> >>>> > >> > > > > >
>> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM,
>> Andrii
>> > > > > > Biletskyi
>> > > > > > > <
>> > > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > > > >> >>>> > >> > > > > >
>> > > > > > > > >> >>>> > >> > > > > > > Jay,
>> > > > > > > > >> >>>> > >> > > > > > >
>> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you are right,
>> in
>> > > most
>> > > > > > cases
>> > > > > > > > >> client
>> > > > > > > > >> >>>> will
>> > > > > > > > >> >>>> > >> have
>> > > > > > > > >> >>>> > >> > > > enough
>> > > > > > > > >> >>>> > >> > > > > > > context to show descriptive error
>> > > message.
>> > > > > My
>> > > > > > > > >> concern is
>> > > > > > > > >> >>>> that
>> > > > > > > > >> >>>> > >> we
>> > > > > > > > >> >>>> > >> > > will
>> > > > > > > > >> >>>> > >> > > > > > have
>> > > > > > > > >> >>>> > >> > > > > > > to
>> > > > > > > > >> >>>> > >> > > > > > > add lots of new error codes for
>> each
>> > > > > possible
>> > > > > > > > >> error. Of
>> > > > > > > > >> >>>> course,
>> > > > > > > > >> >>>> > >> > we
>> > > > > > > > >> >>>> > >> > > > > could
>> > > > > > > > >> >>>> > >> > > > > > > reuse
>> > > > > > > > >> >>>> > >> > > > > > > some of existing like
>> > > > > > > UknownTopicOrPartitionCode,
>> > > > > > > > >> but we
>> > > > > > > > >> >>>> will
>> > > > > > > > >> >>>> > >> > also
>> > > > > > > > >> >>>> > >> > > > need
>> > > > > > > > >> >>>> > >> > > > > > to
>> > > > > > > > >> >>>> > >> > > > > > > add smth like:
>> TopicAlreadyExistsCode,
>> > > > > > > > >> >>>> TopicConfigInvalid (both
>> > > > > > > > >> >>>> > >> > for
>> > > > > > > > >> >>>> > >> > > > > topic
>> > > > > > > > >> >>>> > >> > > > > > > name and config, and probably user
>> would
>> > > > > like
>> > > > > > to
>> > > > > > > > >> know
>> > > > > > > > >> >>>> what
>> > > > > > > > >> >>>> > >> > exactly
>> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
>> > > > > > > InvalidReplicaAssignment,
>> > > > > > > > >> >>>> > >> InternalError
>> > > > > > > > >> >>>> > >> > > > (e.g.
>> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
>> > > > > > > > >> >>>> > >> > > > > > > And this is only for TopicCommand,
>> we
>> > > will
>> > > > > > also
>> > > > > > > > >> need to
>> > > > > > > > >> >>>> add
>> > > > > > > > >> >>>> > >> > similar
>> > > > > > > > >> >>>> > >> > > > > stuff
>> > > > > > > > >> >>>> > >> > > > > > > for
>> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
>> PreferredReplica. So
>> > > > > we'll
>> > > > > > > end
>> > > > > > > > >> up
>> > > > > > > > >> >>>> with a
>> > > > > > > > >> >>>> > >> > large
>> > > > > > > > >> >>>> > >> > > > list
>> > > > > > > > >> >>>> > >> > > > > > of
>> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in Admin
>> > > protocol.
>> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree my
>> proposal is
>> > > > not
>> > > > > > > > >> consistent
>> > > > > > > > >> >>>> with
>> > > > > > > > >> >>>> > >> > other
>> > > > > > > > >> >>>> > >> > > > > cases.
>> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better solution
>> or
>> > > > > something
>> > > > > > > > >> >>>> in-between.
>> > > > > > > > >> >>>> > >> > > > > > >
>> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it is a
>> great
>> > > > idea.
>> > > > > > > This
>> > > > > > > > >> way we
>> > > > > > > > >> >>>> can
>> > > > > > > > >> >>>> > >> move
>> > > > > > > > >> >>>> > >> > > on
>> > > > > > > > >> >>>> > >> > > > > > > faster.
>> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on date/time so
>> > > people
>> > > > > can
>> > > > > > > > join.
>> > > > > > > > >> >>>> Will work
>> > > > > > > > >> >>>> > >> > for
>> > > > > > > > >> >>>> > >> > > me
>> > > > > > > > >> >>>> > >> > > > > > this
>> > > > > > > > >> >>>> > >> > > > > > > and
>> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime if agreed
>> in
>> > > > > advance.
>> > > > > > > > >> >>>> > >> > > > > > >
>> > > > > > > > >> >>>> > >> > > > > > > Thanks,
>> > > > > > > > >> >>>> > >> > > > > > > Andrii
>> > > > > > > > >> >>>> > >> > > > > > >
>> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM,
>> Jay
>> > > > Kreps <
>> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
>> > > > > > > > >> >>>> > >> > > > > wrote:
>> > > > > > > > >> >>>> > >> > > > > > >
>> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
>> > > > > > > > >> >>>> > >> > > > > > >>
>> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good error
>> handling
>> > > > > > without
>> > > > > > > > >> needing
>> > > > > > > > >> >>>> custom
>> > > > > > > > >> >>>> > >> > > > > > server-side
>> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the
>> client has
>> > > > the
>> > > > > > > > >> context to
>> > > > > > > > >> >>>> know
>> > > > > > > > >> >>>> > >> that
>> > > > > > > > >> >>>> > >> > > if
>> > > > > > > > >> >>>> > >> > > > it
>> > > > > > > > >> >>>> > >> > > > > > got
>> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic doesn't
>> exist
>> > > to
>> > > > > say
>> > > > > > > > >> "Topic X
>> > > > > > > > >> >>>> doesn't
>> > > > > > > > >> >>>> > >> > > exist"
>> > > > > > > > >> >>>> > >> > > > > > >> rather
>> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or
>> whatever).
>> > > Maybe
>> > > > > > there
>> > > > > > > > are
>> > > > > > > > >> >>>> specific
>> > > > > > > > >> >>>> > >> > cases
>> > > > > > > > >> >>>> > >> > > > > where
>> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to add
>> > > > server-side
>> > > > > > > error
>> > > > > > > > >> >>>> messages we
>> > > > > > > > >> >>>> > >> > > really
>> > > > > > > > >> >>>> > >> > > > > do
>> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a consistent
>> way
>> > > > across
>> > > > > > the
>> > > > > > > > >> protocol.
>> > > > > > > > >> >>>> > >> > > > > > >>
>> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open
>> questions
>> > > > here
>> > > > > > > from
>> > > > > > > > my
>> > > > > > > > >> >>>> previous
>> > > > > > > > >> >>>> > >> > > list. I
>> > > > > > > > >> >>>> > >> > > > > > will
>> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few days for
>> Strata
>> > > > > > though.
>> > > > > > > > >> Maybe
>> > > > > > > > >> >>>> we could
>> > > > > > > > >> >>>> > >> > do
>> > > > > > > > >> >>>> > >> > > a
>> > > > > > > > >> >>>> > >> > > > > > Google
>> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open issues
>> some
>> > > time
>> > > > > > > towards
>> > > > > > > > >> the
>> > > > > > > > >> >>>> end of
>> > > > > > > > >> >>>> > >> > next
>> > > > > > > > >> >>>> > >> > > > week
>> > > > > > > > >> >>>> > >> > > > > > for
>> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in this ticket?
>> I
>> > > have
>> > > > a
>> > > > > > > > feeling
>> > > > > > > > >> that
>> > > > > > > > >> >>>> might
>> > > > > > > > >> >>>> > >> > > > progress
>> > > > > > > > >> >>>> > >> > > > > > >> things a little faster than
>> email--I
>> > > > think
>> > > > > we
>> > > > > > > > >> could talk
>> > > > > > > > >> >>>> > >> through
>> > > > > > > > >> >>>> > >> > > > those
>> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly
>> quickly...
>> > > > > > > > >> >>>> > >> > > > > > >>
>> > > > > > > > >> >>>> > >> > > > > > >> -Jay
>> > > > > > > > >> >>>> > >> > > > > > >>
>> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM,
>> Andrii
>> > > > > > > > Biletskyi <
>> > > > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly>
>> wrote:
>> > > > > > > > >> >>>> > >> > > > > > >>
>> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
>> > > > > > > > >> >>>> > >> > > > > > >> >
>> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address some of
>> the
>> > > > issues
>> > > > > > > which
>> > > > > > > > >> were
>> > > > > > > > >> >>>> > >> mentioned
>> > > > > > > > >> >>>> > >> > > > > earlier
>> > > > > > > > >> >>>> > >> > > > > > >> about
>> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of
>> those was
>> > > > > about
>> > > > > > > > >> batching
>> > > > > > > > >> >>>> > >> > operations.
>> > > > > > > > >> >>>> > >> > > > What
>> > > > > > > > >> >>>> > >> > > > > > if
>> > > > > > > > >> >>>> > >> > > > > > >> we
>> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand approach
>> and let
>> > > > > people
>> > > > > > > > >> specify
>> > > > > > > > >> >>>> > >> topic-name
>> > > > > > > > >> >>>> > >> > > by
>> > > > > > > > >> >>>> > >> > > > > > >> regexp -
>> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most of the use
>> > > cases?
>> > > > > > > > >> >>>> > >> > > > > > >> >
>> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what information
>> should
>> > > we
>> > > > > > > > generally
>> > > > > > > > >> >>>> provide in
>> > > > > > > > >> >>>> > >> > > Admin
>> > > > > > > > >> >>>> > >> > > > > > >> > responses.
>> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin commands
>> don't
>> > > > imply
>> > > > > > > they
>> > > > > > > > >> will
>> > > > > > > > >> >>>> be used
>> > > > > > > > >> >>>> > >> > only
>> > > > > > > > >> >>>> > >> > > > in
>> > > > > > > > >> >>>> > >> > > > > > CLI
>> > > > > > > > >> >>>> > >> > > > > > >> > but,
>> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a very
>> > > important
>> > > > > > > client
>> > > > > > > > >> of this
>> > > > > > > > >> >>>> > >> > feature.
>> > > > > > > > >> >>>> > >> > > In
>> > > > > > > > >> >>>> > >> > > > > > this
>> > > > > > > > >> >>>> > >> > > > > > >> > case,
>> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would like to
>> > > provide
>> > > > > > users
>> > > > > > > > >> with
>> > > > > > > > >> >>>> rich
>> > > > > > > > >> >>>> > >> > > experience
>> > > > > > > > >> >>>> > >> > > > > in
>> > > > > > > > >> >>>> > >> > > > > > >> terms
>> > > > > > > > >> >>>> > >> > > > > > >> > of
>> > > > > > > > >> >>>> > >> > > > > > >> > getting results / errors of the
>> > > > executed
>> > > > > > > > >> commands.
>> > > > > > > > >> >>>> Usually
>> > > > > > > > >> >>>> > >> we
>> > > > > > > > >> >>>> > >> > > > supply
>> > > > > > > > >> >>>> > >> > > > > > >> with
>> > > > > > > > >> >>>> > >> > > > > > >> > responses only errorCode, which
>> looks
>> > > > > very
>> > > > > > > > >> limiting,
>> > > > > > > > >> >>>> in case
>> > > > > > > > >> >>>> > >> > of
>> > > > > > > > >> >>>> > >> > > > CLI
>> > > > > > > > >> >>>> > >> > > > > we
>> > > > > > > > >> >>>> > >> > > > > > >> may
>> > > > > > > > >> >>>> > >> > > > > > >> > want to print human readable
>> error
>> > > > > > > description.
>> > > > > > > > >> >>>> > >> > > > > > >> >
>> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into account
>> previous item
>> > > > > about
>> > > > > > > > >> batching,
>> > > > > > > > >> >>>> what
>> > > > > > > > >> >>>> > >> do
>> > > > > > > > >> >>>> > >> > > you
>> > > > > > > > >> >>>> > >> > > > > > think
>> > > > > > > > >> >>>> > >> > > > > > >> > about
>> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
>> > > > > > > > >> >>>> > >> > > > > > >> >
>> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support
>> regexp)
>> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName
>> > > > > Partitions
>> > > > > > > > >> Replicas
>> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
>> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
>> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode
>> > > > > > > > ErrorDescription
>> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
>> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription => string
>> (empty
>> > > if
>> > > > > > > > >> successful)
>> > > > > > > > >> >>>> > >> > > > > > >> >
>> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
>> TopicNameRegexp
>> > > >
>
> ...
>
> [Message clipped]

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Joel,

You are right, I removed ClusterMetadata because we have partially
what we need in TopicMetadata. Also, as Jay pointed out earlier, we
would like to have "orthogonal" API, but at the same time we need
to be backward compatible.

But I like your idea and even have some other arguments for this option:
There is also DescribeTopicRequest which was proposed in this KIP,
it returns topic configs, partitions, replication factor plus partition
ISR, ASR,
leader replica. The later part is really already there in
TopicMetadataRequest.
So again we'll have to add stuff to TMR, not to duplicate some info in
newly added requests. However, this way we'll end up with "monster"
request which returns cluster metadata, topic replication and config info
plus partition replication data. Seems logical to split TMR to
- ClusterMetadata (brokers + controller, maybe smth else)
- TopicMetadata (topic info + partition details)
But since current TMR is involved in lots of places (including network
client,
as I understand) this might be very serious change and it probably makes
sense to stick with current approach.

Thanks,
Andrii Biletskyi


On Tue, Mar 17, 2015 at 5:29 PM, Joel Koshy <jj...@gmail.com> wrote:

> I may be missing some context but hopefully this will also be covered
> today: I thought the earlier proposal where there was an explicit
> ClusterMetadata request was clearer and explicit. During the course of
> this thread I think the conclusion was that the main need was for
> controller information and that can be rolled into the topic metadata
> response but that seems a bit irrelevant to topic metadata. FWIW I
> think the full broker-list is also irrelevant to topic metadata, but
> it is already there and in use. I think there is still room for an
> explicit ClusterMetadata request since there may be other
> cluster-level information that we may want to add over time (and that
> have nothing to do with topic metadata).
>
> On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
> > Jun,
> >
> > 101. Okay, if you say that such use case is important. I also think
> > using clientId for these purposes is fine - if we already have this field
> > as part of all Wire protocol messages, why not use that.
> > I will update KIP-4 page if nobody has other ideas (which may come up
> > during the call today).
> >
> > 102.1 Agree, I'll update the KIP accordingly. I think we can add new,
> > fine-grained error codes if some error code received in specific case
> > won't give enough context to return a descriptive error message for user.
> >
> > Look forward to discussing all outstanding issues in detail today during
> > the call.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> >
> >
> > On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > 101. There may be a use case where you only want the topics to be
> created
> > > manually by admins. Currently, you can do that by disabling auto topic
> > > creation and issue topic creation from the TopicCommand. If we disable
> auto
> > > topic creation completely on the broker and don't have a way to
> distinguish
> > > between topic creation requests from the regular clients and the
> admin, we
> > > can't support manual topic creation any more. I was thinking that
> another
> > > way of distinguishing the clients making the topic creation requests is
> > > using clientId. For example, the admin tool can set it to something
> like
> > > admin and the broker can treat that clientId specially.
> > >
> > > Also, there is a related discussion in KAFKA-2020. Currently, we do the
> > > following in TopicMetadataResponse:
> > >
> > > 1. If leader is not available, we set the partition level error code to
> > > LeaderNotAvailable.
> > > 2. If a non-leader replica is not available, we take that replica out
> of
> > > the assigned replica list and isr in the response. As an indication for
> > > doing that, we set the partition level error code to
> ReplicaNotAvailable.
> > >
> > > This has a few problems. First, ReplicaNotAvailable probably shouldn't
> be
> > > an error, at least for the normal producer/consumer clients that just
> want
> > > to find out the leader. Second, it can happen that both the leader and
> > > another replica are not available at the same time. There is no error
> code
> > > to indicate both. Third, even if a replica is not available, it's still
> > > useful to return its replica id since some clients (e.g. admin tool)
> may
> > > still make use of it.
> > >
> > > One way to address this issue is to always return the replica id for
> > > leader, assigned replicas, and isr regardless of whether the
> corresponding
> > > broker is live or not. Since we also return the list of live brokers,
> the
> > > client can figure out whether a leader or a replica is live or not and
> act
> > > accordingly. This way, we don't need to set the partition level error
> code
> > > when the leader or a replica is not available. This doesn't change the
> wire
> > > protocol, but does change the semantics. Since we are evolving the
> protocol
> > > of TopicMetadataRequest here, we can potentially piggyback the change.
> > >
> > > 102.1 For those types of errors due to invalid input, shouldn't we just
> > > guard it at parameter validation time and throw
> InvalidArgumentException
> > > without even sending the request to the broker?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Jun,
> > > >
> > > > Answering your questions:
> > > >
> > > > 101. If I understand you correctly, you are saying future producer
> > > versions
> > > > (which
> > > > will be ported to TMR_V1) won't be able to automatically create
> topic (if
> > > > we
> > > > unconditionally remove topic creation from there). But we need to
> this
> > > > preserve logic.
> > > > Ok, about your proposal: I'm not a big fan too, when it comes to
> > > > differentiating
> > > > clients directly in protocol schema. And also I'm not sure I
> understand
> > > at
> > > > all why
> > > > auto.create.topics.enable is a server side configuration. Can we
> > > deprecate
> > > > this setting
> > > > in future versions, add this setting to producer and based on that
> upon
> > > > receiving
> > > > UnknownTopic create topic explicitly by a separate producer call via
> > > > adminClient?
> > > >
> > > > 102.1. Hm, yes. It's because we want to support batching and at the
> same
> > > > time we
> > > > want to give descriptive error messages for clients. Since
> AdminClient
> > > > holds the context
> > > > to construct such messages (e.g. AdminClient layer can know that
> > > > InvalidArgumentsCode
> > > > means two cases: either invalid number - e.g. -1; or
> replication-factor
> > > was
> > > > provided while
> > > > partitions argument wasn't) - I wrapped responses in Exceptions. But
> I'm
> > > > open to any
> > > > other ideas, this was just initial version.
> > > > 102.2. Yes, I agree. I'll change that to probably some other dto.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Andrii,
> > > > >
> > > > > 101. That's what I was thinking too, but it may not be that
> simple. In
> > > > > TopicMetadataRequest_V1,
> > > > > we can let it not trigger auto topic creation. Then, in the
> producer
> > > > side,
> > > > > if it gets an UnknownTopicException, it can explicitly issue a
> > > > > createTopicRequest for auto topic creation. On the consumer side,
> it
> > > will
> > > > > never issue createTopicRequest. This works when auto topic
> creation is
> > > > > enabled on the broker side. However, I am not sure how things will
> work
> > > > > when auto topic creation is disabled on the broker side. In this
> case,
> > > we
> > > > > want to have a way to manually create a topic, potentially through
> > > admin
> > > > > commands. However, then we need a way to distinguish
> createTopicRequest
> > > > > issued from the producer clients and the admin tools. May be we can
> > > add a
> > > > > new field in createTopicRequest and set it differently in the
> producer
> > > > > client and the admin client. However, I am not sure if that's the
> best
> > > > > approach.
> > > > >
> > > > > 2. Yes, refactoring existing requests is a non-trivial amount of
> work.
> > > I
> > > > > posted some comments in KAFKA-1927. We will probably have to fix
> > > > KAFKA-1927
> > > > > first, before adding the new logic in KAFKA-1694. Otherwise, the
> > > changes
> > > > > will be too big.
> > > > >
> > > > > 102. About the AdminClient:
> > > > > 102.1. It's a bit weird that we return exception in the api. It
> seems
> > > > that
> > > > > we should either return error code or throw an exception when
> getting
> > > the
> > > > > response state.
> > > > > 102.2. We probably shouldn't explicitly use the request object in
> the
> > > > api.
> > > > > Not every request evolution requires an api change.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Jun,
> > > > > >
> > > > > > Thanks for you comments. Answers inline:
> > > > > >
> > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > > > > > ReassignPartitionRequest,
> > > > > > > and PartitionsSerialized that are represented as a string, but
> > > > contain
> > > > > > > composite structures in json. Could we flatten them out
> directly in
> > > > the
> > > > > > > protocol definition as arrays/records?
> > > > > >
> > > > > >
> > > > > > Yes, now with Admin Client this looks a bit weird. My initial
> > > > motivation
> > > > > > was:
> > > > > > ReassignPartitionCommand accepts input in json, we want to remain
> > > > tools'
> > > > > > interfaces unchanged, where possible.
> > > > > > If we port it to deserialized format, in CLI (/tools project) we
> will
> > > > > have
> > > > > > to add some
> > > > > > json library since /tools is written in java and we'll need to
> > > > > deserialize
> > > > > > json file
> > > > > > provided by a user. Can we quickly agree on what this library
> should
> > > be
> > > > > > (Jackson, GSON, whatever)?
> > > > > >
> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> creation?
> > > > This
> > > > > > > will be a bit weird now that we have a separate topic creation
> api.
> > > > > Have
> > > > > > > you thought about how the new createTopicRequest and
> > > > > TopicMetadataRequest
> > > > > > > v1 will be used in the producer/consumer client, in addition to
> > > admin
> > > > > > > tools? For example, ideally, we don't want TopicMetadataRequest
> > > from
> > > > > the
> > > > > > > consumer to trigger auto topic creation.
> > > > > >
> > > > > >
> > > > > > I agree, this strange logic should be fixed. I'm not confident in
> > > this
> > > > > > Kafka part so
> > > > > > correct me if I'm wrong, but it doesn't look like a hard thing to
> > > do, I
> > > > > > think we can
> > > > > > leverage AdminClient for that in Producer and unconditionally
> remove
> > > > > topic
> > > > > > creation from the TopicMetadataRequest_V1.
> > > > > >
> > > > > > 2. I think Jay meant getting rid of scala classes
> > > > > > > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader.
> We
> > > did
> > > > > > that
> > > > > > > as a stop-gap thing when adding the new requests for the
> consumers.
> > > > > > > However, the long term plan is to get rid of all those and just
> > > reuse
> > > > > the
> > > > > > > java request/response in the client. Since this KIP proposes to
> > > add a
> > > > > > > significant number of new requests, perhaps we should bite the
> > > bullet
> > > > > to
> > > > > > > clean up the existing scala requests first before adding new
> ones?
> > > > > > >
> > > > > >
> > > > > > Yes, looks like I misunderstood the point of ...RequestAndHeader.
> > > > Okay, I
> > > > > > will
> > > > > > rework that. The only thing is that I don't see any example how
> it
> > > was
> > > > > done
> > > > > > for at
> > > > > > least one existing protocol message. Thus, as I understand, I
> have to
> > > > > think
> > > > > > how we
> > > > > > are going to do it.
> > > > > > Re porting all existing RQ/RP in this patch. Sounds reasonable,
> but
> > > if
> > > > > it's
> > > > > > an *obligatory*
> > > > > > requirement to have Admin KIP done, I'm afraid this can be a
> serious
> > > > > > blocker for us.
> > > > > > There are 13 protocol messages and all that would require not
> only
> > > unit
> > > > > > tests but quite
> > > > > > intensive manual testing, no? I'm afraid I'm not the right guy to
> > > cover
> > > > > > pretty much all
> > > > > > Kafka core internals :). Let me know your thoughts on this item.
> Btw
> > > > > there
> > > > > > is a ticket to
> > > > > > follow-up this issue (
> > > https://issues.apache.org/jira/browse/KAFKA-2006
> > > > ).
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > >
> > > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <ju...@confluent.io>
> wrote:
> > > > > >
> > > > > > > Andrii,
> > > > > > >
> > > > > > >
> > > > > > > A few more comments.
> > > > > > >
> > > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > > > > > ReassignPartitionRequest,
> > > > > > > and PartitionsSerialized that are represented as a string, but
> > > > contain
> > > > > > > composite structures in json. Could we flatten them out
> directly in
> > > > the
> > > > > > > protocol definition as arrays/records?
> > > > > > >
> > > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> > > creation?
> > > > > This
> > > > > > > will be a bit weird now that we have a separate topic creation
> api.
> > > > > Have
> > > > > > > you thought about how the new createTopicRequest and
> > > > > TopicMetadataRequest
> > > > > > > v1 will be used in the producer/consumer client, in addition to
> > > admin
> > > > > > > tools? For example, ideally, we don't want TopicMetadataRequest
> > > from
> > > > > the
> > > > > > > consumer to trigger auto topic creation.
> > > > > > >
> > > > > > > 2. I think Jay meant getting rid of scala classes
> > > > > > > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader.
> We
> > > did
> > > > > > that
> > > > > > > as a stop-gap thing when adding the new requests for the
> consumers.
> > > > > > > However, the long term plan is to get rid of all those and just
> > > reuse
> > > > > the
> > > > > > > java request/response in the client. Since this KIP proposes to
> > > add a
> > > > > > > significant number of new requests, perhaps we should bite the
> > > bullet
> > > > > to
> > > > > > > clean up the existing scala requests first before adding new
> ones?
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Jun
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >
> > > > > > > > Hi,
> > > > > > > >
> > > > > > > > As said above - I list again all comments from this thread
> so we
> > > > > > > > can see what's left and finalize all pending issues.
> > > > > > > >
> > > > > > > > Comments from Jay:
> > > > > > > > 1. This is much needed functionality, but there are a lot of
> the
> > > so
> > > > > > let's
> > > > > > > > really think these protocols through. We really want to end
> up
> > > > with a
> > > > > > set
> > > > > > > > of well thought-out, orthoganol apis. For this reason I
> think it
> > > is
> > > > > > > really
> > > > > > > > important to think through the end state even if that
> includes
> > > APIs
> > > > > we
> > > > > > > > won't implement in the first phase.
> > > > > > > >
> > > > > > > > A: Definitely behind this. Would appreciate if there are
> concrete
> > > > > > > comments
> > > > > > > > how this can be improved.
> > > > > > > >
> > > > > > > > 2. Let's please please please wait until we have switched the
> > > > server
> > > > > > over
> > > > > > > > to the new java protocol definitions. If we add upteen more
> ad
> > > hoc
> > > > > > scala
> > > > > > > > objects that is just generating more work for the conversion
> we
> > > > know
> > > > > we
> > > > > > > > have to do.
> > > > > > > >
> > > > > > > > A: Fixed in the latest patch - removed scala protocol
> classes.
> > > > > > > >
> > > > > > > > 3. This proposal introduces a new type of optional parameter.
> > > This
> > > > is
> > > > > > > > inconsistent with everything else in the protocol where we
> use -1
> > > > or
> > > > > > some
> > > > > > > > other marker value. You could argue either way but let's
> stick
> > > with
> > > > > > that
> > > > > > > > for consistency. For clients that implemented the protocol
> in a
> > > > > better
> > > > > > > way
> > > > > > > > than our scala code these basic primitives are hard to
> change.
> > > > > > > >
> > > > > > > > A: Fixed in the latest patch - removed MaybeOf type and
> changed
> > > > > > protocol
> > > > > > > > accordingly.
> > > > > > > >
> > > > > > > > 4. ClusterMetadata: This seems to duplicate
> TopicMetadataRequest
> > > > > which
> > > > > > > has
> > > > > > > > brokers, topics, and partitions. I think we should rename
> that
> > > > > request
> > > > > > > > ClusterMetadataRequest (or just MetadataRequest) and include
> the
> > > id
> > > > > of
> > > > > > > the
> > > > > > > > controller. Or are there other things we could add here?
> > > > > > > >
> > > > > > > > A: I agree. Updated the KIP. Let's extends TopicMetadata to
> > > > version 2
> > > > > > and
> > > > > > > > include controller.
> > > > > > > >
> > > > > > > > 5. We have a tendency to try to make a lot of requests that
> can
> > > > only
> > > > > go
> > > > > > > to
> > > > > > > > particular nodes. This adds a lot of burden for client
> > > > > implementations
> > > > > > > (it
> > > > > > > > sounds easy but each discovery can fail in many parts so it
> ends
> > > up
> > > > > > > being a
> > > > > > > > full state machine to do right). I think we should consider
> > > making
> > > > > > admin
> > > > > > > > commands and ideally as many of the other apis as possible
> > > > available
> > > > > on
> > > > > > > all
> > > > > > > > brokers and just redirect to the controller on the broker
> side.
> > > > > Perhaps
> > > > > > > > there would be a general way to encapsulate this re-routing
> > > > behavior.
> > > > > > > >
> > > > > > > > A: It's a very interesting idea, but seems there are some
> > > concerns
> > > > > > about
> > > > > > > > this
> > > > > > > > feature (like performance considerations, how this will
> > > complicate
> > > > > > server
> > > > > > > > etc).
> > > > > > > > I believe this shouldn't be a blocker. If this feature is
> > > > implemented
> > > > > > at
> > > > > > > > some
> > > > > > > > point it won't affect Admin changes - at least no changes to
> > > public
> > > > > API
> > > > > > > > will be required.
> > > > > > > >
> > > > > > > > 6. We should probably normalize the key value pairs used for
> > > > configs
> > > > > > > rather
> > > > > > > > than embedding a new formatting. So two strings rather than
> one
> > > > with
> > > > > an
> > > > > > > > internal equals sign.
> > > > > > > >
> > > > > > > > A: Fixed in the latest patch - normalized configs and changed
> > > > > protocol
> > > > > > > > accordingly.
> > > > > > > >
> > > > > > > > 7. Is the postcondition of these APIs that the command has
> begun
> > > or
> > > > > > that
> > > > > > > > the command has been completed? It is a lot more usable if
> the
> > > > > command
> > > > > > > has
> > > > > > > > been completed so you know that if you create a topic and
> then
> > > > > publish
> > > > > > to
> > > > > > > > it you won't get an exception about there being no such
> topic.
> > > > > > > >
> > > > > > > > A: For long running requests (like reassign partitions) - the
> > > post
> > > > > > > > condition is
> > > > > > > > command has begun - so we don't block the client. In case of
> your
> > > > > > > example -
> > > > > > > > topic commands, this will be refactored and topic commands
> will
> > > be
> > > > > > > executed
> > > > > > > > immediately, since the Controller will serve Admin requests
> > > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > > >
> > > > > > > > 8. Describe topic and list topics duplicate a lot of stuff
> in the
> > > > > > > metadata
> > > > > > > > request. Is there a reason to give back topics marked for
> > > > deletion? I
> > > > > > > feel
> > > > > > > > like if we just make the post-condition of the delete
> command be
> > > > that
> > > > > > the
> > > > > > > > topic is deleted that will get rid of the need for this
> right?
> > > And
> > > > it
> > > > > > > will
> > > > > > > > be much more intuitive.
> > > > > > > >
> > > > > > > > A: Fixed in the latest patch - removed topics marked for
> deletion
> > > > in
> > > > > > > > ListTopicsRequest.
> > > > > > > >
> > > > > > > > 9. Should we consider batching these requests? We have
> generally
> > > > > tried
> > > > > > to
> > > > > > > > allow multiple operations to be batched. My suspicion is that
> > > > without
> > > > > > > this
> > > > > > > > we will get a lot of code that does something like
> > > > > > > >    for(topic: adminClient.listTopics())
> > > > > > > >       adminClient.describeTopic(topic)
> > > > > > > > this code will work great when you test on 5 topics but not
> do as
> > > > > well
> > > > > > if
> > > > > > > > you have 50k.
> > > > > > > >
> > > > > > > > A: Updated the KIP - please check "Topic Admin Schema"
> section.
> > > > > > > >
> > > > > > > > 10. I think we should also discuss how we want to expose a
> > > > > programmatic
> > > > > > > JVM
> > > > > > > > client api for these operations. Currently people rely on
> > > > AdminUtils
> > > > > > > which
> > > > > > > > is totally sketchy. I think we probably need another client
> under
> > > > > > > clients/
> > > > > > > > that exposes administrative functionality. We will need this
> just
> > > > to
> > > > > > > > properly test the new apis, I suspect. We should figure out
> that
> > > > API.
> > > > > > > >
> > > > > > > > A: Updated the KIP - please check "Admin Client" section
> with an
> > > > > > initial
> > > > > > > > API proposal.
> > > > > > > >
> > > > > > > > 11. The other information that would be really useful to get
> > > would
> > > > be
> > > > > > > > information about partitions--how much data is in the
> partition,
> > > > what
> > > > > > are
> > > > > > > > the segment offsets, what is the log-end offset (i.e. last
> > > offset),
> > > > > > what
> > > > > > > is
> > > > > > > > the compaction point, etc. I think that done right this
> would be
> > > > the
> > > > > > > > successor to the very awkward OffsetRequest we have today.
> > > > > > > >
> > > > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest
> patch. I
> > > > > believe
> > > > > > > > this should
> > > > > > > > be resolved in a separate KIP / jira ticket.
> > > > > > > >
> > > > > > > > 12. Generally we can do good error handling without needing
> > > custom
> > > > > > > > server-side
> > > > > > > > messages. I.e. generally the client has the context to know
> that
> > > if
> > > > > it
> > > > > > > got
> > > > > > > > an error that the topic doesn't exist to say "Topic X doesn't
> > > > exist"
> > > > > > > rather
> > > > > > > > than "error code 14" (or whatever). Maybe there are specific
> > > cases
> > > > > > where
> > > > > > > > this is hard? If we want to add server-side error messages we
> > > > really
> > > > > do
> > > > > > > > need to do this in a consistent way across the protocol.
> > > > > > > >
> > > > > > > > A: Updated the KIP - please check "Protocol Errors" section.
> I
> > > > added
> > > > > > the
> > > > > > > > comprehensive, fine-grained list of error codes.
> > > > > > > >
> > > > > > > > Comments from Guozhang:
> > > > > > > > 13. Describe topic request: it would be great to go beyond
> just
> > > > > > batching
> > > > > > > on
> > > > > > > > topic name regex for this request. For example, a very
> common use
> > > > > case
> > > > > > of
> > > > > > > > the topic command is to list all topics whose config A's
> value is
> > > > B.
> > > > > > With
> > > > > > > > topic name regex then we have to first retrieve __all__
> topics's
> > > > > > > > description info and then filter at the client end, which
> will
> > > be a
> > > > > > huge
> > > > > > > > burden on ZK.
> > > > > > > > AND
> > > > > > > > 14. Config K-Vs in create topic: this is related to the
> previous
> > > > > point;
> > > > > > > > maybe we can add another metadata K-V or just a metadata
> string
> > > > along
> > > > > > > side
> > > > > > > > with config K-V in create topic like we did for offset commit
> > > > > request.
> > > > > > > This
> > > > > > > > field can be quite useful in storing information like
> "owner" of
> > > > the
> > > > > > > topic
> > > > > > > > who issue the create command, etc, which is quite important
> for a
> > > > > > > > multi-tenant setting. Then in the describe topic request we
> can
> > > > also
> > > > > > > batch
> > > > > > > > on regex of the metadata field.
> > > > > > > >
> > > > > > > > A: As discussed it is very interesting but can be implemented
> > > later
> > > > > > after
> > > > > > > > we have some basic functionality there.
> > > > > > > >
> > > > > > > > 15. Today all the admin operations are async in the sense
> that
> > > > > command
> > > > > > > will
> > > > > > > > return once it is written in ZK, and that is why we need
> extra
> > > > > > > verification
> > > > > > > > like testUtil.waitForTopicCreated() / verify partition
> > > reassignment
> > > > > > > > request, etc. With admin requests we could add a flag to
> enable /
> > > > > > disable
> > > > > > > > synchronous requests; when it is turned on, the response
> will not
> > > > > > return
> > > > > > > > until the request has been completed. And for async requests
> we
> > > can
> > > > > > add a
> > > > > > > > "token" field in the response, and then only need a general
> > > "admin
> > > > > > > > verification request" with the given token to check if the
> async
> > > > > > request
> > > > > > > > has been completed.
> > > > > > > >
> > > > > > > > A: I see your point. My idea was to provide specific
> > > > Verify...Request
> > > > > > per
> > > > > > > > each
> > > > > > > > long running request, where needed. We can do it the way you
> > > > suggest.
> > > > > > The
> > > > > > > > only
> > > > > > > > concern is that introducing a token we again will make schema
> > > > > > "dynamic".
> > > > > > > We
> > > > > > > > wanted
> > > > > > > > to do similar thing introducing single AdminRequest for all
> topic
> > > > > > > commands
> > > > > > > > but rejected
> > > > > > > > this idea because we wanted to have schema defined. So this
> is
> > > > more a
> > > > > > > > choice between:
> > > > > > > > a) have fixed schema but introduce each time new
> Verify...Request
> > > > for
> > > > > > > > long-running requests
> > > > > > > > b) use one request for verification but generalize it with
> token
> > > > > > > > I'm fine with whatever decision community come to. Just let
> me
> > > know
> > > > > > your
> > > > > > > > thoughts.
> > > > > > > >
> > > > > > > > Comment from Gwen:
> > > > > > > > 16. Specifically for ownership, I think the plan is to add
> ACL
> > > (it
> > > > > > sounds
> > > > > > > > like you are describing ACL) via an external system (Argus,
> > > > Sentry).
> > > > > > > > I remember KIP-11 described this, but I can't find the KIP
> any
> > > > > longer.
> > > > > > > >
> > > > > > > > A: Okay, no problem. Not sure though how we are going to
> handle
> > > it.
> > > > > > Wait
> > > > > > > > which KIP
> > > > > > > > will be committed first and include changes to TopicMetadata
> from
> > > > the
> > > > > > > later
> > > > > > > > one?
> > > > > > > > Anyway, I added this note to "Open Questions" section so we
> don't
> > > > > miss
> > > > > > > this
> > > > > > > > piece.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Andrii Biletskyi
> > > > > > > >
> > > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >
> > > > > > > > > Hi all,
> > > > > > > > >
> > > > > > > > > Today I uploaded the patch that covers some of the
> discussed
> > > and
> > > > > > agreed
> > > > > > > > > items:
> > > > > > > > > - removed MaybeOf optional type
> > > > > > > > > - switched to java protocol definitions
> > > > > > > > > - simplified messages (normalized configs, removed topic
> marked
> > > > for
> > > > > > > > > deletion)
> > > > > > > > >
> > > > > > > > > I also updated the KIP-4 with respective changes and wrote
> down
> > > > my
> > > > > > > > > proposal for
> > > > > > > > > pending items:
> > > > > > > > > - Batch Admin Operations -> updated Wire Protocol schema
> > > proposal
> > > > > > > > > - Remove ClusterMetadata -> changed to extend
> > > > TopicMetadataRequest
> > > > > > > > > - Admin Client -> updated my initial proposal to reflect
> > > batching
> > > > > > > > > - Error codes -> proposed fine-grained error code instead
> of
> > > > > > > > > AdminRequestFailed
> > > > > > > > >
> > > > > > > > > I will also send a separate email to cover all comments
> from
> > > this
> > > > > > > thread.
> > > > > > > > >
> > > > > > > > > Thanks,
> > > > > > > > > Andrii Biletskyi
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> > > > > gshapira@cloudera.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > >> Found KIP-11 (
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > > >> )
> > > > > > > > >> It actually specifies changes to the Metadata protocol, so
> > > > making
> > > > > > sure
> > > > > > > > >> both KIPs are consistent in this regard will be good.
> > > > > > > > >>
> > > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> > > > > > gshapira@cloudera.com
> > > > > > > >
> > > > > > > > >> wrote:
> > > > > > > > >> > Specifically for ownership, I think the plan is to add
> ACL
> > > (it
> > > > > > > sounds
> > > > > > > > >> > like you are describing ACL) via an external system
> (Argus,
> > > > > > Sentry).
> > > > > > > > >> > I remember KIP-11 described this, but I can't find the
> KIP
> > > any
> > > > > > > longer.
> > > > > > > > >> >
> > > > > > > > >> > Regardless, I think KIP-4 focuses on getting information
> > > that
> > > > > > > already
> > > > > > > > >> > exists from Kafka brokers, not on adding information
> that
> > > > > perhaps
> > > > > > > > >> > should exist but doesn't yet?
> > > > > > > > >> >
> > > > > > > > >> > Gwen
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> >
> > > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> > > > > > wangguoz@gmail.com>
> > > > > > > > >> wrote:
> > > > > > > > >> >> Folks,
> > > > > > > > >> >>
> > > > > > > > >> >> Just want to elaborate a bit more on the create-topic
> > > > metadata
> > > > > > and
> > > > > > > > >> batching
> > > > > > > > >> >> describe-topic based on config / metadata in my
> previous
> > > > email
> > > > > as
> > > > > > > we
> > > > > > > > >> work
> > > > > > > > >> >> on KAFKA-1694. The main motivation is to have some
> sort of
> > > > > topic
> > > > > > > > >> management
> > > > > > > > >> >> mechanisms, which I think is quite important in a
> > > > multi-tenant
> > > > > /
> > > > > > > > cloud
> > > > > > > > >> >> architecture: today anyone can create topics in a
> shared
> > > > Kafka
> > > > > > > > >> cluster, but
> > > > > > > > >> >> there is no concept or "ownership" of topics that are
> > > created
> > > > > by
> > > > > > > > >> different
> > > > > > > > >> >> users. For example, at LinkedIn we basically
> distinguish
> > > > topic
> > > > > > > owners
> > > > > > > > >> via
> > > > > > > > >> >> some casual topic name prefix, which is a bit awkward
> and
> > > > does
> > > > > > not
> > > > > > > > fly
> > > > > > > > >> as
> > > > > > > > >> >> we scale our customers. It would be great to use
> > > > > describe-topics
> > > > > > > such
> > > > > > > > >> as:
> > > > > > > > >> >>
> > > > > > > > >> >> Describe all topics that is created by me.
> > > > > > > > >> >>
> > > > > > > > >> >> Describe all topics whose retention time is overriden
> to X.
> > > > > > > > >> >>
> > > > > > > > >> >> Describe all topics whose writable group include user Y
> > > (this
> > > > > is
> > > > > > > > >> related to
> > > > > > > > >> >> authorization), etc..
> > > > > > > > >> >>
> > > > > > > > >> >> One possible way to achieve this is to add a metadata
> file
> > > in
> > > > > the
> > > > > > > > >> >> create-topic request, whose value will also be written
> ZK
> > > as
> > > > we
> > > > > > > > create
> > > > > > > > >> the
> > > > > > > > >> >> topic; then describe-topics can choose to batch topics
> > > based
> > > > on
> > > > > > 1)
> > > > > > > > name
> > > > > > > > >> >> regex, 2) config K-V matching, 3) metadata regex, etc.
> > > > > > > > >> >>
> > > > > > > > >> >> Thoughts?
> > > > > > > > >> >>
> > > > > > > > >> >> Guozhang
> > > > > > > > >> >>
> > > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> > > > > > wangguoz@gmail.com>
> > > > > > > > >> wrote:
> > > > > > > > >> >>
> > > > > > > > >> >>> Thanks for the updated wiki. A few comments below:
> > > > > > > > >> >>>
> > > > > > > > >> >>> 1. Error description in response: I think if some
> > > errorCode
> > > > > > could
> > > > > > > > >> indicate
> > > > > > > > >> >>> several different error cases then we should really
> change
> > > > it
> > > > > to
> > > > > > > > >> multiple
> > > > > > > > >> >>> codes. In general the errorCode itself would be
> precise
> > > and
> > > > > > > > >> sufficient for
> > > > > > > > >> >>> describing the server side errors.
> > > > > > > > >> >>>
> > > > > > > > >> >>> 2. Describe topic request: it would be great to go
> beyond
> > > > just
> > > > > > > > >> batching on
> > > > > > > > >> >>> topic name regex for this request. For example, a very
> > > > common
> > > > > > use
> > > > > > > > >> case of
> > > > > > > > >> >>> the topic command is to list all topics whose config
> A's
> > > > value
> > > > > > is
> > > > > > > B.
> > > > > > > > >> With
> > > > > > > > >> >>> topic name regex then we have to first retrieve
> __all__
> > > > > topics's
> > > > > > > > >> >>> description info and then filter at the client end,
> which
> > > > will
> > > > > > be
> > > > > > > a
> > > > > > > > >> huge
> > > > > > > > >> >>> burden on ZK.
> > > > > > > > >> >>>
> > > > > > > > >> >>> 3. Config K-Vs in create topic: this is related to the
> > > > > previous
> > > > > > > > point;
> > > > > > > > >> >>> maybe we can add another metadata K-V or just a
> metadata
> > > > > string
> > > > > > > > along
> > > > > > > > >> side
> > > > > > > > >> >>> with config K-V in create topic like we did for offset
> > > > commit
> > > > > > > > >> request. This
> > > > > > > > >> >>> field can be quite useful in storing information like
> > > > "owner"
> > > > > of
> > > > > > > the
> > > > > > > > >> topic
> > > > > > > > >> >>> who issue the create command, etc, which is quite
> > > important
> > > > > for
> > > > > > a
> > > > > > > > >> >>> multi-tenant setting. Then in the describe topic
> request
> > > we
> > > > > can
> > > > > > > also
> > > > > > > > >> batch
> > > > > > > > >> >>> on regex of the metadata field.
> > > > > > > > >> >>>
> > > > > > > > >> >>> 4. Today all the admin operations are async in the
> sense
> > > > that
> > > > > > > > command
> > > > > > > > >> will
> > > > > > > > >> >>> return once it is written in ZK, and that is why we
> need
> > > > extra
> > > > > > > > >> verification
> > > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify partition
> > > > > > > reassignment
> > > > > > > > >> >>> request, etc. With admin requests we could add a flag
> to
> > > > > enable
> > > > > > /
> > > > > > > > >> disable
> > > > > > > > >> >>> synchronous requests; when it is turned on, the
> response
> > > > will
> > > > > > not
> > > > > > > > >> return
> > > > > > > > >> >>> until the request has been completed. And for async
> > > requests
> > > > > we
> > > > > > > can
> > > > > > > > >> add a
> > > > > > > > >> >>> "token" field in the response, and then only need a
> > > general
> > > > > > "admin
> > > > > > > > >> >>> verification request" with the given token to check
> if the
> > > > > async
> > > > > > > > >> request
> > > > > > > > >> >>> has been completed.
> > > > > > > > >> >>>
> > > > > > > > >> >>> 5. +1 for extending Metadata request to include
> > > controller /
> > > > > > > > >> coordinator
> > > > > > > > >> >>> information, and then we can remove the
> ConsumerMetadata /
> > > > > > > > >> ClusterMetadata
> > > > > > > > >> >>> requests.
> > > > > > > > >> >>>
> > > > > > > > >> >>> Guozhang
> > > > > > > > >> >>>
> > > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> > > > > > jjkoshy.w@gmail.com>
> > > > > > > > >> wrote:
> > > > > > > > >> >>>
> > > > > > > > >> >>>> Thanks for sending that out Joe - I don't think I
> will be
> > > > > able
> > > > > > to
> > > > > > > > >> make
> > > > > > > > >> >>>> it today, so if notes can be sent out afterward that
> > > would
> > > > be
> > > > > > > > great.
> > > > > > > > >> >>>>
> > > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen
> Shapira
> > > > wrote:
> > > > > > > > >> >>>> > Thanks for sending this out Joe. Looking forward to
> > > > > chatting
> > > > > > > with
> > > > > > > > >> >>>> everyone :)
> > > > > > > > >> >>>> >
> > > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> > > > > > > joe.stein@stealth.ly>
> > > > > > > > >> wrote:
> > > > > > > > >> >>>> > > Hey, I just sent out a google hangout invite to
> all
> > > > pmc,
> > > > > > > > >> committers
> > > > > > > > >> >>>> and
> > > > > > > > >> >>>> > > everyone I found working on a KIP. If I missed
> anyone
> > > > in
> > > > > > the
> > > > > > > > >> invite
> > > > > > > > >> >>>> please
> > > > > > > > >> >>>> > > let me know and can update it, np.
> > > > > > > > >> >>>> > >
> > > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm Eastern
> Time.
> > > > Maybe
> > > > > > we
> > > > > > > > can
> > > > > > > > >> get
> > > > > > > > >> >>>> INFRA
> > > > > > > > >> >>>> > > help to make a google account so we can manage
> > > better?
> > > > > > > > >> >>>> > >
> > > > > > > > >> >>>> > > To discuss
> > > > > > > > >> >>>> > >
> > > > > > > > >> >>>>
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > > >> >>>> > > in progress and related JIRA that are
> interdependent
> > > > and
> > > > > > > common
> > > > > > > > >> work.
> > > > > > > > >> >>>> > >
> > > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > > >> >>>> > >
> > > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
> > > > > > > > jay.kreps@gmail.com>
> > > > > > > > >> >>>> wrote:
> > > > > > > > >> >>>> > >
> > > > > > > > >> >>>> > >> Let's stay on Google hangouts that will also
> record
> > > > and
> > > > > > make
> > > > > > > > the
> > > > > > > > >> >>>> sessions
> > > > > > > > >> >>>> > >> available on youtube.
> > > > > > > > >> >>>> > >>
> > > > > > > > >> >>>> > >> -Jay
> > > > > > > > >> >>>> > >>
> > > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> > > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > >> >>>> > >>
> > > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >> > We're happy to send out a Webex for this
> purpose.
> > > We
> > > > > > could
> > > > > > > > >> record
> > > > > > > > >> >>>> the
> > > > > > > > >> >>>> > >> > sessions if there is interest and publish them
> > > out.
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >> > Thanks
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >> > Jeff
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> > > > > > > > >> jay.kreps@gmail.com>
> > > > > > > > >> >>>> wrote:
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >> > > Let's try to get the technical hang-ups
> sorted
> > > > out,
> > > > > > > > though.
> > > > > > > > >> I
> > > > > > > > >> >>>> really
> > > > > > > > >> >>>> > >> > think
> > > > > > > > >> >>>> > >> > > there is some benefit to live discussion vs
> > > > > writing. I
> > > > > > > am
> > > > > > > > >> >>>> hopeful that
> > > > > > > > >> >>>> > >> if
> > > > > > > > >> >>>> > >> > > we post instructions and give ourselves a
> few
> > > > > attempts
> > > > > > > we
> > > > > > > > >> can
> > > > > > > > >> >>>> get it
> > > > > > > > >> >>>> > >> > > working.
> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> >>>> > >> > > Tuesday at that time would work for me...any
> > > > > > objections?
> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> >>>> > >> > > -Jay
> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> > > > > > > > >> joe.stein@stealth.ly
> > > > > > > > >> >>>> >
> > > > > > > > >> >>>> > >> wrote:
> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> >>>> > >> > > > Weekly would be great maybe like every
> > > Tuesday ~
> > > > > 1pm
> > > > > > > ET
> > > > > > > > /
> > > > > > > > >> 10am
> > > > > > > > >> >>>> PT
> > > > > > > > >> >>>> > >> ????
> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> >>>> > >> > > > I don't mind google hangout but there is
> > > always
> > > > > some
> > > > > > > > >> issue or
> > > > > > > > >> >>>> > >> whatever
> > > > > > > > >> >>>> > >> > so
> > > > > > > > >> >>>> > >> > > > we know the apache irc channel works. We
> can
> > > > start
> > > > > > > there
> > > > > > > > >> and
> > > > > > > > >> >>>> see how
> > > > > > > > >> >>>> > >> it
> > > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too and
> > > associate
> > > > to
> > > > > > > > >> tickets if
> > > > > > > > >> >>>> need be
> > > > > > > > >> >>>> > >> > > makes
> > > > > > > > >> >>>> > >> > > > it helpful for things.
> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay
> Kreps <
> > > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > > > >> >>>> > >> > wrote:
> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> >>>> > >> > > > > We'd talked about doing a Google
> Hangout to
> > > > chat
> > > > > > > about
> > > > > > > > >> this.
> > > > > > > > >> >>>> What
> > > > > > > > >> >>>> > >> > about
> > > > > > > > >> >>>> > >> > > > > generalizing that a little further...I
> > > > actually
> > > > > > > think
> > > > > > > > it
> > > > > > > > >> >>>> would be
> > > > > > > > >> >>>> > >> > good
> > > > > > > > >> >>>> > >> > > > for
> > > > > > > > >> >>>> > >> > > > > everyone spending a reasonable chunk of
> > > their
> > > > > week
> > > > > > > on
> > > > > > > > >> Kafka
> > > > > > > > >> >>>> stuff
> > > > > > > > >> >>>> > >> to
> > > > > > > > >> >>>> > >> > > > maybe
> > > > > > > > >> >>>> > >> > > > > sync up once a week. I think we could
> use
> > > time
> > > > > to
> > > > > > > talk
> > > > > > > > >> >>>> through
> > > > > > > > >> >>>> > >> design
> > > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of code
> > > > reviews,
> > > > > > talk
> > > > > > > > >> through
> > > > > > > > >> >>>> any
> > > > > > > > >> >>>> > >> > tricky
> > > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > We can make it publicly available so
> that
> > > any
> > > > > one
> > > > > > > can
> > > > > > > > >> follow
> > > > > > > > >> >>>> along
> > > > > > > > >> >>>> > >> > who
> > > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > Any interest in doing this? If so I'll
> try
> > > to
> > > > > set
> > > > > > it
> > > > > > > > up
> > > > > > > > >> >>>> starting
> > > > > > > > >> >>>> > >> next
> > > > > > > > >> >>>> > >> > > > week.
> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii
> > > > > Biletskyi
> > > > > > <
> > > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed / aligned
> > > > > document
> > > > > > > > >> structure.
> > > > > > > > >> >>>> Also I
> > > > > > > > >> >>>> > >> > > added
> > > > > > > > >> >>>> > >> > > > > > some
> > > > > > > > >> >>>> > >> > > > > > very initial proposal for AdminClient
> so
> > > we
> > > > > have
> > > > > > > > >> something
> > > > > > > > >> >>>> to
> > > > > > > > >> >>>> > >> start
> > > > > > > > >> >>>> > >> > > > from
> > > > > > > > >> >>>> > >> > > > > > while
> > > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >>
> > > > > > > > >> >>>>
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > Thanks,
> > > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM,
> Andrii
> > > > > > Biletskyi
> > > > > > > <
> > > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > > Jay,
> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > > > Re error messages: you are right, in
> > > most
> > > > > > cases
> > > > > > > > >> client
> > > > > > > > >> >>>> will
> > > > > > > > >> >>>> > >> have
> > > > > > > > >> >>>> > >> > > > enough
> > > > > > > > >> >>>> > >> > > > > > > context to show descriptive error
> > > message.
> > > > > My
> > > > > > > > >> concern is
> > > > > > > > >> >>>> that
> > > > > > > > >> >>>> > >> we
> > > > > > > > >> >>>> > >> > > will
> > > > > > > > >> >>>> > >> > > > > > have
> > > > > > > > >> >>>> > >> > > > > > > to
> > > > > > > > >> >>>> > >> > > > > > > add lots of new error codes for each
> > > > > possible
> > > > > > > > >> error. Of
> > > > > > > > >> >>>> course,
> > > > > > > > >> >>>> > >> > we
> > > > > > > > >> >>>> > >> > > > > could
> > > > > > > > >> >>>> > >> > > > > > > reuse
> > > > > > > > >> >>>> > >> > > > > > > some of existing like
> > > > > > > UknownTopicOrPartitionCode,
> > > > > > > > >> but we
> > > > > > > > >> >>>> will
> > > > > > > > >> >>>> > >> > also
> > > > > > > > >> >>>> > >> > > > need
> > > > > > > > >> >>>> > >> > > > > > to
> > > > > > > > >> >>>> > >> > > > > > > add smth like:
> TopicAlreadyExistsCode,
> > > > > > > > >> >>>> TopicConfigInvalid (both
> > > > > > > > >> >>>> > >> > for
> > > > > > > > >> >>>> > >> > > > > topic
> > > > > > > > >> >>>> > >> > > > > > > name and config, and probably user
> would
> > > > > like
> > > > > > to
> > > > > > > > >> know
> > > > > > > > >> >>>> what
> > > > > > > > >> >>>> > >> > exactly
> > > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
> > > > > > > InvalidReplicaAssignment,
> > > > > > > > >> >>>> > >> InternalError
> > > > > > > > >> >>>> > >> > > > (e.g.
> > > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > > > > > > >> >>>> > >> > > > > > > And this is only for TopicCommand,
> we
> > > will
> > > > > > also
> > > > > > > > >> need to
> > > > > > > > >> >>>> add
> > > > > > > > >> >>>> > >> > similar
> > > > > > > > >> >>>> > >> > > > > stuff
> > > > > > > > >> >>>> > >> > > > > > > for
> > > > > > > > >> >>>> > >> > > > > > > ReassignPartitions,
> PreferredReplica. So
> > > > > we'll
> > > > > > > end
> > > > > > > > >> up
> > > > > > > > >> >>>> with a
> > > > > > > > >> >>>> > >> > large
> > > > > > > > >> >>>> > >> > > > list
> > > > > > > > >> >>>> > >> > > > > > of
> > > > > > > > >> >>>> > >> > > > > > > error codes, used only in Admin
> > > protocol.
> > > > > > > > >> >>>> > >> > > > > > > Having said that, I agree my
> proposal is
> > > > not
> > > > > > > > >> consistent
> > > > > > > > >> >>>> with
> > > > > > > > >> >>>> > >> > other
> > > > > > > > >> >>>> > >> > > > > cases.
> > > > > > > > >> >>>> > >> > > > > > > Maybe we can find better solution or
> > > > > something
> > > > > > > > >> >>>> in-between.
> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it is a
> great
> > > > idea.
> > > > > > > This
> > > > > > > > >> way we
> > > > > > > > >> >>>> can
> > > > > > > > >> >>>> > >> move
> > > > > > > > >> >>>> > >> > > on
> > > > > > > > >> >>>> > >> > > > > > > faster.
> > > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on date/time so
> > > people
> > > > > can
> > > > > > > > join.
> > > > > > > > >> >>>> Will work
> > > > > > > > >> >>>> > >> > for
> > > > > > > > >> >>>> > >> > > me
> > > > > > > > >> >>>> > >> > > > > > this
> > > > > > > > >> >>>> > >> > > > > > > and
> > > > > > > > >> >>>> > >> > > > > > > next week almost anytime if agreed
> in
> > > > > advance.
> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > > > > > > >> >>>> > >> > > > > > > Andrii
> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay
> > > > Kreps <
> > > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > > > > > >> >>>> > >> > > > > wrote:
> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> >>>> > >> > > > > > >> Generally we can do good error
> handling
> > > > > > without
> > > > > > > > >> needing
> > > > > > > > >> >>>> custom
> > > > > > > > >> >>>> > >> > > > > > server-side
> > > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the
> client has
> > > > the
> > > > > > > > >> context to
> > > > > > > > >> >>>> know
> > > > > > > > >> >>>> > >> that
> > > > > > > > >> >>>> > >> > > if
> > > > > > > > >> >>>> > >> > > > it
> > > > > > > > >> >>>> > >> > > > > > got
> > > > > > > > >> >>>> > >> > > > > > >> an error that the topic doesn't
> exist
> > > to
> > > > > say
> > > > > > > > >> "Topic X
> > > > > > > > >> >>>> doesn't
> > > > > > > > >> >>>> > >> > > exist"
> > > > > > > > >> >>>> > >> > > > > > >> rather
> > > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or whatever).
> > > Maybe
> > > > > > there
> > > > > > > > are
> > > > > > > > >> >>>> specific
> > > > > > > > >> >>>> > >> > cases
> > > > > > > > >> >>>> > >> > > > > where
> > > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to add
> > > > server-side
> > > > > > > error
> > > > > > > > >> >>>> messages we
> > > > > > > > >> >>>> > >> > > really
> > > > > > > > >> >>>> > >> > > > > do
> > > > > > > > >> >>>> > >> > > > > > >> need to do this in a consistent way
> > > > across
> > > > > > the
> > > > > > > > >> protocol.
> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open
> questions
> > > > here
> > > > > > > from
> > > > > > > > my
> > > > > > > > >> >>>> previous
> > > > > > > > >> >>>> > >> > > list. I
> > > > > > > > >> >>>> > >> > > > > > will
> > > > > > > > >> >>>> > >> > > > > > >> be out for the next few days for
> Strata
> > > > > > though.
> > > > > > > > >> Maybe
> > > > > > > > >> >>>> we could
> > > > > > > > >> >>>> > >> > do
> > > > > > > > >> >>>> > >> > > a
> > > > > > > > >> >>>> > >> > > > > > Google
> > > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open issues
> some
> > > time
> > > > > > > towards
> > > > > > > > >> the
> > > > > > > > >> >>>> end of
> > > > > > > > >> >>>> > >> > next
> > > > > > > > >> >>>> > >> > > > week
> > > > > > > > >> >>>> > >> > > > > > for
> > > > > > > > >> >>>> > >> > > > > > >> anyone interested in this ticket? I
> > > have
> > > > a
> > > > > > > > feeling
> > > > > > > > >> that
> > > > > > > > >> >>>> might
> > > > > > > > >> >>>> > >> > > > progress
> > > > > > > > >> >>>> > >> > > > > > >> things a little faster than
> email--I
> > > > think
> > > > > we
> > > > > > > > >> could talk
> > > > > > > > >> >>>> > >> through
> > > > > > > > >> >>>> > >> > > > those
> > > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly
> quickly...
> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> >>>> > >> > > > > > >> -Jay
> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM,
> Andrii
> > > > > > > > Biletskyi <
> > > > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly>
> wrote:
> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address some of the
> > > > issues
> > > > > > > which
> > > > > > > > >> were
> > > > > > > > >> >>>> > >> mentioned
> > > > > > > > >> >>>> > >> > > > > earlier
> > > > > > > > >> >>>> > >> > > > > > >> about
> > > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of those
> was
> > > > > about
> > > > > > > > >> batching
> > > > > > > > >> >>>> > >> > operations.
> > > > > > > > >> >>>> > >> > > > What
> > > > > > > > >> >>>> > >> > > > > > if
> > > > > > > > >> >>>> > >> > > > > > >> we
> > > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand approach and
> let
> > > > > people
> > > > > > > > >> specify
> > > > > > > > >> >>>> > >> topic-name
> > > > > > > > >> >>>> > >> > > by
> > > > > > > > >> >>>> > >> > > > > > >> regexp -
> > > > > > > > >> >>>> > >> > > > > > >> > would that cover most of the use
> > > cases?
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what information
> should
> > > we
> > > > > > > > generally
> > > > > > > > >> >>>> provide in
> > > > > > > > >> >>>> > >> > > Admin
> > > > > > > > >> >>>> > >> > > > > > >> > responses.
> > > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin commands
> don't
> > > > imply
> > > > > > > they
> > > > > > > > >> will
> > > > > > > > >> >>>> be used
> > > > > > > > >> >>>> > >> > only
> > > > > > > > >> >>>> > >> > > > in
> > > > > > > > >> >>>> > >> > > > > > CLI
> > > > > > > > >> >>>> > >> > > > > > >> > but,
> > > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a very
> > > important
> > > > > > > client
> > > > > > > > >> of this
> > > > > > > > >> >>>> > >> > feature.
> > > > > > > > >> >>>> > >> > > In
> > > > > > > > >> >>>> > >> > > > > > this
> > > > > > > > >> >>>> > >> > > > > > >> > case,
> > > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would like to
> > > provide
> > > > > > users
> > > > > > > > >> with
> > > > > > > > >> >>>> rich
> > > > > > > > >> >>>> > >> > > experience
> > > > > > > > >> >>>> > >> > > > > in
> > > > > > > > >> >>>> > >> > > > > > >> terms
> > > > > > > > >> >>>> > >> > > > > > >> > of
> > > > > > > > >> >>>> > >> > > > > > >> > getting results / errors of the
> > > > executed
> > > > > > > > >> commands.
> > > > > > > > >> >>>> Usually
> > > > > > > > >> >>>> > >> we
> > > > > > > > >> >>>> > >> > > > supply
> > > > > > > > >> >>>> > >> > > > > > >> with
> > > > > > > > >> >>>> > >> > > > > > >> > responses only errorCode, which
> looks
> > > > > very
> > > > > > > > >> limiting,
> > > > > > > > >> >>>> in case
> > > > > > > > >> >>>> > >> > of
> > > > > > > > >> >>>> > >> > > > CLI
> > > > > > > > >> >>>> > >> > > > > we
> > > > > > > > >> >>>> > >> > > > > > >> may
> > > > > > > > >> >>>> > >> > > > > > >> > want to print human readable
> error
> > > > > > > description.
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > So, taking into account previous
> item
> > > > > about
> > > > > > > > >> batching,
> > > > > > > > >> >>>> what
> > > > > > > > >> >>>> > >> do
> > > > > > > > >> >>>> > >> > > you
> > > > > > > > >> >>>> > >> > > > > > think
> > > > > > > > >> >>>> > >> > > > > > >> > about
> > > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support regexp)
> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName
> > > > > Partitions
> > > > > > > > >> Replicas
> > > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > > > > > > >> >>>> > >> > > > > > >> > [Config]
> > > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode
> > > > > > > > ErrorDescription
> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription => string
> (empty
> > > if
> > > > > > > > >> successful)
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest ->
> TopicNameRegexp
> > > > > > > Partitions
> > > > > > > > >> >>>> > >> > > ReplicaAssignment
> > > > > > > > >> >>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> > > > > > > > >> >>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName
> > > > > ErrorCode
> > > > > > > > >> >>>> ErrorDescription]
> > > > > > > > >> >>>> > >> > > > > > >> > CommandErrorCode
> > > > CommandErrorDescription
> > > > > > > > >> >>>> > >> > > > > > >> >   CommandErrorCode => int16
> > > > > > > > >> >>>> > >> > > > > > >> >   CommandErrorDescription =>
> string
> > > > > > (nonempty
> > > > > > > > in
> > > > > > > > >> case
> > > > > > > > >> >>>> of
> > > > > > > > >> >>>> > >> fatal
> > > > > > > > >> >>>> > >> > > > > error,
> > > > > > > > >> >>>> > >> > > > > > >> e.g.
> > > > > > > > >> >>>> > >> > > > > > >> > we couldn't get topics by regexp)
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > DescribeTopicRequest ->
> > > TopicNameRegexp
> > > > > > > > >> >>>> > >> > > > > > >> > DescribeTopicResponse ->
> [TopicName
> > > > > > > > >> TopicDescription
> > > > > > > > >> >>>> > >> ErrorCode
> > > > > > > > >> >>>> > >> > > > > > >> > ErrorDescription]
> CommandErrorCode
> > > > > > > > >> >>>> CommandErrorDescription
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > Also, any thoughts about our
> > > discussion
> > > > > > > > regarding
> > > > > > > > >> >>>> re-routing
> > > > > > > > >> >>>> > >> > > > > facility?
> > > > > > > > >> >>>> > >> > > > > > >> In
> > > > > > > > >> >>>> > >> > > > > > >> > my
> > > > > > > > >> >>>> > >> > > > > > >> > understanding, it is like between
> > > > > > augmenting
> > > > > > > > >> >>>> > >> > > TopicMetadataRequest
> > > > > > > > >> >>>> > >> > > > > > >> > (to include at least
> controllerId)
> > > and
> > > > > > > > >> implementing
> > > > > > > > >> >>>> new
> > > > > > > > >> >>>> > >> > generic
> > > > > > > > >> >>>> > >> > > > > > >> re-routing
> > > > > > > > >> >>>> > >> > > > > > >> > facility so sending messages to
> > > > > controller
> > > > > > > will
> > > > > > > > >> be
> > > > > > > > >> >>>> handled
> > > > > > > > >> >>>> > >> by
> > > > > > > > >> >>>> > >> > > it.
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > Thanks,
> > > > > > > > >> >>>> > >> > > > > > >> > Andrii Biletskyi
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM,
> > > Andrii
> > > > > > > > >> Biletskyi <
> > > > > > > > >> >>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly>
> wrote:
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > > @Guozhang:
> > > > > > > > >> >>>> > >> > > > > > >> > > Thanks for your comments, I've
> > > > answered
> > > > > > > some
> > > > > > > > of
> > > > > > > > >> >>>> those. The
> > > > > > > > >> >>>> > >> > > main
> > > > > > > > >> >>>> > >> > > > > > thing
> > > > > > > > >> >>>> > >> > > > > > >> is
> > > > > > > > >> >>>> > >> > > > > > >> > > having merged request for
> > > > > > > > >> >>>> create-alter-delete-describe - I
> > > > > > > > >> >>>> > >> > > have
> > > > > > > > >> >>>> > >> > > > > some
> > > > > > > > >> >>>> > >> > > > > > >> > > concerns about this approach.
> > > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > > @*Jay*:
> > > > > > > > >> >>>> > >> > > > > > >> > > I see that introduced
> > > > > > ClusterMetadaRequest
> > > > > > > is
> > > > > > > > >> also
> > > > > > > > >> >>>> one of
> > > > > > > > >> >>>> > >> > the
> > > > > > > > >> >>>> > >> > > > > > >> concerns.
> > > > > > > > >> >>>> > >> > > > > > >> > We
> > > > > > > > >> >>>> > >> > > > > > >> > > can solve it if we implement
> > > > re-routing
> > > > > > > > >> facility.
> > > > > > > > >> >>>> But I
> > > > > > > > >> >>>> > >> > agree
> > > > > > > > >> >>>> > >> > > > with
> > > > > > > > >> >>>> > >> > > > > > >> > > Guozhang - it will make
> clients'
> > > > > > internals
> > > > > > > a
> > > > > > > > >> little
> > > > > > > > >> >>>> bit
> > > > > > > > >> >>>> > >> > easier
> > > > > > > > >> >>>> > >> > > > but
> > > > > > > > >> >>>> > >> > > > > > >> this
> > > > > > > > >> >>>> > >> > > > > > >> > > seems to be a complex logic to
> > > > > implement
> > > > > > > and
> > > > > > > > >> >>>> support then.
> > > > > > > > >> >>>> > >> > > > > > Especially
> > > > > > > > >> >>>> > >> > > > > > >> for
> > > > > > > > >> >>>> > >> > > > > > >> > > Fetch and Produce (even if we
> add
> > > > > > > re-routing
> > > > > > > > >> later
> > > > > > > > >> >>>> for
> > > > > > > > >> >>>> > >> these
> > > > > > > > >> >>>> > >> > > > > > >> requests).
> > > > > > > > >> >>>> > >> > > > > > >> > > Also people will tend to avoid
> this
> > > > > > > > re-routing
> > > > > > > > >> >>>> facility
> > > > > > > > >> >>>> > >> and
> > > > > > > > >> >>>> > >> > > hold
> > > > > > > > >> >>>> > >> > > > > > local
> > > > > > > > >> >>>> > >> > > > > > >> > > cluster cache to ensure their
> > > > > > high-priority
> > > > > > > > >> requests
> > > > > > > > >> >>>> > >> (which
> > > > > > > > >> >>>> > >> > > some
> > > > > > > > >> >>>> > >> > > > > of
> > > > > > > > >> >>>> > >> > > > > > >> the
> > > > > > > > >> >>>> > >> > > > > > >> > > admin requests are) not sent to
> > > some
> > > > > busy
> > > > > > > > >> broker
> > > > > > > > >> >>>> where
> > > > > > > > >> >>>> > >> they
> > > > > > > > >> >>>> > >> > > wait
> > > > > > > > >> >>>> > >> > > > > to
> > > > > > > > >> >>>> > >> > > > > > be
> > > > > > > > >> >>>> > >> > > > > > >> > > routed to the correct one.
> > > > > > > > >> >>>> > >> > > > > > >> > > As pointed out by Jun here (
> > > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >>
> > > > > > > > >> >>>>
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > > > > > > >> >>>> > >> > > > > > >> > )
> > > > > > > > >> >>>> > >> > > > > > >> > > to solve the issue we might
> > > > introduce a
> > > > > > > > message
> > > > > > > > >> >>>> type to
> > > > > > > > >> >>>> > >> get
> > > > > > > > >> >>>> > >> > > > > cluster
> > > > > > > > >> >>>> > >> > > > > > >> > state.
> > > > > > > > >> >>>> > >> > > > > > >> > > But I agree we can just update
> > > > > > > > >> >>>> TopicMetadataResponse to
> > > > > > > > >> >>>> > >> > > include
> > > > > > > > >> >>>> > >> > > > > > >> > > controllerId (and probably smth
> > > > else).
> > > > > > > > >> >>>> > >> > > > > > >> > > What are you thougths?
> > > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > > Thanks,
> > > > > > > > >> >>>> > >> > > > > > >> > > Andrii
> > > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31
> AM,
> > > > > Guozhang
> > > > > > > > Wang
> > > > > > > > >> <
> > > > > > > > >> >>>> > >> > > > > wangguoz@gmail.com>
> > > > > > > > >> >>>> > >> > > > > > >> > wrote:
> > > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> I think for the topics
> commands we
> > > > can
> > > > > > > > >> actually
> > > > > > > > >> >>>> merge
> > > > > > > > >> >>>> > >> > > > > > >> > >> create/alter/delete/describe
> as
> > > one
> > > > > > > request
> > > > > > > > >> type
> > > > > > > > >> >>>> since
> > > > > > > > >> >>>> > >> > their
> > > > > > > > >> >>>> > >> > > > > > formats
> > > > > > > > >> >>>> > >> > > > > > >> are
> > > > > > > > >> >>>> > >> > > > > > >> > >> very much similar, and keep
> > > > > list-topics
> > > > > > > and
> > > > > > > > >> others
> > > > > > > > >> >>>> like
> > > > > > > > >> >>>> > >> > > > > > >> > >> partition-reassignment /
> > > > > > > > >> preferred-leader-election
> > > > > > > > >> >>>> as
> > > > > > > > >> >>>> > >> > > separate
> > > > > > > > >> >>>> > >> > > > > > >> request
> > > > > > > > >> >>>> > >> > > > > > >> > >> types, I also left some other
> > > > comments
> > > > > > on
> > > > > > > > the
> > > > > > > > >> RB (
> > > > > > > > >> >>>> > >> > > > > > >> > >>
> > > https://reviews.apache.org/r/29301/
> > > > ).
> > > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > > >> >>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04
> PM,
> > > Jay
> > > > > > > Kreps <
> > > > > > > > >> >>>> > >> > > > jay.kreps@gmail.com>
> > > > > > > > >> >>>> > >> > > > > > >> wrote:
> > > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > > >> >>>> > >> > > > > > >> > >> > Yeah I totally agree that we
> > > don't
> > > > > > want
> > > > > > > to
> > > > > > > > >> just
> > > > > > > > >> >>>> have
> > > > > > > > >> >>>> > >> one
> > > > > > > > >> >>>> > >> > > "do
> > > > > > > > >> >>>> > >> > > > > > admin
> > > > > > > > >> >>>> > >> > > > > > >> > >> stuff"
> > > > > > > > >> >>>> > >> > > > > > >> > >> > command that has the union
> of
> > > all
> > > > > > > > >> parameters.
> > > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > What I am saying is that
> command
> > > > > line
> > > > > > > > tools
> > > > > > > > >> are
> > > > > > > > >> >>>> one
> > > > > > > > >> >>>> > >> > client
> > > > > > > > >> >>>> > >> > > of
> > > > > > > > >> >>>> > >> > > > > the
> > > > > > > > >> >>>> > >> > > > > > >> > >> > administrative apis, but
> these
> > > > will
> > > > > be
> > > > > > > > used
> > > > > > > > >> in a
> > > > > > > > >> >>>> number
> > > > > > > > >> >>>> > >> > of
> > > > > > > > >> >>>> > >> > > > > > >> scenarios
> > > > > > > > >> >>>> > >> > > > > > >> > so
> > > > > > > > >> >>>> > >> > > > > > >> > >> > they should make logical
> sense
> > > > even
> > > > > in
> > > > > > > the
> > > > > > > > >> >>>> absence of
> > > > > > > > >> >>>> > >> the
> > > > > > > > >> >>>> > >> > > > > command
> > > > > > > > >> >>>> > >> > > > > > >> line
> > > > > > > > >> >>>> > >> > > > > > >> > >> > tool. Hence comments like
> trying
> > > > to
> > > > > > > > clarify
> > > > > > > > >> the
> > > > > > > > >> >>>> > >> > > relationship
> > > > > > > > >> >>>> > >> > > > > > >> between
> > > > > > > > >> >>>> > >> > > > > > >> > >> > ClusterMetadata and
> > > > > > > TopicMetadata...these
> > > > > > > > >> kinds
> > > > > > > > >> >>>> of
> > > > > > > > >> >>>> > >> things
> > > > > > > > >> >>>> > >> > > > > really
> > > > > > > > >> >>>> > >> > > > > > >> need
> > > > > > > > >> >>>> > >> > > > > > >> > >> to be
> > > > > > > > >> >>>> > >> > > > > > >> > >> > thought through.
> > > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > Hope that makes sense.
> > > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > -Jay
> > > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at
> 1:41 PM,
> > > > > > Andrii
> > > > > > > > >> >>>> Biletskyi <
> > > > > > > > >> >>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly
> >
> > > > wrote:
> > > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > Jay,
> > > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > Thanks for answering. You
> > > > > understood
> > > > > > > > >> >>>> correctly, most
> > > > > > > > >> >>>> > >> of
> > > > > > > > >> >>>> > >> > > my
> > > > > > > > >> >>>> > >> > > > > > >> comments
> > > > > > > > >> >>>> > >> > > > > > >> > >> were
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > related to your point 1) -
> > > about
> > > > > > "well
> > > > > > > > >> >>>> thought-out"
> > > > > > > > >> >>>> > >> > apis.
> > > > > > > > >> >>>> > >> > > > > Also,
> > > > > > > > >> >>>> > >> > > > > > >> yes,
> > > > > > > > >> >>>> > >> > > > > > >> > >> as I
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > understood we would like
> to
> > > > > > introduce
> > > > > > > a
> > > > > > > > >> single
> > > > > > > > >> >>>> > >> unified
> > > > > > > > >> >>>> > >> > > CLI
> > > > > > > > >> >>>> > >> > > > > tool
> > > > > > > > >> >>>> > >> > > > > > >> with
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > centralized server-side
> > > request
> > > > > > > handling
> > > > > > > > >> for
> > > > > > > > >> >>>> lots of
> > > > > > > > >> >>>> > >> > > > existing
> > > > > > > > >> >>>> > >> > > > > > >> ones
> > > > > > > > >> >>>> > >> > > > > > >> > >> (incl.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > TopicCommand,
> > > > CommitOffsetChecker,
> > > > > > > > >> >>>> > >> ReassignPartitions,
> > > > > > > > >> >>>> > >> > > smth
> > > > > > > > >> >>>> > >> > > > > > else
> > > > > > > > >> >>>> > >> > > > > > >> if
> > > > > > > > >> >>>> > >> > > > > > >> > >> added
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > in future). In our
> previous
> > > > > > > discussion (
> > > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694)
> > > > > > > > >> >>>> > >> > people
> > > > > > > > >> >>>> > >> > > > > said
> > > > > > > > >> >>>> > >> > > > > > >> > they'd
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > rather
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > have a separate message
> for
> > > each
> > > > > > > > command,
> > > > > > > > >> so,
> > > > > > > > >> >>>> yes,
> > > > > > > > >> >>>> > >> this
> > > > > > > > >> >>>> > >> > > > way I
> > > > > > > > >> >>>> > >> > > > > > >> came
> > > > > > > > >> >>>> > >> > > > > > >> > to
> > > > > > > > >> >>>> > >> > > > > > >> > >> 1-1
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > mapping between commands
> in
> > > the
> > > > > tool
> > > > > > > and
> > > > > > > > >> >>>> protocol
> > > > > > > > >> >>>> > >> > > > additions.
> > > > > > > > >> >>>> > >> > > > > > But
> > > > > > > > >> >>>> > >> > > > > > >> I
> > > > > > > > >> >>>> > >> > > > > > >> > >> might
> > > > > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > wrong.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > At the end I just try to
> start
> > > > > > > > discussion
> > > > > > > > >> how
> > > > > > > > >> >>>> at
> > > > > > > > >> >>>> > >> least
> > > > > > > > >> >>>> > >> > > > > > generally
> > > > > > > > >> >>>> > >> > > > > > >> > this
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > protocol should look like.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > Thanks,
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > Andrii
> > > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at
> 11:10
> > > > PM,
> > > > > > Jay
> > > > > > > > >> Kreps <
> > > > > > > > >> >>>> > >> > > > > > jay.kreps@gmail.com
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > >> wrote:
> > > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > Hey Andrii,
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > To answer your earlier
> > > > question
> > > > > we
> > > > > > > > just
> > > > > > > > >> >>>> really
> > > > > > > > >> >>>> > >> can't
> > > > > > > > >> >>>> > >> > be
> > > > > > > > >> >>>> > >> > > > > > adding
> > > > > > > > >> >>>> > >> > > > > > >> any
> > > > > > > > >> >>>> > >> > > > > > >> > >> more
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > scala protocol objects.
> > > These
> > > > > > things
> > > > > > > > are
> > > > > > > > >> >>>> super hard
> > > > > > > > >> >>>> > >> > to
> > > > > > > > >> >>>> > >> > > > > > maintain
> > > > > > > > >> >>>> > >> > > > > > >> > >> because
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > they hand code the byte
> > > > parsing
> > > > > > and
> > > > > > > > >> don't
> > > > > > > > >> >>>> have good
> > > > > > > > >> >>>> > >> > > > > > versioning
> > > > > > > > >> >>>> > >> > > > > > >> > >> support.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > Since we are already
> > > planning
> > > > on
> > > > > > > > >> converting
> > > > > > > > >> >>>> we
> > > > > > > > >> >>>> > >> > > definitely
> > > > > > > > >> >>>> > >> > > > > > don't
> > > > > > > > >> >>>> > >> > > > > > >> > >> want to
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > add
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > a ton more of
> these--they
> > > are
> > > > > > total
> > > > > > > > tech
> > > > > > > > >> >>>> debt.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > What does it mean that
> the
> > > > > changes
> > > > > > > are
> > > > > > > > >> >>>> isolated
> > > > > > > > >> >>>> > >> from
> > > > > > > > >> >>>> > >> > > the
> > > > > > > > >> >>>> > >> > > > > > >> current
> > > > > > > > >> >>>> > >> > > > > > >> > >> code
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > base?
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > I actually didn't
> understand
> > > > the
> > > > > > > > >> remaining
> > > > > > > > >> >>>> > >> comments,
> > > > > > > > >> >>>> > >> > > > which
> > > > > > > > >> >>>> > >> > > > > of
> > > > > > > > >> >>>> > >> > > > > > >> the
> > > > > > > > >> >>>> > >> > > > > > >> > >> > points
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > are you responding to?
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > Maybe one sticking point
> > > here
> > > > is
> > > > > > > that
> > > > > > > > it
> > > > > > > > >> >>>> seems like
> > > > > > > > >> >>>> > >> > you
> > > > > > > > >> >>>> > >> > > > > want
> > > > > > > > >> >>>> > >> > > > > > to
> > > > > > > > >> >>>> > >> > > > > > >> > make
> > > > > > > > >> >>>> > >> > > > > > >> > >> > some
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > kind of tool, and you
> have
> > > > made
> > > > > a
> > > > > > > 1-1
> > > > > > > > >> mapping
> > > > > > > > >> >>>> > >> between
> > > > > > > > >> >>>> > >> > > > > > commands
> > > > > > > > >> >>>> > >> > > > > > >> you
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > imagine
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > in the tool and protocol
> > > > > > additions.
> > > > > > > I
> > > > > > > > >> want
> > > > > > > > >> >>>> to make
> > > > > > > > >> >>>> > >> > sure
> > > > > > > > >> >>>> > >> > > > we
> > > > > > > > >> >>>> > >> > > > > > >> don't
> > > > > > > > >> >>>> > >> > > > > > >> > do
> > > > > > > > >> >>>> > >> > > > > > >> > >> > that.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > The protocol needs to be
> > > > really
> > > > > > > really
> > > > > > > > >> well
> > > > > > > > >> >>>> thought
> > > > > > > > >> >>>> > >> > out
> > > > > > > > >> >>>> > >> > > > > > against
> > > > > > > > >> >>>> > >> > > > > > >> > many
> > > > > > > > >> >>>> > >> > > > > > >> > >> > use
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > cases so it should make
> > > > perfect
> > > > > > > > logical
> > > > > > > > >> >>>> sense in
> > > > > > > > >> >>>> > >> the
> > > > > > > > >> >>>> > >> > > > > absence
> > > > > > > > >> >>>> > >> > > > > > of
> > > > > > > > >> >>>> > >> > > > > > >> > >> knowing
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > the
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > command line tool,
> right?
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > -Jay
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at
> > > 11:57
> > > > > AM,
> > > > > > > > Andrii
> > > > > > > > >> >>>> Biletskyi
> > > > > > > > >> >>>> > >> <
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> andrii.biletskyi@stealth.ly
> > > >
> > > > > > wrote:
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Hey Jay,
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > I would like to
> continue
> > > > this
> > > > > > > > >> discussion
> > > > > > > > >> >>>> as it
> > > > > > > > >> >>>> > >> seem
> > > > > > > > >> >>>> > >> > > > there
> > > > > > > > >> >>>> > >> > > > > > is
> > > > > > > > >> >>>> > >> > > > > > >> no
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > progress
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > here.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > First of all, could
> you
> > > > please
> > > > > > > > explain
> > > > > > > > >> >>>> what did
> > > > > > > > >> >>>> > >> you
> > > > > > > > >> >>>> > >> > > > mean
> > > > > > > > >> >>>> > >> > > > > in
> > > > > > > > >> >>>> > >> > > > > > >> 2?
> > > > > > > > >> >>>> > >> > > > > > >> > How
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > exactly
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > are we going to
> migrate to
> > > > the
> > > > > > new
> > > > > > > > >> java
> > > > > > > > >> >>>> protocol
> > > > > > > > >> >>>> > >> > > > > > definitions.
> > > > > > > > >> >>>> > >> > > > > > >> > And
> > > > > > > > >> >>>> > >> > > > > > >> > >> why
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > it's
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > a blocker for
> centralized
> > > > CLI?
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > I agree with you, this
> > > > feature
> > > > > > > > >> includes
> > > > > > > > >> >>>> lots of
> > > > > > > > >> >>>> > >> > > stuff,
> > > > > > > > >> >>>> > >> > > > > but
> > > > > > > > >> >>>> > >> > > > > > >> > >> thankfully
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > almost all changes are
> > > > > isolated
> > > > > > > from
> > > > > > > > >> the
> > > > > > > > >> >>>> current
> > > > > > > > >> >>>> > >> > code
> > > > > > > > >> >>>> > >> > > > > base,
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > so the main thing, I
> > > think,
> > > > we
> > > > > > > need
> > > > > > > > to
> > > > > > > > >> >>>> agree is
> > > > > > > > >> >>>> > >> > RQ/RP
> > > > > > > > >> >>>> > >> > > > > > format.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > So how can we start
> > > > discussion
> > > > > > > about
> > > > > > > > >> the
> > > > > > > > >> >>>> concrete
> > > > > > > > >> >>>> > >> > > > > messages
> > > > > > > > >> >>>> > >> > > > > > >> > format?
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Can we take (
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >>
> > > > > > > > >> >>>>
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > as starting point?
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > We had some doubts
> earlier
> > > > > > whether
> > > > > > > > it
> > > > > > > > >> worth
> > > > > > > > >> >>>> > >> > > introducing
> > > > > > > > >> >>>> > >> > > > > one
> > > > > > > > >> >>>> > >> > > > > > >> > >> generic
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > Admin
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Request for all
> commands (
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > but then everybody
> agreed
> > > it
> > > > > > would
> > > > > > > > be
> > > > > > > > >> >>>> better to
> > > > > > > > >> >>>> > >> > have
> > > > > > > > >> >>>> > >> > > > > > separate
> > > > > > > > >> >>>> > >> > > > > > >> > >> message
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > for
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > each admin command.
> The
> > > > > Request
> > > > > > > part
> > > > > > > > >> is
> > > > > > > > >> >>>> really
> > > > > > > > >> >>>> > >> > > dictated
> > > > > > > > >> >>>> > >> > > > > > from
> > > > > > > > >> >>>> > >> > > > > > >> the
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > command
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand)
> > > > arguments
> > > > > > > > itself,
> > > > > > > > >> so
> > > > > > > > >> >>>> the
> > > > > > > > >> >>>> > >> > proposed
> > > > > > > > >> >>>> > >> > > > > > version
> > > > > > > > >> >>>> > >> > > > > > >> > >> should
> > > > > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > fine (let's put aside
> for
> > > > now
> > > > > > > > remarks
> > > > > > > > >> about
> > > > > > > > >> >>>> > >> > Optional
> > > > > > > > >> >>>> > >> > > > > type,
> > > > > > > > >> >>>> > >> > > > > > >> > >> batching,
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > configs normalization
> - I
> > > > > agree
> > > > > > > with
> > > > > > > > >> all of
> > > > > > > > >> >>>> > >> them).
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > So the second part is
> > > > > Response.
> > > > > > I
> > > > > > > > see
> > > > > > > > >> >>>> there are
> > > > > > > > >> >>>> > >> two
> > > > > > > > >> >>>> > >> > > > cases
> > > > > > > > >> >>>> > >> > > > > > >> here.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> > > > > > > > >> Create/Alter/... ;
> > > > > > > > >> >>>> b)
> > > > > > > > >> >>>> > >> "Get"
> > > > > > > > >> >>>> > >> > > > > > requests -
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > List/Describe...
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > a) should only hold
> > > request
> > > > > > result
> > > > > > > > >> >>>> (regardless
> > > > > > > > >> >>>> > >> what
> > > > > > > > >> >>>> > >> > > we
> > > > > > > > >> >>>> > >> > > > > > decide
> > > > > > > > >> >>>> > >> > > > > > >> > >> about
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > blocking/non-blocking
> > > > commands
> > > > > > > > >> execution).
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Usually we provide
> error
> > > > code
> > > > > in
> > > > > > > > >> response
> > > > > > > > >> >>>> but
> > > > > > > > >> >>>> > >> since
> > > > > > > > >> >>>> > >> > > we
> > > > > > > > >> >>>> > >> > > > > will
> > > > > > > > >> >>>> > >> > > > > > >> use
> > > > > > > > >> >>>> > >> > > > > > >> > >> this
> > > > > > > > >> >>>> > >> > > > > > >> > >> > in
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > interactive shell we
> need
> > > > some
> > > > > > > human
> > > > > > > > >> >>>> readable
> > > > > > > > >> >>>> > >> error
> > > > > > > > >> >>>> > >> > > > > > >> description
> > > > > > > > >> >>>> > >> > > > > > >> > -
> > > > > > > > >> >>>> > >> > > > > > >> > >> so
> > > > > > > > >> >>>> > >> > > > > > >> > >> > I
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > added errorDesription
> > > field
> > > > > > where
> > > > > > > > you
> > > > > > > > >> can
> > > > > > > > >> >>>> at
> > > > > > > > >> >>>> > >> least
> > > > > > > > >> >>>> > >> > > > leave
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > exception.getMessage.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > b) in addition to
> previous
> > > > > item
> > > > > > > > >> message
> > > > > > > > >> >>>> should
> > > > > > > > >> >>>> > >> hold
> > > > > > > > >> >>>> > >> > > > > command
> > > > > > > > >> >>>> > >> > > > > > >> > >> specific
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > response data. We can
> > > > discuss
> > > > > in
> > > > > > > > >> detail
> > > > > > > > >> >>>> each of
> > > > > > > > >> >>>> > >> > them
> > > > > > > > >> >>>> > >> > > > but
> > > > > > > > >> >>>> > >> > > > > > >> let's
> > > > > > > > >> >>>> > >> > > > > > >> > for
> > > > > > > > >> >>>> > >> > > > > > >> > >> > now
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > agree about the
> overall
> > > > > pattern.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Thanks,
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015
> at
> > > 6:59
> > > > > AM,
> > > > > > > Jay
> > > > > > > > >> Kreps
> > > > > > > > >> >>>> <
> > > > > > > > >> >>>> > >> > > > > > >> jay.kreps@gmail.com
> > > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > wrote:
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > Hey Joe,
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > This is great. A few
> > > > > comments
> > > > > > on
> > > > > > > > >> KIP-4
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 1. This is much
> needed
> > > > > > > > >> functionality,
> > > > > > > > >> >>>> but there
> > > > > > > > >> >>>> > >> > > are a
> > > > > > > > >> >>>> > >> > > > > lot
> > > > > > > > >> >>>> > >> > > > > > >> of
> > > > > > > > >> >>>> > >> > > > > > >> > >> the so
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > let's
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > really think these
> > > > protocols
> > > > > > > > >> through. We
> > > > > > > > >> >>>> really
> > > > > > > > >> >>>> > >> > > want
> > > > > > > > >> >>>> > >> > > > to
> > > > > > > > >> >>>> > >> > > > > > >> end up
> > > > > > > > >> >>>> > >> > > > > > >> > >> > with a
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > set
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > of well thought-out,
> > > > > > orthoganol
> > > > > > > > >> apis.
> > > > > > > > >> >>>> For this
> > > > > > > > >> >>>> > >> > > > reason I
> > > > > > > > >> >>>> > >> > > > > > >> think
> > > > > > > > >> >>>> > >> > > > > > >> > >> it is
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > really
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > important to think
> > > through
> > > > > the
> > > > > > > end
> > > > > > > > >> state
> > > > > > > > >> >>>> even
> > > > > > > > >> >>>> > >> if
> > > > > > > > >> >>>> > >> > > that
> > > > > > > > >> >>>> > >> > > > > > >> includes
> > > > > > > > >> >>>> > >> > > > > > >> > >> APIs
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > we
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > won't implement in
> the
> > > > first
> > > > > > > > phase.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 2. Let's please
> please
> > > > > please
> > > > > > > wait
> > > > > > > > >> until
> > > > > > > > >> >>>> we
> > > > > > > > >> >>>> > >> have
> > > > > > > > >> >>>> > >> > > > > switched
> > > > > > > > >> >>>> > >> > > > > > >> the
> > > > > > > > >> >>>> > >> > > > > > >> > >> > server
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > over
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > to the new java
> protocol
> > > > > > > > >> definitions. If
> > > > > > > > >> >>>> we add
> > > > > > > > >> >>>> > >> > > > upteen
> > > > > > > > >> >>>> > >> > > > > > >> more ad
> > > > > > > > >> >>>> > >> > > > > > >> > >> hoc
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > scala
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > objects that is just
> > > > > > generating
> > > > > > > > more
> > > > > > > > >> >>>> work for
> > > > > > > > >> >>>> > >> the
> > > > > > > > >> >>>> > >> > > > > > >> conversion
> > > > > > > > >> >>>> > >> > > > > > >> > we
> > > > > > > > >> >>>> > >> > > > > > >> > >> > know
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > we
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > have to do.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 3. This proposal
> > > > introduces
> > > > > a
> > > > > > > new
> > > > > > > > >> type of
> > > > > > > > >> >>>> > >> > optional
> > > > > > > > >> >>>> > >> > > > > > >> parameter.
> > > > > > > > >> >>>> > >> > > > > > >> > >> This
> > > > > > > > >> >>>> > >> > > > > > >> > >> > is
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > inconsistent with
> > > > everything
> > > > > > > else
> > > > > > > > >> in the
> > > > > > > > >> >>>> > >> protocol
> > > > > > > > >> >>>> > >> > > > where
> > > > > > > > >> >>>> > >> > > > > > we
> > > > > > > > >> >>>> > >> > > > > > >> use
> > > > > > > > >> >>>> > >> > > > > > >> > >> -1
> > > > > > > > >> >>>> > >> > > > > > >> > >> > or
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > some
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > other marker value.
> You
> > > > > could
> > > > > > > > argue
> > > > > > > > >> >>>> either way
> > > > > > > > >> >>>> > >> > but
> > > > > > > > >> >>>> > >> > > > > let's
> > > > > > > > >> >>>> > >> > > > > > >> stick
> > > > > > > > >> >>>> > >> > > > > > >> > >> with
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > that
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > for consistency. For
> > > > clients
> > > > > > > that
> > > > > > > > >> >>>> implemented
> > > > > > > > >> >>>> > >> the
> > > > > > > > >> >>>> > >> > > > > > protocol
> > > > > > > > >> >>>> > >> > > > > > >> in
> > > > > > > > >> >>>> > >> > > > > > >> > a
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > better
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > way
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > than our scala code
> > > these
> > > > > > basic
> > > > > > > > >> >>>> primitives are
> > > > > > > > >> >>>> > >> > hard
> > > > > > > > >> >>>> > >> > > > to
> > > > > > > > >> >>>> > >> > > > > > >> change.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata:
> This
> > > > > seems
> > > > > > > to
> > > > > > > > >> >>>> duplicate
> > > > > > > > >> >>>> > >> > > > > > >> > TopicMetadataRequest
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > which
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > brokers, topics, and
> > > > > > > partitions. I
> > > > > > > > >> think
> > > > > > > > >> >>>> we
> > > > > > > > >> >>>> > >> > should
> > > > > > > > >> >>>> > >> > > > > rename
> > > > > > > > >> >>>> > >> > > > > > >> that
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > request
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> ClusterMetadataRequest
> > > (or
> > > > > > just
> > > > > > > > >> >>>> > >> MetadataRequest)
> > > > > > > > >> >>>> > >> > > and
> > > > > > > > >> >>>> > >> > > > > > >> include
> > > > > > > > >> >>>> > >> > > > > > >> > >> the id
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > of
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > the
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > controller. Or are
> there
> > > > > other
> > > > > > > > >> things we
> > > > > > > > >> >>>> could
> > > > > > > > >> >>>> > >> > add
> > > > > > > > >> >>>> > >> > > > > here?
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 5. We have a
> tendency to
> > > > try
> > > > > > to
> > > > > > > > >> make a
> > > > > > > > >> >>>> lot of
> > > > > > > > >> >>>> > >> > > > requests
> > > > > > > > >> >>>> > >> > > > > > that
> > > > > > > > >> >>>> > >> > > > > > >> > can
> > > > > > > > >> >>>> > >> > > > > > >> > >> > only
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > go
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > to
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > particular nodes.
> This
> > > > adds
> > > > > a
> > > > > > > lot
> > > > > > > > of
> > > > > > > > >> >>>> burden for
> > > > > > > > >> >>>> > >> > > > client
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > implementations
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > (it
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > sounds easy but each
> > > > > discovery
> > > > > > > can
> > > > > > > > >> fail
> > > > > > > > >> >>>> in many
> > > > > > > > >> >>>> > >> > > parts
> > > > > > > > >> >>>> > >> > > > > so
> > > > > > > > >> >>>> > >> > > > > > it
> > > > > > > > >> >>>> > >> > > > > > >> > >> ends up
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > being a
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > full state machine
> to do
> > > > > > > right). I
> > > > > > > > >> think
> > > > > > > > >> >>>> we
> > > > > > > > >> >>>> > >> > should
> > > > > > > > >> >>>> > >> > > > > > consider
> > > > > > > > >> >>>> > >> > > > > > >> > >> making
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > admin
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > commands and
> ideally as
> > > > many
> > > > > > of
> > > > > > > > the
> > > > > > > > >> >>>> other apis
> > > > > > > > >> >>>> > >> as
> > > > > > > > >> >>>> > >> > > > > > possible
> > > > > > > > >> >>>> > >> > > > > > >> > >> > available
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > on
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > all
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > brokers and just
> > > redirect
> > > > to
> > > > > > the
> > > > > > > > >> >>>> controller on
> > > > > > > > >> >>>> > >> > the
> > > > > > > > >> >>>> > >> > > > > broker
> > > > > > > > >> >>>> > >> > > > > > >> > side.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > Perhaps
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > there would be a
> general
> > > > way
> > > > > > to
> > > > > > > > >> >>>> encapsulate
> > > > > > > > >> >>>> > >> this
> > > > > > > > >> >>>> > >> > > > > > re-routing
> > > > > > > > >> >>>> > >> > > > > > >> > >> > behavior.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 6. We should
> probably
> > > > > > normalize
> > > > > > > > the
> > > > > > > > >> key
> > > > > > > > >> >>>> value
> > > > > > > > >> >>>> > >> > pairs
> > > > > > > > >> >>>> > >> > > > > used
> > > > > > > > >> >>>> > >> > > > > > >> for
> > > > > > > > >> >>>> > >> > > > > > >> > >> > configs
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > rather
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > than embedding a new
> > > > > > formatting.
> > > > > > > > So
> > > > > > > > >> two
> > > > > > > > >> >>>> strings
> > > > > > > > >> >>>> > >> > > > rather
> > > > > > > > >> >>>> > >> > > > > > than
> > > > > > > > >> >>>> > >> > > > > > >> > one
> > > > > > > > >> >>>> > >> > > > > > >> > >> > with
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > an
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > internal equals
> sign.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 7. Is the
> postcondition
> > > of
> > > > > > these
> > > > > > > > >> APIs
> > > > > > > > >> >>>> that the
> > > > > > > > >> >>>> > >> > > > command
> > > > > > > > >> >>>> > >> > > > > > has
> > > > > > > > >> >>>> > >> > > > > > >> > >> begun or
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > that
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > the command has been
> > > > > > completed?
> > > > > > > It
> > > > > > > > >> is a
> > > > > > > > >> >>>> lot
> > > > > > > > >> >>>> > >> more
> > > > > > > > >> >>>> > >> > > > usable
> > > > > > > > >> >>>> > >> > > > > > if
> > > > > > > > >> >>>> > >> > > > > > >> the
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > command
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > been completed so
> you
> > > know
> > > > > > that
> > > > > > > if
> > > > > > > > >> you
> > > > > > > > >> >>>> create a
> > > > > > > > >> >>>> > >> > > topic
> > > > > > > > >> >>>> > >> > > > > and
> > > > > > > > >> >>>> > >> > > > > > >> then
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > publish
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > to
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > it you won't get an
> > > > > exception
> > > > > > > > about
> > > > > > > > >> >>>> there being
> > > > > > > > >> >>>> > >> > no
> > > > > > > > >> >>>> > >> > > > such
> > > > > > > > >> >>>> > >> > > > > > >> topic.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 8. Describe topic
> and
> > > list
> > > > > > > topics
> > > > > > > > >> >>>> duplicate a
> > > > > > > > >> >>>> > >> lot
> > > > > > > > >> >>>> > >> > > of
> > > > > > > > >> >>>> > >> > > > > > stuff
> > > > > > > > >> >>>> > >> > > > > > >> in
> > > > > > > > >> >>>> > >> > > > > > >> > >> the
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > metadata
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > request. Is there a
> > > reason
> > > > > to
> > > > > > > give
> > > > > > > > >> back
> > > > > > > > >> >>>> topics
> > > > > > > > >> >>>> > >> > > marked
> > > > > > > > >> >>>> > >> > > > > for
> > > > > > > > >> >>>> > >> > > > > > >> > >> > deletion? I
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > feel
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > like if we just
> make the
> > > > > > > > >> post-condition
> > > > > > > > >> >>>> of the
> > > > > > > > >> >>>> > >> > > delete
> > > > > > > > >> >>>> > >> > > > > > >> command
> > > > > > > > >> >>>> > >> > > > > > >> > be
> > > > > > > > >> >>>> > >> > > > > > >> > >> > that
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > the
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > topic is deleted
> that
> > > will
> > > > > get
> > > > > > > rid
> > > > > > > > >> of
> > > > > > > > >> >>>> the need
> > > > > > > > >> >>>> > >> > for
> > > > > > > > >> >>>> > >> > > > this
> > > > > > > > >> >>>> > >> > > > > > >> right?
> > > > > > > > >> >>>> > >> > > > > > >> > >> And
> > > > > > > > >> >>>> > >> > > > > > >> > >> > it
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > will
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > be much more
> intuitive.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 9. Should we
> consider
> > > > > batching
> > > > > > > > these
> > > > > > > > >> >>>> requests?
> > > > > > > > >> >>>> > >> We
> > > > > > > > >> >>>> > >> > > > have
> > > > > > > > >> >>>> > >> > > > > > >> > generally
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > tried
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > to
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > allow multiple
> > > operations
> > > > to
> > > > > > be
> > > > > > > > >> batched.
> > > > > > > > >> >>>> My
> > > > > > > > >> >>>> > >> > > suspicion
> > > > > > > > >> >>>> > >> > > > > is
> > > > > > > > >> >>>> > >> > > > > > >> that
> > > > > > > > >> >>>> > >> > > > > > >> > >> > without
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > this
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > we will get a lot of
> > > code
> > > > > that
> > > > > > > > does
> > > > > > > > >> >>>> something
> > > > > > > > >> >>>> > >> > like
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >    for(topic:
> > > > > > > > >> adminClient.listTopics())
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >>  adminClient.describeTopic(topic)
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > this code will work
> > > great
> > > > > when
> > > > > > > you
> > > > > > > > >> test
> > > > > > > > >> >>>> on 5
> > > > > > > > >> >>>> > >> > topics
> > > > > > > > >> >>>> > >> > > > but
> > > > > > > > >> >>>> > >> > > > > > >> not do
> > > > > > > > >> >>>> > >> > > > > > >> > >> as
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > well
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > if
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > you have 50k.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 10. I think we
> should
> > > also
> > > > > > > discuss
> > > > > > > > >> how
> > > > > > > > >> >>>> we want
> > > > > > > > >> >>>> > >> to
> > > > > > > > >> >>>> > >> > > > > expose
> > > > > > > > >> >>>> > >> > > > > > a
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > programmatic
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > JVM
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > client api for these
> > > > > > operations.
> > > > > > > > >> >>>> Currently
> > > > > > > > >> >>>> > >> people
> > > > > > > > >> >>>> > >> > > > rely
> > > > > > > > >> >>>> > >> > > > > on
> > > > > > > > >> >>>> > >> > > > > > >> > >> > AdminUtils
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > which
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > is totally sketchy.
> I
> > > > think
> > > > > we
> > > > > > > > >> probably
> > > > > > > > >> >>>> need
> > > > > > > > >> >>>> > >> > > another
> > > > > > > > >> >>>> > >> > > > > > client
> > > > > > > > >> >>>> > >> > > > > > >> > >> under
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > clients/
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > that exposes
> > > > administrative
> > > > > > > > >> >>>> functionality. We
> > > > > > > > >> >>>> > >> > will
> > > > > > > > >> >>>> > >> > > > need
> > > > > > > > >> >>>> > >> > > > > > >> this
> > > > > > > > >> >>>> > >> > > > > > >> > >> just
> > > > > > > > >> >>>> > >> > > > > > >> > >> > to
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > properly test the
> new
> > > > apis,
> > > > > I
> > > > > > > > >> suspect. We
> > > > > > > > >> >>>> > >> should
> > > > > > > > >> >>>> > >> > > > figure
> > > > > > > > >> >>>> > >> > > > > > out
> > > > > > > > >> >>>> > >> > > > > > >> > that
> > > > > > > > >> >>>> > >> > > > > > >> > >> > API.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 11. The other
> > > information
> > > > > that
> > > > > > > > >> would be
> > > > > > > > >> >>>> really
> > > > > > > > >> >>>> > >> > > useful
> > > > > > > > >> >>>> > >> > > > > to
> > > > > > > > >> >>>> > >> > > > > > >> get
> > > > > > > > >> >>>> > >> > > > > > >> > >> would
> > > > > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > information about
> > > > > > > partitions--how
> > > > > > > > >> much
> > > > > > > > >> >>>> data is
> > > > > > > > >> >>>> > >> in
> > > > > > > > >> >>>> > >> > > the
> > > > > > > > >> >>>> > >> > > > > > >> > partition,
> > > > > > > > >> >>>> > >> > > > > > >> > >> > what
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > are
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > the segment offsets,
> > > what
> > > > is
> > > > > > the
> > > > > > > > >> log-end
> > > > > > > > >> >>>> offset
> > > > > > > > >> >>>> > >> > > (i.e.
> > > > > > > > >> >>>> > >> > > > > > last
> > > > > > > > >> >>>> > >> > > > > > >> > >> offset),
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > what
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > is
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > the compaction
> point,
> > > > etc. I
> > > > > > > think
> > > > > > > > >> that
> > > > > > > > >> >>>> done
> > > > > > > > >> >>>> > >> > right
> > > > > > > > >> >>>> > >> > > > this
> > > > > > > > >> >>>> > >> > > > > > >> would
> > > > > > > > >> >>>> > >> > > > > > >> > be
> > > > > > > > >> >>>> > >> > > > > > >> > >> > the
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > successor to the
> very
> > > > > awkward
> > > > > > > > >> >>>> OffsetRequest we
> > > > > > > > >> >>>> > >> > have
> > > > > > > > >> >>>> > >> > > > > > today.
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > -Jay
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21,
> 2015 at
> > > > > 10:27
> > > > > > > PM,
> > > > > > > > >> Joe
> > > > > > > > >> >>>> Stein <
> > > > > > > > >> >>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > wrote:
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >>
> > > > > > > > >> >>>>
> > > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > > JIRA
> > > > > > > > >> >>>> > >> > > > >
> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > > >> >>>> /*******************************************
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Founder,
> Principal
> > > > > > Consultant
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Big Data Open
> Source
> > > > > > Security
> > > > > > > > LLC
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > http://www.stealth.ly
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Twitter:
> > > > > @allthingshadoop <
> > > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > http://www.twitter.com/allthingshadoop
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > > >> >>>> ********************************************/
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > > >> >>>> > >> > > > > > >> > >> --
> > > > > > > > >> >>>> > >> > > > > > >> > >> -- Guozhang
> > > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > > >
> > > > > > > > >> >>>> > >> > > > > >
> > > > > > > > >> >>>> > >> > > > >
> > > > > > > > >> >>>> > >> > > >
> > > > > > > > >> >>>> > >> > >
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >> > --
> > > > > > > > >> >>>> > >> > Jeff Holoman
> > > > > > > > >> >>>> > >> > Systems Engineer
> > > > > > > > >> >>>> > >> >
> > > > > > > > >> >>>> > >>
> > > > > > > > >> >>>>
> > > > > > > > >> >>>>
> > > > > > > > >> >>>
> > > > > > > > >> >>>
> > > > > > > > >> >>> --
> > > > > > > > >> >>> -- Guozhang
> > > > > > > > >> >>>
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >> >>
> > > > > > > > >> >> --
> > > > > > > > >> >> -- Guozhang
> > > > > > > > >>
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
>
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Joel Koshy <jj...@gmail.com>.
I may be missing some context but hopefully this will also be covered
today: I thought the earlier proposal where there was an explicit
ClusterMetadata request was clearer and explicit. During the course of
this thread I think the conclusion was that the main need was for
controller information and that can be rolled into the topic metadata
response but that seems a bit irrelevant to topic metadata. FWIW I
think the full broker-list is also irrelevant to topic metadata, but
it is already there and in use. I think there is still room for an
explicit ClusterMetadata request since there may be other
cluster-level information that we may want to add over time (and that
have nothing to do with topic metadata).

On Tue, Mar 17, 2015 at 02:45:30PM +0200, Andrii Biletskyi wrote:
> Jun,
> 
> 101. Okay, if you say that such use case is important. I also think
> using clientId for these purposes is fine - if we already have this field
> as part of all Wire protocol messages, why not use that.
> I will update KIP-4 page if nobody has other ideas (which may come up
> during the call today).
> 
> 102.1 Agree, I'll update the KIP accordingly. I think we can add new,
> fine-grained error codes if some error code received in specific case
> won't give enough context to return a descriptive error message for user.
> 
> Look forward to discussing all outstanding issues in detail today during
> the call.
> 
> Thanks,
> Andrii Biletskyi
> 
> 
> 
> On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io> wrote:
> 
> > 101. There may be a use case where you only want the topics to be created
> > manually by admins. Currently, you can do that by disabling auto topic
> > creation and issue topic creation from the TopicCommand. If we disable auto
> > topic creation completely on the broker and don't have a way to distinguish
> > between topic creation requests from the regular clients and the admin, we
> > can't support manual topic creation any more. I was thinking that another
> > way of distinguishing the clients making the topic creation requests is
> > using clientId. For example, the admin tool can set it to something like
> > admin and the broker can treat that clientId specially.
> >
> > Also, there is a related discussion in KAFKA-2020. Currently, we do the
> > following in TopicMetadataResponse:
> >
> > 1. If leader is not available, we set the partition level error code to
> > LeaderNotAvailable.
> > 2. If a non-leader replica is not available, we take that replica out of
> > the assigned replica list and isr in the response. As an indication for
> > doing that, we set the partition level error code to ReplicaNotAvailable.
> >
> > This has a few problems. First, ReplicaNotAvailable probably shouldn't be
> > an error, at least for the normal producer/consumer clients that just want
> > to find out the leader. Second, it can happen that both the leader and
> > another replica are not available at the same time. There is no error code
> > to indicate both. Third, even if a replica is not available, it's still
> > useful to return its replica id since some clients (e.g. admin tool) may
> > still make use of it.
> >
> > One way to address this issue is to always return the replica id for
> > leader, assigned replicas, and isr regardless of whether the corresponding
> > broker is live or not. Since we also return the list of live brokers, the
> > client can figure out whether a leader or a replica is live or not and act
> > accordingly. This way, we don't need to set the partition level error code
> > when the leader or a replica is not available. This doesn't change the wire
> > protocol, but does change the semantics. Since we are evolving the protocol
> > of TopicMetadataRequest here, we can potentially piggyback the change.
> >
> > 102.1 For those types of errors due to invalid input, shouldn't we just
> > guard it at parameter validation time and throw InvalidArgumentException
> > without even sending the request to the broker?
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Jun,
> > >
> > > Answering your questions:
> > >
> > > 101. If I understand you correctly, you are saying future producer
> > versions
> > > (which
> > > will be ported to TMR_V1) won't be able to automatically create topic (if
> > > we
> > > unconditionally remove topic creation from there). But we need to this
> > > preserve logic.
> > > Ok, about your proposal: I'm not a big fan too, when it comes to
> > > differentiating
> > > clients directly in protocol schema. And also I'm not sure I understand
> > at
> > > all why
> > > auto.create.topics.enable is a server side configuration. Can we
> > deprecate
> > > this setting
> > > in future versions, add this setting to producer and based on that upon
> > > receiving
> > > UnknownTopic create topic explicitly by a separate producer call via
> > > adminClient?
> > >
> > > 102.1. Hm, yes. It's because we want to support batching and at the same
> > > time we
> > > want to give descriptive error messages for clients. Since AdminClient
> > > holds the context
> > > to construct such messages (e.g. AdminClient layer can know that
> > > InvalidArgumentsCode
> > > means two cases: either invalid number - e.g. -1; or replication-factor
> > was
> > > provided while
> > > partitions argument wasn't) - I wrapped responses in Exceptions. But I'm
> > > open to any
> > > other ideas, this was just initial version.
> > > 102.2. Yes, I agree. I'll change that to probably some other dto.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Andrii,
> > > >
> > > > 101. That's what I was thinking too, but it may not be that simple. In
> > > > TopicMetadataRequest_V1,
> > > > we can let it not trigger auto topic creation. Then, in the producer
> > > side,
> > > > if it gets an UnknownTopicException, it can explicitly issue a
> > > > createTopicRequest for auto topic creation. On the consumer side, it
> > will
> > > > never issue createTopicRequest. This works when auto topic creation is
> > > > enabled on the broker side. However, I am not sure how things will work
> > > > when auto topic creation is disabled on the broker side. In this case,
> > we
> > > > want to have a way to manually create a topic, potentially through
> > admin
> > > > commands. However, then we need a way to distinguish createTopicRequest
> > > > issued from the producer clients and the admin tools. May be we can
> > add a
> > > > new field in createTopicRequest and set it differently in the producer
> > > > client and the admin client. However, I am not sure if that's the best
> > > > approach.
> > > >
> > > > 2. Yes, refactoring existing requests is a non-trivial amount of work.
> > I
> > > > posted some comments in KAFKA-1927. We will probably have to fix
> > > KAFKA-1927
> > > > first, before adding the new logic in KAFKA-1694. Otherwise, the
> > changes
> > > > will be too big.
> > > >
> > > > 102. About the AdminClient:
> > > > 102.1. It's a bit weird that we return exception in the api. It seems
> > > that
> > > > we should either return error code or throw an exception when getting
> > the
> > > > response state.
> > > > 102.2. We probably shouldn't explicitly use the request object in the
> > > api.
> > > > Not every request evolution requires an api change.
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Jun,
> > > > >
> > > > > Thanks for you comments. Answers inline:
> > > > >
> > > > > 100. There are a few fields such as ReplicaAssignment,
> > > > > > ReassignPartitionRequest,
> > > > > > and PartitionsSerialized that are represented as a string, but
> > > contain
> > > > > > composite structures in json. Could we flatten them out directly in
> > > the
> > > > > > protocol definition as arrays/records?
> > > > >
> > > > >
> > > > > Yes, now with Admin Client this looks a bit weird. My initial
> > > motivation
> > > > > was:
> > > > > ReassignPartitionCommand accepts input in json, we want to remain
> > > tools'
> > > > > interfaces unchanged, where possible.
> > > > > If we port it to deserialized format, in CLI (/tools project) we will
> > > > have
> > > > > to add some
> > > > > json library since /tools is written in java and we'll need to
> > > > deserialize
> > > > > json file
> > > > > provided by a user. Can we quickly agree on what this library should
> > be
> > > > > (Jackson, GSON, whatever)?
> > > > >
> > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic creation?
> > > This
> > > > > > will be a bit weird now that we have a separate topic creation api.
> > > > Have
> > > > > > you thought about how the new createTopicRequest and
> > > > TopicMetadataRequest
> > > > > > v1 will be used in the producer/consumer client, in addition to
> > admin
> > > > > > tools? For example, ideally, we don't want TopicMetadataRequest
> > from
> > > > the
> > > > > > consumer to trigger auto topic creation.
> > > > >
> > > > >
> > > > > I agree, this strange logic should be fixed. I'm not confident in
> > this
> > > > > Kafka part so
> > > > > correct me if I'm wrong, but it doesn't look like a hard thing to
> > do, I
> > > > > think we can
> > > > > leverage AdminClient for that in Producer and unconditionally remove
> > > > topic
> > > > > creation from the TopicMetadataRequest_V1.
> > > > >
> > > > > 2. I think Jay meant getting rid of scala classes
> > > > > > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We
> > did
> > > > > that
> > > > > > as a stop-gap thing when adding the new requests for the consumers.
> > > > > > However, the long term plan is to get rid of all those and just
> > reuse
> > > > the
> > > > > > java request/response in the client. Since this KIP proposes to
> > add a
> > > > > > significant number of new requests, perhaps we should bite the
> > bullet
> > > > to
> > > > > > clean up the existing scala requests first before adding new ones?
> > > > > >
> > > > >
> > > > > Yes, looks like I misunderstood the point of ...RequestAndHeader.
> > > Okay, I
> > > > > will
> > > > > rework that. The only thing is that I don't see any example how it
> > was
> > > > done
> > > > > for at
> > > > > least one existing protocol message. Thus, as I understand, I have to
> > > > think
> > > > > how we
> > > > > are going to do it.
> > > > > Re porting all existing RQ/RP in this patch. Sounds reasonable, but
> > if
> > > > it's
> > > > > an *obligatory*
> > > > > requirement to have Admin KIP done, I'm afraid this can be a serious
> > > > > blocker for us.
> > > > > There are 13 protocol messages and all that would require not only
> > unit
> > > > > tests but quite
> > > > > intensive manual testing, no? I'm afraid I'm not the right guy to
> > cover
> > > > > pretty much all
> > > > > Kafka core internals :). Let me know your thoughts on this item. Btw
> > > > there
> > > > > is a ticket to
> > > > > follow-up this issue (
> > https://issues.apache.org/jira/browse/KAFKA-2006
> > > ).
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > >
> > > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <ju...@confluent.io> wrote:
> > > > >
> > > > > > Andrii,
> > > > > >
> > > > > >
> > > > > > A few more comments.
> > > > > >
> > > > > > 100. There are a few fields such as ReplicaAssignment,
> > > > > > ReassignPartitionRequest,
> > > > > > and PartitionsSerialized that are represented as a string, but
> > > contain
> > > > > > composite structures in json. Could we flatten them out directly in
> > > the
> > > > > > protocol definition as arrays/records?
> > > > > >
> > > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> > creation?
> > > > This
> > > > > > will be a bit weird now that we have a separate topic creation api.
> > > > Have
> > > > > > you thought about how the new createTopicRequest and
> > > > TopicMetadataRequest
> > > > > > v1 will be used in the producer/consumer client, in addition to
> > admin
> > > > > > tools? For example, ideally, we don't want TopicMetadataRequest
> > from
> > > > the
> > > > > > consumer to trigger auto topic creation.
> > > > > >
> > > > > > 2. I think Jay meant getting rid of scala classes
> > > > > > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We
> > did
> > > > > that
> > > > > > as a stop-gap thing when adding the new requests for the consumers.
> > > > > > However, the long term plan is to get rid of all those and just
> > reuse
> > > > the
> > > > > > java request/response in the client. Since this KIP proposes to
> > add a
> > > > > > significant number of new requests, perhaps we should bite the
> > bullet
> > > > to
> > > > > > clean up the existing scala requests first before adding new ones?
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Jun
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Hi,
> > > > > > >
> > > > > > > As said above - I list again all comments from this thread so we
> > > > > > > can see what's left and finalize all pending issues.
> > > > > > >
> > > > > > > Comments from Jay:
> > > > > > > 1. This is much needed functionality, but there are a lot of the
> > so
> > > > > let's
> > > > > > > really think these protocols through. We really want to end up
> > > with a
> > > > > set
> > > > > > > of well thought-out, orthoganol apis. For this reason I think it
> > is
> > > > > > really
> > > > > > > important to think through the end state even if that includes
> > APIs
> > > > we
> > > > > > > won't implement in the first phase.
> > > > > > >
> > > > > > > A: Definitely behind this. Would appreciate if there are concrete
> > > > > > comments
> > > > > > > how this can be improved.
> > > > > > >
> > > > > > > 2. Let's please please please wait until we have switched the
> > > server
> > > > > over
> > > > > > > to the new java protocol definitions. If we add upteen more ad
> > hoc
> > > > > scala
> > > > > > > objects that is just generating more work for the conversion we
> > > know
> > > > we
> > > > > > > have to do.
> > > > > > >
> > > > > > > A: Fixed in the latest patch - removed scala protocol classes.
> > > > > > >
> > > > > > > 3. This proposal introduces a new type of optional parameter.
> > This
> > > is
> > > > > > > inconsistent with everything else in the protocol where we use -1
> > > or
> > > > > some
> > > > > > > other marker value. You could argue either way but let's stick
> > with
> > > > > that
> > > > > > > for consistency. For clients that implemented the protocol in a
> > > > better
> > > > > > way
> > > > > > > than our scala code these basic primitives are hard to change.
> > > > > > >
> > > > > > > A: Fixed in the latest patch - removed MaybeOf type and changed
> > > > > protocol
> > > > > > > accordingly.
> > > > > > >
> > > > > > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
> > > > which
> > > > > > has
> > > > > > > brokers, topics, and partitions. I think we should rename that
> > > > request
> > > > > > > ClusterMetadataRequest (or just MetadataRequest) and include the
> > id
> > > > of
> > > > > > the
> > > > > > > controller. Or are there other things we could add here?
> > > > > > >
> > > > > > > A: I agree. Updated the KIP. Let's extends TopicMetadata to
> > > version 2
> > > > > and
> > > > > > > include controller.
> > > > > > >
> > > > > > > 5. We have a tendency to try to make a lot of requests that can
> > > only
> > > > go
> > > > > > to
> > > > > > > particular nodes. This adds a lot of burden for client
> > > > implementations
> > > > > > (it
> > > > > > > sounds easy but each discovery can fail in many parts so it ends
> > up
> > > > > > being a
> > > > > > > full state machine to do right). I think we should consider
> > making
> > > > > admin
> > > > > > > commands and ideally as many of the other apis as possible
> > > available
> > > > on
> > > > > > all
> > > > > > > brokers and just redirect to the controller on the broker side.
> > > > Perhaps
> > > > > > > there would be a general way to encapsulate this re-routing
> > > behavior.
> > > > > > >
> > > > > > > A: It's a very interesting idea, but seems there are some
> > concerns
> > > > > about
> > > > > > > this
> > > > > > > feature (like performance considerations, how this will
> > complicate
> > > > > server
> > > > > > > etc).
> > > > > > > I believe this shouldn't be a blocker. If this feature is
> > > implemented
> > > > > at
> > > > > > > some
> > > > > > > point it won't affect Admin changes - at least no changes to
> > public
> > > > API
> > > > > > > will be required.
> > > > > > >
> > > > > > > 6. We should probably normalize the key value pairs used for
> > > configs
> > > > > > rather
> > > > > > > than embedding a new formatting. So two strings rather than one
> > > with
> > > > an
> > > > > > > internal equals sign.
> > > > > > >
> > > > > > > A: Fixed in the latest patch - normalized configs and changed
> > > > protocol
> > > > > > > accordingly.
> > > > > > >
> > > > > > > 7. Is the postcondition of these APIs that the command has begun
> > or
> > > > > that
> > > > > > > the command has been completed? It is a lot more usable if the
> > > > command
> > > > > > has
> > > > > > > been completed so you know that if you create a topic and then
> > > > publish
> > > > > to
> > > > > > > it you won't get an exception about there being no such topic.
> > > > > > >
> > > > > > > A: For long running requests (like reassign partitions) - the
> > post
> > > > > > > condition is
> > > > > > > command has begun - so we don't block the client. In case of your
> > > > > > example -
> > > > > > > topic commands, this will be refactored and topic commands will
> > be
> > > > > > executed
> > > > > > > immediately, since the Controller will serve Admin requests
> > > > > > > (follow-up ticket KAFKA-1777).
> > > > > > >
> > > > > > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > > > > > metadata
> > > > > > > request. Is there a reason to give back topics marked for
> > > deletion? I
> > > > > > feel
> > > > > > > like if we just make the post-condition of the delete command be
> > > that
> > > > > the
> > > > > > > topic is deleted that will get rid of the need for this right?
> > And
> > > it
> > > > > > will
> > > > > > > be much more intuitive.
> > > > > > >
> > > > > > > A: Fixed in the latest patch - removed topics marked for deletion
> > > in
> > > > > > > ListTopicsRequest.
> > > > > > >
> > > > > > > 9. Should we consider batching these requests? We have generally
> > > > tried
> > > > > to
> > > > > > > allow multiple operations to be batched. My suspicion is that
> > > without
> > > > > > this
> > > > > > > we will get a lot of code that does something like
> > > > > > >    for(topic: adminClient.listTopics())
> > > > > > >       adminClient.describeTopic(topic)
> > > > > > > this code will work great when you test on 5 topics but not do as
> > > > well
> > > > > if
> > > > > > > you have 50k.
> > > > > > >
> > > > > > > A: Updated the KIP - please check "Topic Admin Schema" section.
> > > > > > >
> > > > > > > 10. I think we should also discuss how we want to expose a
> > > > programmatic
> > > > > > JVM
> > > > > > > client api for these operations. Currently people rely on
> > > AdminUtils
> > > > > > which
> > > > > > > is totally sketchy. I think we probably need another client under
> > > > > > clients/
> > > > > > > that exposes administrative functionality. We will need this just
> > > to
> > > > > > > properly test the new apis, I suspect. We should figure out that
> > > API.
> > > > > > >
> > > > > > > A: Updated the KIP - please check "Admin Client" section with an
> > > > > initial
> > > > > > > API proposal.
> > > > > > >
> > > > > > > 11. The other information that would be really useful to get
> > would
> > > be
> > > > > > > information about partitions--how much data is in the partition,
> > > what
> > > > > are
> > > > > > > the segment offsets, what is the log-end offset (i.e. last
> > offset),
> > > > > what
> > > > > > is
> > > > > > > the compaction point, etc. I think that done right this would be
> > > the
> > > > > > > successor to the very awkward OffsetRequest we have today.
> > > > > > >
> > > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest patch. I
> > > > believe
> > > > > > > this should
> > > > > > > be resolved in a separate KIP / jira ticket.
> > > > > > >
> > > > > > > 12. Generally we can do good error handling without needing
> > custom
> > > > > > > server-side
> > > > > > > messages. I.e. generally the client has the context to know that
> > if
> > > > it
> > > > > > got
> > > > > > > an error that the topic doesn't exist to say "Topic X doesn't
> > > exist"
> > > > > > rather
> > > > > > > than "error code 14" (or whatever). Maybe there are specific
> > cases
> > > > > where
> > > > > > > this is hard? If we want to add server-side error messages we
> > > really
> > > > do
> > > > > > > need to do this in a consistent way across the protocol.
> > > > > > >
> > > > > > > A: Updated the KIP - please check "Protocol Errors" section. I
> > > added
> > > > > the
> > > > > > > comprehensive, fine-grained list of error codes.
> > > > > > >
> > > > > > > Comments from Guozhang:
> > > > > > > 13. Describe topic request: it would be great to go beyond just
> > > > > batching
> > > > > > on
> > > > > > > topic name regex for this request. For example, a very common use
> > > > case
> > > > > of
> > > > > > > the topic command is to list all topics whose config A's value is
> > > B.
> > > > > With
> > > > > > > topic name regex then we have to first retrieve __all__ topics's
> > > > > > > description info and then filter at the client end, which will
> > be a
> > > > > huge
> > > > > > > burden on ZK.
> > > > > > > AND
> > > > > > > 14. Config K-Vs in create topic: this is related to the previous
> > > > point;
> > > > > > > maybe we can add another metadata K-V or just a metadata string
> > > along
> > > > > > side
> > > > > > > with config K-V in create topic like we did for offset commit
> > > > request.
> > > > > > This
> > > > > > > field can be quite useful in storing information like "owner" of
> > > the
> > > > > > topic
> > > > > > > who issue the create command, etc, which is quite important for a
> > > > > > > multi-tenant setting. Then in the describe topic request we can
> > > also
> > > > > > batch
> > > > > > > on regex of the metadata field.
> > > > > > >
> > > > > > > A: As discussed it is very interesting but can be implemented
> > later
> > > > > after
> > > > > > > we have some basic functionality there.
> > > > > > >
> > > > > > > 15. Today all the admin operations are async in the sense that
> > > > command
> > > > > > will
> > > > > > > return once it is written in ZK, and that is why we need extra
> > > > > > verification
> > > > > > > like testUtil.waitForTopicCreated() / verify partition
> > reassignment
> > > > > > > request, etc. With admin requests we could add a flag to enable /
> > > > > disable
> > > > > > > synchronous requests; when it is turned on, the response will not
> > > > > return
> > > > > > > until the request has been completed. And for async requests we
> > can
> > > > > add a
> > > > > > > "token" field in the response, and then only need a general
> > "admin
> > > > > > > verification request" with the given token to check if the async
> > > > > request
> > > > > > > has been completed.
> > > > > > >
> > > > > > > A: I see your point. My idea was to provide specific
> > > Verify...Request
> > > > > per
> > > > > > > each
> > > > > > > long running request, where needed. We can do it the way you
> > > suggest.
> > > > > The
> > > > > > > only
> > > > > > > concern is that introducing a token we again will make schema
> > > > > "dynamic".
> > > > > > We
> > > > > > > wanted
> > > > > > > to do similar thing introducing single AdminRequest for all topic
> > > > > > commands
> > > > > > > but rejected
> > > > > > > this idea because we wanted to have schema defined. So this is
> > > more a
> > > > > > > choice between:
> > > > > > > a) have fixed schema but introduce each time new Verify...Request
> > > for
> > > > > > > long-running requests
> > > > > > > b) use one request for verification but generalize it with token
> > > > > > > I'm fine with whatever decision community come to. Just let me
> > know
> > > > > your
> > > > > > > thoughts.
> > > > > > >
> > > > > > > Comment from Gwen:
> > > > > > > 16. Specifically for ownership, I think the plan is to add ACL
> > (it
> > > > > sounds
> > > > > > > like you are describing ACL) via an external system (Argus,
> > > Sentry).
> > > > > > > I remember KIP-11 described this, but I can't find the KIP any
> > > > longer.
> > > > > > >
> > > > > > > A: Okay, no problem. Not sure though how we are going to handle
> > it.
> > > > > Wait
> > > > > > > which KIP
> > > > > > > will be committed first and include changes to TopicMetadata from
> > > the
> > > > > > later
> > > > > > > one?
> > > > > > > Anyway, I added this note to "Open Questions" section so we don't
> > > > miss
> > > > > > this
> > > > > > > piece.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >
> > > > > > > > Hi all,
> > > > > > > >
> > > > > > > > Today I uploaded the patch that covers some of the discussed
> > and
> > > > > agreed
> > > > > > > > items:
> > > > > > > > - removed MaybeOf optional type
> > > > > > > > - switched to java protocol definitions
> > > > > > > > - simplified messages (normalized configs, removed topic marked
> > > for
> > > > > > > > deletion)
> > > > > > > >
> > > > > > > > I also updated the KIP-4 with respective changes and wrote down
> > > my
> > > > > > > > proposal for
> > > > > > > > pending items:
> > > > > > > > - Batch Admin Operations -> updated Wire Protocol schema
> > proposal
> > > > > > > > - Remove ClusterMetadata -> changed to extend
> > > TopicMetadataRequest
> > > > > > > > - Admin Client -> updated my initial proposal to reflect
> > batching
> > > > > > > > - Error codes -> proposed fine-grained error code instead of
> > > > > > > > AdminRequestFailed
> > > > > > > >
> > > > > > > > I will also send a separate email to cover all comments from
> > this
> > > > > > thread.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Andrii Biletskyi
> > > > > > > >
> > > > > > > >
> > > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> > > > gshapira@cloudera.com
> > > > > >
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > >> Found KIP-11 (
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > > >> )
> > > > > > > >> It actually specifies changes to the Metadata protocol, so
> > > making
> > > > > sure
> > > > > > > >> both KIPs are consistent in this regard will be good.
> > > > > > > >>
> > > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> > > > > gshapira@cloudera.com
> > > > > > >
> > > > > > > >> wrote:
> > > > > > > >> > Specifically for ownership, I think the plan is to add ACL
> > (it
> > > > > > sounds
> > > > > > > >> > like you are describing ACL) via an external system (Argus,
> > > > > Sentry).
> > > > > > > >> > I remember KIP-11 described this, but I can't find the KIP
> > any
> > > > > > longer.
> > > > > > > >> >
> > > > > > > >> > Regardless, I think KIP-4 focuses on getting information
> > that
> > > > > > already
> > > > > > > >> > exists from Kafka brokers, not on adding information that
> > > > perhaps
> > > > > > > >> > should exist but doesn't yet?
> > > > > > > >> >
> > > > > > > >> > Gwen
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> > > > > wangguoz@gmail.com>
> > > > > > > >> wrote:
> > > > > > > >> >> Folks,
> > > > > > > >> >>
> > > > > > > >> >> Just want to elaborate a bit more on the create-topic
> > > metadata
> > > > > and
> > > > > > > >> batching
> > > > > > > >> >> describe-topic based on config / metadata in my previous
> > > email
> > > > as
> > > > > > we
> > > > > > > >> work
> > > > > > > >> >> on KAFKA-1694. The main motivation is to have some sort of
> > > > topic
> > > > > > > >> management
> > > > > > > >> >> mechanisms, which I think is quite important in a
> > > multi-tenant
> > > > /
> > > > > > > cloud
> > > > > > > >> >> architecture: today anyone can create topics in a shared
> > > Kafka
> > > > > > > >> cluster, but
> > > > > > > >> >> there is no concept or "ownership" of topics that are
> > created
> > > > by
> > > > > > > >> different
> > > > > > > >> >> users. For example, at LinkedIn we basically distinguish
> > > topic
> > > > > > owners
> > > > > > > >> via
> > > > > > > >> >> some casual topic name prefix, which is a bit awkward and
> > > does
> > > > > not
> > > > > > > fly
> > > > > > > >> as
> > > > > > > >> >> we scale our customers. It would be great to use
> > > > describe-topics
> > > > > > such
> > > > > > > >> as:
> > > > > > > >> >>
> > > > > > > >> >> Describe all topics that is created by me.
> > > > > > > >> >>
> > > > > > > >> >> Describe all topics whose retention time is overriden to X.
> > > > > > > >> >>
> > > > > > > >> >> Describe all topics whose writable group include user Y
> > (this
> > > > is
> > > > > > > >> related to
> > > > > > > >> >> authorization), etc..
> > > > > > > >> >>
> > > > > > > >> >> One possible way to achieve this is to add a metadata file
> > in
> > > > the
> > > > > > > >> >> create-topic request, whose value will also be written ZK
> > as
> > > we
> > > > > > > create
> > > > > > > >> the
> > > > > > > >> >> topic; then describe-topics can choose to batch topics
> > based
> > > on
> > > > > 1)
> > > > > > > name
> > > > > > > >> >> regex, 2) config K-V matching, 3) metadata regex, etc.
> > > > > > > >> >>
> > > > > > > >> >> Thoughts?
> > > > > > > >> >>
> > > > > > > >> >> Guozhang
> > > > > > > >> >>
> > > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> > > > > wangguoz@gmail.com>
> > > > > > > >> wrote:
> > > > > > > >> >>
> > > > > > > >> >>> Thanks for the updated wiki. A few comments below:
> > > > > > > >> >>>
> > > > > > > >> >>> 1. Error description in response: I think if some
> > errorCode
> > > > > could
> > > > > > > >> indicate
> > > > > > > >> >>> several different error cases then we should really change
> > > it
> > > > to
> > > > > > > >> multiple
> > > > > > > >> >>> codes. In general the errorCode itself would be precise
> > and
> > > > > > > >> sufficient for
> > > > > > > >> >>> describing the server side errors.
> > > > > > > >> >>>
> > > > > > > >> >>> 2. Describe topic request: it would be great to go beyond
> > > just
> > > > > > > >> batching on
> > > > > > > >> >>> topic name regex for this request. For example, a very
> > > common
> > > > > use
> > > > > > > >> case of
> > > > > > > >> >>> the topic command is to list all topics whose config A's
> > > value
> > > > > is
> > > > > > B.
> > > > > > > >> With
> > > > > > > >> >>> topic name regex then we have to first retrieve __all__
> > > > topics's
> > > > > > > >> >>> description info and then filter at the client end, which
> > > will
> > > > > be
> > > > > > a
> > > > > > > >> huge
> > > > > > > >> >>> burden on ZK.
> > > > > > > >> >>>
> > > > > > > >> >>> 3. Config K-Vs in create topic: this is related to the
> > > > previous
> > > > > > > point;
> > > > > > > >> >>> maybe we can add another metadata K-V or just a metadata
> > > > string
> > > > > > > along
> > > > > > > >> side
> > > > > > > >> >>> with config K-V in create topic like we did for offset
> > > commit
> > > > > > > >> request. This
> > > > > > > >> >>> field can be quite useful in storing information like
> > > "owner"
> > > > of
> > > > > > the
> > > > > > > >> topic
> > > > > > > >> >>> who issue the create command, etc, which is quite
> > important
> > > > for
> > > > > a
> > > > > > > >> >>> multi-tenant setting. Then in the describe topic request
> > we
> > > > can
> > > > > > also
> > > > > > > >> batch
> > > > > > > >> >>> on regex of the metadata field.
> > > > > > > >> >>>
> > > > > > > >> >>> 4. Today all the admin operations are async in the sense
> > > that
> > > > > > > command
> > > > > > > >> will
> > > > > > > >> >>> return once it is written in ZK, and that is why we need
> > > extra
> > > > > > > >> verification
> > > > > > > >> >>> like testUtil.waitForTopicCreated() / verify partition
> > > > > > reassignment
> > > > > > > >> >>> request, etc. With admin requests we could add a flag to
> > > > enable
> > > > > /
> > > > > > > >> disable
> > > > > > > >> >>> synchronous requests; when it is turned on, the response
> > > will
> > > > > not
> > > > > > > >> return
> > > > > > > >> >>> until the request has been completed. And for async
> > requests
> > > > we
> > > > > > can
> > > > > > > >> add a
> > > > > > > >> >>> "token" field in the response, and then only need a
> > general
> > > > > "admin
> > > > > > > >> >>> verification request" with the given token to check if the
> > > > async
> > > > > > > >> request
> > > > > > > >> >>> has been completed.
> > > > > > > >> >>>
> > > > > > > >> >>> 5. +1 for extending Metadata request to include
> > controller /
> > > > > > > >> coordinator
> > > > > > > >> >>> information, and then we can remove the ConsumerMetadata /
> > > > > > > >> ClusterMetadata
> > > > > > > >> >>> requests.
> > > > > > > >> >>>
> > > > > > > >> >>> Guozhang
> > > > > > > >> >>>
> > > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> > > > > jjkoshy.w@gmail.com>
> > > > > > > >> wrote:
> > > > > > > >> >>>
> > > > > > > >> >>>> Thanks for sending that out Joe - I don't think I will be
> > > > able
> > > > > to
> > > > > > > >> make
> > > > > > > >> >>>> it today, so if notes can be sent out afterward that
> > would
> > > be
> > > > > > > great.
> > > > > > > >> >>>>
> > > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira
> > > wrote:
> > > > > > > >> >>>> > Thanks for sending this out Joe. Looking forward to
> > > > chatting
> > > > > > with
> > > > > > > >> >>>> everyone :)
> > > > > > > >> >>>> >
> > > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> > > > > > joe.stein@stealth.ly>
> > > > > > > >> wrote:
> > > > > > > >> >>>> > > Hey, I just sent out a google hangout invite to all
> > > pmc,
> > > > > > > >> committers
> > > > > > > >> >>>> and
> > > > > > > >> >>>> > > everyone I found working on a KIP. If I missed anyone
> > > in
> > > > > the
> > > > > > > >> invite
> > > > > > > >> >>>> please
> > > > > > > >> >>>> > > let me know and can update it, np.
> > > > > > > >> >>>> > >
> > > > > > > >> >>>> > > We should do this every Tuesday @ 2pm Eastern Time.
> > > Maybe
> > > > > we
> > > > > > > can
> > > > > > > >> get
> > > > > > > >> >>>> INFRA
> > > > > > > >> >>>> > > help to make a google account so we can manage
> > better?
> > > > > > > >> >>>> > >
> > > > > > > >> >>>> > > To discuss
> > > > > > > >> >>>> > >
> > > > > > > >> >>>>
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > > >> >>>> > > in progress and related JIRA that are interdependent
> > > and
> > > > > > common
> > > > > > > >> work.
> > > > > > > >> >>>> > >
> > > > > > > >> >>>> > > ~ Joe Stein
> > > > > > > >> >>>> > >
> > > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
> > > > > > > jay.kreps@gmail.com>
> > > > > > > >> >>>> wrote:
> > > > > > > >> >>>> > >
> > > > > > > >> >>>> > >> Let's stay on Google hangouts that will also record
> > > and
> > > > > make
> > > > > > > the
> > > > > > > >> >>>> sessions
> > > > > > > >> >>>> > >> available on youtube.
> > > > > > > >> >>>> > >>
> > > > > > > >> >>>> > >> -Jay
> > > > > > > >> >>>> > >>
> > > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> > > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > > >> >>>> > >> wrote:
> > > > > > > >> >>>> > >>
> > > > > > > >> >>>> > >> > Jay / Joe
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >> > We're happy to send out a Webex for this purpose.
> > We
> > > > > could
> > > > > > > >> record
> > > > > > > >> >>>> the
> > > > > > > >> >>>> > >> > sessions if there is interest and publish them
> > out.
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >> > Thanks
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >> > Jeff
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> > > > > > > >> jay.kreps@gmail.com>
> > > > > > > >> >>>> wrote:
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >> > > Let's try to get the technical hang-ups sorted
> > > out,
> > > > > > > though.
> > > > > > > >> I
> > > > > > > >> >>>> really
> > > > > > > >> >>>> > >> > think
> > > > > > > >> >>>> > >> > > there is some benefit to live discussion vs
> > > > writing. I
> > > > > > am
> > > > > > > >> >>>> hopeful that
> > > > > > > >> >>>> > >> if
> > > > > > > >> >>>> > >> > > we post instructions and give ourselves a few
> > > > attempts
> > > > > > we
> > > > > > > >> can
> > > > > > > >> >>>> get it
> > > > > > > >> >>>> > >> > > working.
> > > > > > > >> >>>> > >> > >
> > > > > > > >> >>>> > >> > > Tuesday at that time would work for me...any
> > > > > objections?
> > > > > > > >> >>>> > >> > >
> > > > > > > >> >>>> > >> > > -Jay
> > > > > > > >> >>>> > >> > >
> > > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> > > > > > > >> joe.stein@stealth.ly
> > > > > > > >> >>>> >
> > > > > > > >> >>>> > >> wrote:
> > > > > > > >> >>>> > >> > >
> > > > > > > >> >>>> > >> > > > Weekly would be great maybe like every
> > Tuesday ~
> > > > 1pm
> > > > > > ET
> > > > > > > /
> > > > > > > >> 10am
> > > > > > > >> >>>> PT
> > > > > > > >> >>>> > >> ????
> > > > > > > >> >>>> > >> > > >
> > > > > > > >> >>>> > >> > > > I don't mind google hangout but there is
> > always
> > > > some
> > > > > > > >> issue or
> > > > > > > >> >>>> > >> whatever
> > > > > > > >> >>>> > >> > so
> > > > > > > >> >>>> > >> > > > we know the apache irc channel works. We can
> > > start
> > > > > > there
> > > > > > > >> and
> > > > > > > >> >>>> see how
> > > > > > > >> >>>> > >> it
> > > > > > > >> >>>> > >> > > > goes? We can pull transcripts too and
> > associate
> > > to
> > > > > > > >> tickets if
> > > > > > > >> >>>> need be
> > > > > > > >> >>>> > >> > > makes
> > > > > > > >> >>>> > >> > > > it helpful for things.
> > > > > > > >> >>>> > >> > > >
> > > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > > >> >>>> > >> > > >
> > > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> > > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > > >> >>>> > >> > wrote:
> > > > > > > >> >>>> > >> > > >
> > > > > > > >> >>>> > >> > > > > We'd talked about doing a Google Hangout to
> > > chat
> > > > > > about
> > > > > > > >> this.
> > > > > > > >> >>>> What
> > > > > > > >> >>>> > >> > about
> > > > > > > >> >>>> > >> > > > > generalizing that a little further...I
> > > actually
> > > > > > think
> > > > > > > it
> > > > > > > >> >>>> would be
> > > > > > > >> >>>> > >> > good
> > > > > > > >> >>>> > >> > > > for
> > > > > > > >> >>>> > >> > > > > everyone spending a reasonable chunk of
> > their
> > > > week
> > > > > > on
> > > > > > > >> Kafka
> > > > > > > >> >>>> stuff
> > > > > > > >> >>>> > >> to
> > > > > > > >> >>>> > >> > > > maybe
> > > > > > > >> >>>> > >> > > > > sync up once a week. I think we could use
> > time
> > > > to
> > > > > > talk
> > > > > > > >> >>>> through
> > > > > > > >> >>>> > >> design
> > > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of code
> > > reviews,
> > > > > talk
> > > > > > > >> through
> > > > > > > >> >>>> any
> > > > > > > >> >>>> > >> > tricky
> > > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > > > >> >>>> > >> > > > >
> > > > > > > >> >>>> > >> > > > > We can make it publicly available so that
> > any
> > > > one
> > > > > > can
> > > > > > > >> follow
> > > > > > > >> >>>> along
> > > > > > > >> >>>> > >> > who
> > > > > > > >> >>>> > >> > > > > likes.
> > > > > > > >> >>>> > >> > > > >
> > > > > > > >> >>>> > >> > > > > Any interest in doing this? If so I'll try
> > to
> > > > set
> > > > > it
> > > > > > > up
> > > > > > > >> >>>> starting
> > > > > > > >> >>>> > >> next
> > > > > > > >> >>>> > >> > > > week.
> > > > > > > >> >>>> > >> > > > >
> > > > > > > >> >>>> > >> > > > > -Jay
> > > > > > > >> >>>> > >> > > > >
> > > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii
> > > > Biletskyi
> > > > > <
> > > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >> >>>> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed / aligned
> > > > document
> > > > > > > >> structure.
> > > > > > > >> >>>> Also I
> > > > > > > >> >>>> > >> > > added
> > > > > > > >> >>>> > >> > > > > > some
> > > > > > > >> >>>> > >> > > > > > very initial proposal for AdminClient so
> > we
> > > > have
> > > > > > > >> something
> > > > > > > >> >>>> to
> > > > > > > >> >>>> > >> start
> > > > > > > >> >>>> > >> > > > from
> > > > > > > >> >>>> > >> > > > > > while
> > > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> >>>> > >> > > > >
> > > > > > > >> >>>> > >> > > >
> > > > > > > >> >>>> > >> > >
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >>
> > > > > > > >> >>>>
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > Thanks,
> > > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii
> > > > > Biletskyi
> > > > > > <
> > > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > > Jay,
> > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > > > Re error messages: you are right, in
> > most
> > > > > cases
> > > > > > > >> client
> > > > > > > >> >>>> will
> > > > > > > >> >>>> > >> have
> > > > > > > >> >>>> > >> > > > enough
> > > > > > > >> >>>> > >> > > > > > > context to show descriptive error
> > message.
> > > > My
> > > > > > > >> concern is
> > > > > > > >> >>>> that
> > > > > > > >> >>>> > >> we
> > > > > > > >> >>>> > >> > > will
> > > > > > > >> >>>> > >> > > > > > have
> > > > > > > >> >>>> > >> > > > > > > to
> > > > > > > >> >>>> > >> > > > > > > add lots of new error codes for each
> > > > possible
> > > > > > > >> error. Of
> > > > > > > >> >>>> course,
> > > > > > > >> >>>> > >> > we
> > > > > > > >> >>>> > >> > > > > could
> > > > > > > >> >>>> > >> > > > > > > reuse
> > > > > > > >> >>>> > >> > > > > > > some of existing like
> > > > > > UknownTopicOrPartitionCode,
> > > > > > > >> but we
> > > > > > > >> >>>> will
> > > > > > > >> >>>> > >> > also
> > > > > > > >> >>>> > >> > > > need
> > > > > > > >> >>>> > >> > > > > > to
> > > > > > > >> >>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> > > > > > > >> >>>> TopicConfigInvalid (both
> > > > > > > >> >>>> > >> > for
> > > > > > > >> >>>> > >> > > > > topic
> > > > > > > >> >>>> > >> > > > > > > name and config, and probably user would
> > > > like
> > > > > to
> > > > > > > >> know
> > > > > > > >> >>>> what
> > > > > > > >> >>>> > >> > exactly
> > > > > > > >> >>>> > >> > > > > > > is wrong in his config),
> > > > > > InvalidReplicaAssignment,
> > > > > > > >> >>>> > >> InternalError
> > > > > > > >> >>>> > >> > > > (e.g.
> > > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > > > > > >> >>>> > >> > > > > > > And this is only for TopicCommand, we
> > will
> > > > > also
> > > > > > > >> need to
> > > > > > > >> >>>> add
> > > > > > > >> >>>> > >> > similar
> > > > > > > >> >>>> > >> > > > > stuff
> > > > > > > >> >>>> > >> > > > > > > for
> > > > > > > >> >>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So
> > > > we'll
> > > > > > end
> > > > > > > >> up
> > > > > > > >> >>>> with a
> > > > > > > >> >>>> > >> > large
> > > > > > > >> >>>> > >> > > > list
> > > > > > > >> >>>> > >> > > > > > of
> > > > > > > >> >>>> > >> > > > > > > error codes, used only in Admin
> > protocol.
> > > > > > > >> >>>> > >> > > > > > > Having said that, I agree my proposal is
> > > not
> > > > > > > >> consistent
> > > > > > > >> >>>> with
> > > > > > > >> >>>> > >> > other
> > > > > > > >> >>>> > >> > > > > cases.
> > > > > > > >> >>>> > >> > > > > > > Maybe we can find better solution or
> > > > something
> > > > > > > >> >>>> in-between.
> > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it is a great
> > > idea.
> > > > > > This
> > > > > > > >> way we
> > > > > > > >> >>>> can
> > > > > > > >> >>>> > >> move
> > > > > > > >> >>>> > >> > > on
> > > > > > > >> >>>> > >> > > > > > > faster.
> > > > > > > >> >>>> > >> > > > > > > Let's agree somehow on date/time so
> > people
> > > > can
> > > > > > > join.
> > > > > > > >> >>>> Will work
> > > > > > > >> >>>> > >> > for
> > > > > > > >> >>>> > >> > > me
> > > > > > > >> >>>> > >> > > > > > this
> > > > > > > >> >>>> > >> > > > > > > and
> > > > > > > >> >>>> > >> > > > > > > next week almost anytime if agreed in
> > > > advance.
> > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > > > Thanks,
> > > > > > > >> >>>> > >> > > > > > > Andrii
> > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay
> > > Kreps <
> > > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > > > > >> >>>> > >> > > > > wrote:
> > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> >>>> > >> > > > > > >> Generally we can do good error handling
> > > > > without
> > > > > > > >> needing
> > > > > > > >> >>>> custom
> > > > > > > >> >>>> > >> > > > > > server-side
> > > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the client has
> > > the
> > > > > > > >> context to
> > > > > > > >> >>>> know
> > > > > > > >> >>>> > >> that
> > > > > > > >> >>>> > >> > > if
> > > > > > > >> >>>> > >> > > > it
> > > > > > > >> >>>> > >> > > > > > got
> > > > > > > >> >>>> > >> > > > > > >> an error that the topic doesn't exist
> > to
> > > > say
> > > > > > > >> "Topic X
> > > > > > > >> >>>> doesn't
> > > > > > > >> >>>> > >> > > exist"
> > > > > > > >> >>>> > >> > > > > > >> rather
> > > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or whatever).
> > Maybe
> > > > > there
> > > > > > > are
> > > > > > > >> >>>> specific
> > > > > > > >> >>>> > >> > cases
> > > > > > > >> >>>> > >> > > > > where
> > > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to add
> > > server-side
> > > > > > error
> > > > > > > >> >>>> messages we
> > > > > > > >> >>>> > >> > > really
> > > > > > > >> >>>> > >> > > > > do
> > > > > > > >> >>>> > >> > > > > > >> need to do this in a consistent way
> > > across
> > > > > the
> > > > > > > >> protocol.
> > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open questions
> > > here
> > > > > > from
> > > > > > > my
> > > > > > > >> >>>> previous
> > > > > > > >> >>>> > >> > > list. I
> > > > > > > >> >>>> > >> > > > > > will
> > > > > > > >> >>>> > >> > > > > > >> be out for the next few days for Strata
> > > > > though.
> > > > > > > >> Maybe
> > > > > > > >> >>>> we could
> > > > > > > >> >>>> > >> > do
> > > > > > > >> >>>> > >> > > a
> > > > > > > >> >>>> > >> > > > > > Google
> > > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open issues some
> > time
> > > > > > towards
> > > > > > > >> the
> > > > > > > >> >>>> end of
> > > > > > > >> >>>> > >> > next
> > > > > > > >> >>>> > >> > > > week
> > > > > > > >> >>>> > >> > > > > > for
> > > > > > > >> >>>> > >> > > > > > >> anyone interested in this ticket? I
> > have
> > > a
> > > > > > > feeling
> > > > > > > >> that
> > > > > > > >> >>>> might
> > > > > > > >> >>>> > >> > > > progress
> > > > > > > >> >>>> > >> > > > > > >> things a little faster than email--I
> > > think
> > > > we
> > > > > > > >> could talk
> > > > > > > >> >>>> > >> through
> > > > > > > >> >>>> > >> > > > those
> > > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly quickly...
> > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> >>>> > >> > > > > > >> -Jay
> > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii
> > > > > > > Biletskyi <
> > > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >> > I'm trying to address some of the
> > > issues
> > > > > > which
> > > > > > > >> were
> > > > > > > >> >>>> > >> mentioned
> > > > > > > >> >>>> > >> > > > > earlier
> > > > > > > >> >>>> > >> > > > > > >> about
> > > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of those was
> > > > about
> > > > > > > >> batching
> > > > > > > >> >>>> > >> > operations.
> > > > > > > >> >>>> > >> > > > What
> > > > > > > >> >>>> > >> > > > > > if
> > > > > > > >> >>>> > >> > > > > > >> we
> > > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand approach and let
> > > > people
> > > > > > > >> specify
> > > > > > > >> >>>> > >> topic-name
> > > > > > > >> >>>> > >> > > by
> > > > > > > >> >>>> > >> > > > > > >> regexp -
> > > > > > > >> >>>> > >> > > > > > >> > would that cover most of the use
> > cases?
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >> > Secondly, is what information should
> > we
> > > > > > > generally
> > > > > > > >> >>>> provide in
> > > > > > > >> >>>> > >> > > Admin
> > > > > > > >> >>>> > >> > > > > > >> > responses.
> > > > > > > >> >>>> > >> > > > > > >> > I realize that Admin commands don't
> > > imply
> > > > > > they
> > > > > > > >> will
> > > > > > > >> >>>> be used
> > > > > > > >> >>>> > >> > only
> > > > > > > >> >>>> > >> > > > in
> > > > > > > >> >>>> > >> > > > > > CLI
> > > > > > > >> >>>> > >> > > > > > >> > but,
> > > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a very
> > important
> > > > > > client
> > > > > > > >> of this
> > > > > > > >> >>>> > >> > feature.
> > > > > > > >> >>>> > >> > > In
> > > > > > > >> >>>> > >> > > > > > this
> > > > > > > >> >>>> > >> > > > > > >> > case,
> > > > > > > >> >>>> > >> > > > > > >> > seems logical, we would like to
> > provide
> > > > > users
> > > > > > > >> with
> > > > > > > >> >>>> rich
> > > > > > > >> >>>> > >> > > experience
> > > > > > > >> >>>> > >> > > > > in
> > > > > > > >> >>>> > >> > > > > > >> terms
> > > > > > > >> >>>> > >> > > > > > >> > of
> > > > > > > >> >>>> > >> > > > > > >> > getting results / errors of the
> > > executed
> > > > > > > >> commands.
> > > > > > > >> >>>> Usually
> > > > > > > >> >>>> > >> we
> > > > > > > >> >>>> > >> > > > supply
> > > > > > > >> >>>> > >> > > > > > >> with
> > > > > > > >> >>>> > >> > > > > > >> > responses only errorCode, which looks
> > > > very
> > > > > > > >> limiting,
> > > > > > > >> >>>> in case
> > > > > > > >> >>>> > >> > of
> > > > > > > >> >>>> > >> > > > CLI
> > > > > > > >> >>>> > >> > > > > we
> > > > > > > >> >>>> > >> > > > > > >> may
> > > > > > > >> >>>> > >> > > > > > >> > want to print human readable error
> > > > > > description.
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >> > So, taking into account previous item
> > > > about
> > > > > > > >> batching,
> > > > > > > >> >>>> what
> > > > > > > >> >>>> > >> do
> > > > > > > >> >>>> > >> > > you
> > > > > > > >> >>>> > >> > > > > > think
> > > > > > > >> >>>> > >> > > > > > >> > about
> > > > > > > >> >>>> > >> > > > > > >> > having smth like:
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support regexp)
> > > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName
> > > > Partitions
> > > > > > > >> Replicas
> > > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > > > > > >> >>>> > >> > > > > > >> > [Config]
> > > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode
> > > > > > > ErrorDescription
> > > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription => string (empty
> > if
> > > > > > > >> successful)
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp
> > > > > > Partitions
> > > > > > > >> >>>> > >> > > ReplicaAssignment
> > > > > > > >> >>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> > > > > > > >> >>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName
> > > > ErrorCode
> > > > > > > >> >>>> ErrorDescription]
> > > > > > > >> >>>> > >> > > > > > >> > CommandErrorCode
> > > CommandErrorDescription
> > > > > > > >> >>>> > >> > > > > > >> >   CommandErrorCode => int16
> > > > > > > >> >>>> > >> > > > > > >> >   CommandErrorDescription => string
> > > > > (nonempty
> > > > > > > in
> > > > > > > >> case
> > > > > > > >> >>>> of
> > > > > > > >> >>>> > >> fatal
> > > > > > > >> >>>> > >> > > > > error,
> > > > > > > >> >>>> > >> > > > > > >> e.g.
> > > > > > > >> >>>> > >> > > > > > >> > we couldn't get topics by regexp)
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >> > DescribeTopicRequest ->
> > TopicNameRegexp
> > > > > > > >> >>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> > > > > > > >> TopicDescription
> > > > > > > >> >>>> > >> ErrorCode
> > > > > > > >> >>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> > > > > > > >> >>>> CommandErrorDescription
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >> > Also, any thoughts about our
> > discussion
> > > > > > > regarding
> > > > > > > >> >>>> re-routing
> > > > > > > >> >>>> > >> > > > > facility?
> > > > > > > >> >>>> > >> > > > > > >> In
> > > > > > > >> >>>> > >> > > > > > >> > my
> > > > > > > >> >>>> > >> > > > > > >> > understanding, it is like between
> > > > > augmenting
> > > > > > > >> >>>> > >> > > TopicMetadataRequest
> > > > > > > >> >>>> > >> > > > > > >> > (to include at least controllerId)
> > and
> > > > > > > >> implementing
> > > > > > > >> >>>> new
> > > > > > > >> >>>> > >> > generic
> > > > > > > >> >>>> > >> > > > > > >> re-routing
> > > > > > > >> >>>> > >> > > > > > >> > facility so sending messages to
> > > > controller
> > > > > > will
> > > > > > > >> be
> > > > > > > >> >>>> handled
> > > > > > > >> >>>> > >> by
> > > > > > > >> >>>> > >> > > it.
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >> > Thanks,
> > > > > > > >> >>>> > >> > > > > > >> > Andrii Biletskyi
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM,
> > Andrii
> > > > > > > >> Biletskyi <
> > > > > > > >> >>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >> > > @Guozhang:
> > > > > > > >> >>>> > >> > > > > > >> > > Thanks for your comments, I've
> > > answered
> > > > > > some
> > > > > > > of
> > > > > > > >> >>>> those. The
> > > > > > > >> >>>> > >> > > main
> > > > > > > >> >>>> > >> > > > > > thing
> > > > > > > >> >>>> > >> > > > > > >> is
> > > > > > > >> >>>> > >> > > > > > >> > > having merged request for
> > > > > > > >> >>>> create-alter-delete-describe - I
> > > > > > > >> >>>> > >> > > have
> > > > > > > >> >>>> > >> > > > > some
> > > > > > > >> >>>> > >> > > > > > >> > > concerns about this approach.
> > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > > @*Jay*:
> > > > > > > >> >>>> > >> > > > > > >> > > I see that introduced
> > > > > ClusterMetadaRequest
> > > > > > is
> > > > > > > >> also
> > > > > > > >> >>>> one of
> > > > > > > >> >>>> > >> > the
> > > > > > > >> >>>> > >> > > > > > >> concerns.
> > > > > > > >> >>>> > >> > > > > > >> > We
> > > > > > > >> >>>> > >> > > > > > >> > > can solve it if we implement
> > > re-routing
> > > > > > > >> facility.
> > > > > > > >> >>>> But I
> > > > > > > >> >>>> > >> > agree
> > > > > > > >> >>>> > >> > > > with
> > > > > > > >> >>>> > >> > > > > > >> > > Guozhang - it will make clients'
> > > > > internals
> > > > > > a
> > > > > > > >> little
> > > > > > > >> >>>> bit
> > > > > > > >> >>>> > >> > easier
> > > > > > > >> >>>> > >> > > > but
> > > > > > > >> >>>> > >> > > > > > >> this
> > > > > > > >> >>>> > >> > > > > > >> > > seems to be a complex logic to
> > > > implement
> > > > > > and
> > > > > > > >> >>>> support then.
> > > > > > > >> >>>> > >> > > > > > Especially
> > > > > > > >> >>>> > >> > > > > > >> for
> > > > > > > >> >>>> > >> > > > > > >> > > Fetch and Produce (even if we add
> > > > > > re-routing
> > > > > > > >> later
> > > > > > > >> >>>> for
> > > > > > > >> >>>> > >> these
> > > > > > > >> >>>> > >> > > > > > >> requests).
> > > > > > > >> >>>> > >> > > > > > >> > > Also people will tend to avoid this
> > > > > > > re-routing
> > > > > > > >> >>>> facility
> > > > > > > >> >>>> > >> and
> > > > > > > >> >>>> > >> > > hold
> > > > > > > >> >>>> > >> > > > > > local
> > > > > > > >> >>>> > >> > > > > > >> > > cluster cache to ensure their
> > > > > high-priority
> > > > > > > >> requests
> > > > > > > >> >>>> > >> (which
> > > > > > > >> >>>> > >> > > some
> > > > > > > >> >>>> > >> > > > > of
> > > > > > > >> >>>> > >> > > > > > >> the
> > > > > > > >> >>>> > >> > > > > > >> > > admin requests are) not sent to
> > some
> > > > busy
> > > > > > > >> broker
> > > > > > > >> >>>> where
> > > > > > > >> >>>> > >> they
> > > > > > > >> >>>> > >> > > wait
> > > > > > > >> >>>> > >> > > > > to
> > > > > > > >> >>>> > >> > > > > > be
> > > > > > > >> >>>> > >> > > > > > >> > > routed to the correct one.
> > > > > > > >> >>>> > >> > > > > > >> > > As pointed out by Jun here (
> > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> >>>> > >> > > > >
> > > > > > > >> >>>> > >> > > >
> > > > > > > >> >>>> > >> > >
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >>
> > > > > > > >> >>>>
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > > > > > >> >>>> > >> > > > > > >> > )
> > > > > > > >> >>>> > >> > > > > > >> > > to solve the issue we might
> > > introduce a
> > > > > > > message
> > > > > > > >> >>>> type to
> > > > > > > >> >>>> > >> get
> > > > > > > >> >>>> > >> > > > > cluster
> > > > > > > >> >>>> > >> > > > > > >> > state.
> > > > > > > >> >>>> > >> > > > > > >> > > But I agree we can just update
> > > > > > > >> >>>> TopicMetadataResponse to
> > > > > > > >> >>>> > >> > > include
> > > > > > > >> >>>> > >> > > > > > >> > > controllerId (and probably smth
> > > else).
> > > > > > > >> >>>> > >> > > > > > >> > > What are you thougths?
> > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > > Thanks,
> > > > > > > >> >>>> > >> > > > > > >> > > Andrii
> > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM,
> > > > Guozhang
> > > > > > > Wang
> > > > > > > >> <
> > > > > > > >> >>>> > >> > > > > wangguoz@gmail.com>
> > > > > > > >> >>>> > >> > > > > > >> > wrote:
> > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > >> I think for the topics commands we
> > > can
> > > > > > > >> actually
> > > > > > > >> >>>> merge
> > > > > > > >> >>>> > >> > > > > > >> > >> create/alter/delete/describe as
> > one
> > > > > > request
> > > > > > > >> type
> > > > > > > >> >>>> since
> > > > > > > >> >>>> > >> > their
> > > > > > > >> >>>> > >> > > > > > formats
> > > > > > > >> >>>> > >> > > > > > >> are
> > > > > > > >> >>>> > >> > > > > > >> > >> very much similar, and keep
> > > > list-topics
> > > > > > and
> > > > > > > >> others
> > > > > > > >> >>>> like
> > > > > > > >> >>>> > >> > > > > > >> > >> partition-reassignment /
> > > > > > > >> preferred-leader-election
> > > > > > > >> >>>> as
> > > > > > > >> >>>> > >> > > separate
> > > > > > > >> >>>> > >> > > > > > >> request
> > > > > > > >> >>>> > >> > > > > > >> > >> types, I also left some other
> > > comments
> > > > > on
> > > > > > > the
> > > > > > > >> RB (
> > > > > > > >> >>>> > >> > > > > > >> > >>
> > https://reviews.apache.org/r/29301/
> > > ).
> > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > >> >>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM,
> > Jay
> > > > > > Kreps <
> > > > > > > >> >>>> > >> > > > jay.kreps@gmail.com>
> > > > > > > >> >>>> > >> > > > > > >> wrote:
> > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > >> >>>> > >> > > > > > >> > >> > Yeah I totally agree that we
> > don't
> > > > > want
> > > > > > to
> > > > > > > >> just
> > > > > > > >> >>>> have
> > > > > > > >> >>>> > >> one
> > > > > > > >> >>>> > >> > > "do
> > > > > > > >> >>>> > >> > > > > > admin
> > > > > > > >> >>>> > >> > > > > > >> > >> stuff"
> > > > > > > >> >>>> > >> > > > > > >> > >> > command that has the union of
> > all
> > > > > > > >> parameters.
> > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > >> >>>> > >> > > > > > >> > >> > What I am saying is that command
> > > > line
> > > > > > > tools
> > > > > > > >> are
> > > > > > > >> >>>> one
> > > > > > > >> >>>> > >> > client
> > > > > > > >> >>>> > >> > > of
> > > > > > > >> >>>> > >> > > > > the
> > > > > > > >> >>>> > >> > > > > > >> > >> > administrative apis, but these
> > > will
> > > > be
> > > > > > > used
> > > > > > > >> in a
> > > > > > > >> >>>> number
> > > > > > > >> >>>> > >> > of
> > > > > > > >> >>>> > >> > > > > > >> scenarios
> > > > > > > >> >>>> > >> > > > > > >> > so
> > > > > > > >> >>>> > >> > > > > > >> > >> > they should make logical sense
> > > even
> > > > in
> > > > > > the
> > > > > > > >> >>>> absence of
> > > > > > > >> >>>> > >> the
> > > > > > > >> >>>> > >> > > > > command
> > > > > > > >> >>>> > >> > > > > > >> line
> > > > > > > >> >>>> > >> > > > > > >> > >> > tool. Hence comments like trying
> > > to
> > > > > > > clarify
> > > > > > > >> the
> > > > > > > >> >>>> > >> > > relationship
> > > > > > > >> >>>> > >> > > > > > >> between
> > > > > > > >> >>>> > >> > > > > > >> > >> > ClusterMetadata and
> > > > > > TopicMetadata...these
> > > > > > > >> kinds
> > > > > > > >> >>>> of
> > > > > > > >> >>>> > >> things
> > > > > > > >> >>>> > >> > > > > really
> > > > > > > >> >>>> > >> > > > > > >> need
> > > > > > > >> >>>> > >> > > > > > >> > >> to be
> > > > > > > >> >>>> > >> > > > > > >> > >> > thought through.
> > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > >> >>>> > >> > > > > > >> > >> > Hope that makes sense.
> > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > >> >>>> > >> > > > > > >> > >> > -Jay
> > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > >> >>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM,
> > > > > Andrii
> > > > > > > >> >>>> Biletskyi <
> > > > > > > >> >>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > Jay,
> > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > Thanks for answering. You
> > > > understood
> > > > > > > >> >>>> correctly, most
> > > > > > > >> >>>> > >> of
> > > > > > > >> >>>> > >> > > my
> > > > > > > >> >>>> > >> > > > > > >> comments
> > > > > > > >> >>>> > >> > > > > > >> > >> were
> > > > > > > >> >>>> > >> > > > > > >> > >> > > related to your point 1) -
> > about
> > > > > "well
> > > > > > > >> >>>> thought-out"
> > > > > > > >> >>>> > >> > apis.
> > > > > > > >> >>>> > >> > > > > Also,
> > > > > > > >> >>>> > >> > > > > > >> yes,
> > > > > > > >> >>>> > >> > > > > > >> > >> as I
> > > > > > > >> >>>> > >> > > > > > >> > >> > > understood we would like to
> > > > > introduce
> > > > > > a
> > > > > > > >> single
> > > > > > > >> >>>> > >> unified
> > > > > > > >> >>>> > >> > > CLI
> > > > > > > >> >>>> > >> > > > > tool
> > > > > > > >> >>>> > >> > > > > > >> with
> > > > > > > >> >>>> > >> > > > > > >> > >> > > centralized server-side
> > request
> > > > > > handling
> > > > > > > >> for
> > > > > > > >> >>>> lots of
> > > > > > > >> >>>> > >> > > > existing
> > > > > > > >> >>>> > >> > > > > > >> ones
> > > > > > > >> >>>> > >> > > > > > >> > >> (incl.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > TopicCommand,
> > > CommitOffsetChecker,
> > > > > > > >> >>>> > >> ReassignPartitions,
> > > > > > > >> >>>> > >> > > smth
> > > > > > > >> >>>> > >> > > > > > else
> > > > > > > >> >>>> > >> > > > > > >> if
> > > > > > > >> >>>> > >> > > > > > >> > >> added
> > > > > > > >> >>>> > >> > > > > > >> > >> > > in future). In our previous
> > > > > > discussion (
> > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694)
> > > > > > > >> >>>> > >> > people
> > > > > > > >> >>>> > >> > > > > said
> > > > > > > >> >>>> > >> > > > > > >> > they'd
> > > > > > > >> >>>> > >> > > > > > >> > >> > > rather
> > > > > > > >> >>>> > >> > > > > > >> > >> > > have a separate message for
> > each
> > > > > > > command,
> > > > > > > >> so,
> > > > > > > >> >>>> yes,
> > > > > > > >> >>>> > >> this
> > > > > > > >> >>>> > >> > > > way I
> > > > > > > >> >>>> > >> > > > > > >> came
> > > > > > > >> >>>> > >> > > > > > >> > to
> > > > > > > >> >>>> > >> > > > > > >> > >> 1-1
> > > > > > > >> >>>> > >> > > > > > >> > >> > > mapping between commands in
> > the
> > > > tool
> > > > > > and
> > > > > > > >> >>>> protocol
> > > > > > > >> >>>> > >> > > > additions.
> > > > > > > >> >>>> > >> > > > > > But
> > > > > > > >> >>>> > >> > > > > > >> I
> > > > > > > >> >>>> > >> > > > > > >> > >> might
> > > > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > > > >> >>>> > >> > > > > > >> > >> > > wrong.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > At the end I just try to start
> > > > > > > discussion
> > > > > > > >> how
> > > > > > > >> >>>> at
> > > > > > > >> >>>> > >> least
> > > > > > > >> >>>> > >> > > > > > generally
> > > > > > > >> >>>> > >> > > > > > >> > this
> > > > > > > >> >>>> > >> > > > > > >> > >> > > protocol should look like.
> > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > Thanks,
> > > > > > > >> >>>> > >> > > > > > >> > >> > > Andrii
> > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10
> > > PM,
> > > > > Jay
> > > > > > > >> Kreps <
> > > > > > > >> >>>> > >> > > > > > jay.kreps@gmail.com
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >> > >> wrote:
> > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > Hey Andrii,
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > To answer your earlier
> > > question
> > > > we
> > > > > > > just
> > > > > > > >> >>>> really
> > > > > > > >> >>>> > >> can't
> > > > > > > >> >>>> > >> > be
> > > > > > > >> >>>> > >> > > > > > adding
> > > > > > > >> >>>> > >> > > > > > >> any
> > > > > > > >> >>>> > >> > > > > > >> > >> more
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > scala protocol objects.
> > These
> > > > > things
> > > > > > > are
> > > > > > > >> >>>> super hard
> > > > > > > >> >>>> > >> > to
> > > > > > > >> >>>> > >> > > > > > maintain
> > > > > > > >> >>>> > >> > > > > > >> > >> because
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > they hand code the byte
> > > parsing
> > > > > and
> > > > > > > >> don't
> > > > > > > >> >>>> have good
> > > > > > > >> >>>> > >> > > > > > versioning
> > > > > > > >> >>>> > >> > > > > > >> > >> support.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > Since we are already
> > planning
> > > on
> > > > > > > >> converting
> > > > > > > >> >>>> we
> > > > > > > >> >>>> > >> > > definitely
> > > > > > > >> >>>> > >> > > > > > don't
> > > > > > > >> >>>> > >> > > > > > >> > >> want to
> > > > > > > >> >>>> > >> > > > > > >> > >> > > add
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > a ton more of these--they
> > are
> > > > > total
> > > > > > > tech
> > > > > > > >> >>>> debt.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > What does it mean that the
> > > > changes
> > > > > > are
> > > > > > > >> >>>> isolated
> > > > > > > >> >>>> > >> from
> > > > > > > >> >>>> > >> > > the
> > > > > > > >> >>>> > >> > > > > > >> current
> > > > > > > >> >>>> > >> > > > > > >> > >> code
> > > > > > > >> >>>> > >> > > > > > >> > >> > > base?
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > I actually didn't understand
> > > the
> > > > > > > >> remaining
> > > > > > > >> >>>> > >> comments,
> > > > > > > >> >>>> > >> > > > which
> > > > > > > >> >>>> > >> > > > > of
> > > > > > > >> >>>> > >> > > > > > >> the
> > > > > > > >> >>>> > >> > > > > > >> > >> > points
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > are you responding to?
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > Maybe one sticking point
> > here
> > > is
> > > > > > that
> > > > > > > it
> > > > > > > >> >>>> seems like
> > > > > > > >> >>>> > >> > you
> > > > > > > >> >>>> > >> > > > > want
> > > > > > > >> >>>> > >> > > > > > to
> > > > > > > >> >>>> > >> > > > > > >> > make
> > > > > > > >> >>>> > >> > > > > > >> > >> > some
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > kind of tool, and you have
> > > made
> > > > a
> > > > > > 1-1
> > > > > > > >> mapping
> > > > > > > >> >>>> > >> between
> > > > > > > >> >>>> > >> > > > > > commands
> > > > > > > >> >>>> > >> > > > > > >> you
> > > > > > > >> >>>> > >> > > > > > >> > >> > > imagine
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > in the tool and protocol
> > > > > additions.
> > > > > > I
> > > > > > > >> want
> > > > > > > >> >>>> to make
> > > > > > > >> >>>> > >> > sure
> > > > > > > >> >>>> > >> > > > we
> > > > > > > >> >>>> > >> > > > > > >> don't
> > > > > > > >> >>>> > >> > > > > > >> > do
> > > > > > > >> >>>> > >> > > > > > >> > >> > that.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > The protocol needs to be
> > > really
> > > > > > really
> > > > > > > >> well
> > > > > > > >> >>>> thought
> > > > > > > >> >>>> > >> > out
> > > > > > > >> >>>> > >> > > > > > against
> > > > > > > >> >>>> > >> > > > > > >> > many
> > > > > > > >> >>>> > >> > > > > > >> > >> > use
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > cases so it should make
> > > perfect
> > > > > > > logical
> > > > > > > >> >>>> sense in
> > > > > > > >> >>>> > >> the
> > > > > > > >> >>>> > >> > > > > absence
> > > > > > > >> >>>> > >> > > > > > of
> > > > > > > >> >>>> > >> > > > > > >> > >> knowing
> > > > > > > >> >>>> > >> > > > > > >> > >> > > the
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > command line tool, right?
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > -Jay
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at
> > 11:57
> > > > AM,
> > > > > > > Andrii
> > > > > > > >> >>>> Biletskyi
> > > > > > > >> >>>> > >> <
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly
> > >
> > > > > wrote:
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Hey Jay,
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > I would like to continue
> > > this
> > > > > > > >> discussion
> > > > > > > >> >>>> as it
> > > > > > > >> >>>> > >> seem
> > > > > > > >> >>>> > >> > > > there
> > > > > > > >> >>>> > >> > > > > > is
> > > > > > > >> >>>> > >> > > > > > >> no
> > > > > > > >> >>>> > >> > > > > > >> > >> > > progress
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > here.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > First of all, could you
> > > please
> > > > > > > explain
> > > > > > > >> >>>> what did
> > > > > > > >> >>>> > >> you
> > > > > > > >> >>>> > >> > > > mean
> > > > > > > >> >>>> > >> > > > > in
> > > > > > > >> >>>> > >> > > > > > >> 2?
> > > > > > > >> >>>> > >> > > > > > >> > How
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > exactly
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > are we going to migrate to
> > > the
> > > > > new
> > > > > > > >> java
> > > > > > > >> >>>> protocol
> > > > > > > >> >>>> > >> > > > > > definitions.
> > > > > > > >> >>>> > >> > > > > > >> > And
> > > > > > > >> >>>> > >> > > > > > >> > >> why
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > it's
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > a blocker for centralized
> > > CLI?
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > I agree with you, this
> > > feature
> > > > > > > >> includes
> > > > > > > >> >>>> lots of
> > > > > > > >> >>>> > >> > > stuff,
> > > > > > > >> >>>> > >> > > > > but
> > > > > > > >> >>>> > >> > > > > > >> > >> thankfully
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > almost all changes are
> > > > isolated
> > > > > > from
> > > > > > > >> the
> > > > > > > >> >>>> current
> > > > > > > >> >>>> > >> > code
> > > > > > > >> >>>> > >> > > > > base,
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > so the main thing, I
> > think,
> > > we
> > > > > > need
> > > > > > > to
> > > > > > > >> >>>> agree is
> > > > > > > >> >>>> > >> > RQ/RP
> > > > > > > >> >>>> > >> > > > > > format.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > So how can we start
> > > discussion
> > > > > > about
> > > > > > > >> the
> > > > > > > >> >>>> concrete
> > > > > > > >> >>>> > >> > > > > messages
> > > > > > > >> >>>> > >> > > > > > >> > format?
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Can we take (
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> >>>> > >> > > > >
> > > > > > > >> >>>> > >> > > >
> > > > > > > >> >>>> > >> > >
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >>
> > > > > > > >> >>>>
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > as starting point?
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > We had some doubts earlier
> > > > > whether
> > > > > > > it
> > > > > > > >> worth
> > > > > > > >> >>>> > >> > > introducing
> > > > > > > >> >>>> > >> > > > > one
> > > > > > > >> >>>> > >> > > > > > >> > >> generic
> > > > > > > >> >>>> > >> > > > > > >> > >> > > Admin
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Request for all commands (
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > but then everybody agreed
> > it
> > > > > would
> > > > > > > be
> > > > > > > >> >>>> better to
> > > > > > > >> >>>> > >> > have
> > > > > > > >> >>>> > >> > > > > > separate
> > > > > > > >> >>>> > >> > > > > > >> > >> message
> > > > > > > >> >>>> > >> > > > > > >> > >> > > for
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > each admin command. The
> > > > Request
> > > > > > part
> > > > > > > >> is
> > > > > > > >> >>>> really
> > > > > > > >> >>>> > >> > > dictated
> > > > > > > >> >>>> > >> > > > > > from
> > > > > > > >> >>>> > >> > > > > > >> the
> > > > > > > >> >>>> > >> > > > > > >> > >> > > command
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand)
> > > arguments
> > > > > > > itself,
> > > > > > > >> so
> > > > > > > >> >>>> the
> > > > > > > >> >>>> > >> > proposed
> > > > > > > >> >>>> > >> > > > > > version
> > > > > > > >> >>>> > >> > > > > > >> > >> should
> > > > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > fine (let's put aside for
> > > now
> > > > > > > remarks
> > > > > > > >> about
> > > > > > > >> >>>> > >> > Optional
> > > > > > > >> >>>> > >> > > > > type,
> > > > > > > >> >>>> > >> > > > > > >> > >> batching,
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > configs normalization - I
> > > > agree
> > > > > > with
> > > > > > > >> all of
> > > > > > > >> >>>> > >> them).
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > So the second part is
> > > > Response.
> > > > > I
> > > > > > > see
> > > > > > > >> >>>> there are
> > > > > > > >> >>>> > >> two
> > > > > > > >> >>>> > >> > > > cases
> > > > > > > >> >>>> > >> > > > > > >> here.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> > > > > > > >> Create/Alter/... ;
> > > > > > > >> >>>> b)
> > > > > > > >> >>>> > >> "Get"
> > > > > > > >> >>>> > >> > > > > > requests -
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > List/Describe...
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > a) should only hold
> > request
> > > > > result
> > > > > > > >> >>>> (regardless
> > > > > > > >> >>>> > >> what
> > > > > > > >> >>>> > >> > > we
> > > > > > > >> >>>> > >> > > > > > decide
> > > > > > > >> >>>> > >> > > > > > >> > >> about
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > blocking/non-blocking
> > > commands
> > > > > > > >> execution).
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Usually we provide error
> > > code
> > > > in
> > > > > > > >> response
> > > > > > > >> >>>> but
> > > > > > > >> >>>> > >> since
> > > > > > > >> >>>> > >> > > we
> > > > > > > >> >>>> > >> > > > > will
> > > > > > > >> >>>> > >> > > > > > >> use
> > > > > > > >> >>>> > >> > > > > > >> > >> this
> > > > > > > >> >>>> > >> > > > > > >> > >> > in
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > interactive shell we need
> > > some
> > > > > > human
> > > > > > > >> >>>> readable
> > > > > > > >> >>>> > >> error
> > > > > > > >> >>>> > >> > > > > > >> description
> > > > > > > >> >>>> > >> > > > > > >> > -
> > > > > > > >> >>>> > >> > > > > > >> > >> so
> > > > > > > >> >>>> > >> > > > > > >> > >> > I
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > added errorDesription
> > field
> > > > > where
> > > > > > > you
> > > > > > > >> can
> > > > > > > >> >>>> at
> > > > > > > >> >>>> > >> least
> > > > > > > >> >>>> > >> > > > leave
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > exception.getMessage.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > b) in addition to previous
> > > > item
> > > > > > > >> message
> > > > > > > >> >>>> should
> > > > > > > >> >>>> > >> hold
> > > > > > > >> >>>> > >> > > > > command
> > > > > > > >> >>>> > >> > > > > > >> > >> specific
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > response data. We can
> > > discuss
> > > > in
> > > > > > > >> detail
> > > > > > > >> >>>> each of
> > > > > > > >> >>>> > >> > them
> > > > > > > >> >>>> > >> > > > but
> > > > > > > >> >>>> > >> > > > > > >> let's
> > > > > > > >> >>>> > >> > > > > > >> > for
> > > > > > > >> >>>> > >> > > > > > >> > >> > now
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > agree about the overall
> > > > pattern.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Thanks,
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at
> > 6:59
> > > > AM,
> > > > > > Jay
> > > > > > > >> Kreps
> > > > > > > >> >>>> <
> > > > > > > >> >>>> > >> > > > > > >> jay.kreps@gmail.com
> > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > wrote:
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > Hey Joe,
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > This is great. A few
> > > > comments
> > > > > on
> > > > > > > >> KIP-4
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 1. This is much needed
> > > > > > > >> functionality,
> > > > > > > >> >>>> but there
> > > > > > > >> >>>> > >> > > are a
> > > > > > > >> >>>> > >> > > > > lot
> > > > > > > >> >>>> > >> > > > > > >> of
> > > > > > > >> >>>> > >> > > > > > >> > >> the so
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > let's
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > really think these
> > > protocols
> > > > > > > >> through. We
> > > > > > > >> >>>> really
> > > > > > > >> >>>> > >> > > want
> > > > > > > >> >>>> > >> > > > to
> > > > > > > >> >>>> > >> > > > > > >> end up
> > > > > > > >> >>>> > >> > > > > > >> > >> > with a
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > set
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > of well thought-out,
> > > > > orthoganol
> > > > > > > >> apis.
> > > > > > > >> >>>> For this
> > > > > > > >> >>>> > >> > > > reason I
> > > > > > > >> >>>> > >> > > > > > >> think
> > > > > > > >> >>>> > >> > > > > > >> > >> it is
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > really
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > important to think
> > through
> > > > the
> > > > > > end
> > > > > > > >> state
> > > > > > > >> >>>> even
> > > > > > > >> >>>> > >> if
> > > > > > > >> >>>> > >> > > that
> > > > > > > >> >>>> > >> > > > > > >> includes
> > > > > > > >> >>>> > >> > > > > > >> > >> APIs
> > > > > > > >> >>>> > >> > > > > > >> > >> > > we
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > won't implement in the
> > > first
> > > > > > > phase.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 2. Let's please please
> > > > please
> > > > > > wait
> > > > > > > >> until
> > > > > > > >> >>>> we
> > > > > > > >> >>>> > >> have
> > > > > > > >> >>>> > >> > > > > switched
> > > > > > > >> >>>> > >> > > > > > >> the
> > > > > > > >> >>>> > >> > > > > > >> > >> > server
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > over
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > to the new java protocol
> > > > > > > >> definitions. If
> > > > > > > >> >>>> we add
> > > > > > > >> >>>> > >> > > > upteen
> > > > > > > >> >>>> > >> > > > > > >> more ad
> > > > > > > >> >>>> > >> > > > > > >> > >> hoc
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > scala
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > objects that is just
> > > > > generating
> > > > > > > more
> > > > > > > >> >>>> work for
> > > > > > > >> >>>> > >> the
> > > > > > > >> >>>> > >> > > > > > >> conversion
> > > > > > > >> >>>> > >> > > > > > >> > we
> > > > > > > >> >>>> > >> > > > > > >> > >> > know
> > > > > > > >> >>>> > >> > > > > > >> > >> > > we
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > have to do.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 3. This proposal
> > > introduces
> > > > a
> > > > > > new
> > > > > > > >> type of
> > > > > > > >> >>>> > >> > optional
> > > > > > > >> >>>> > >> > > > > > >> parameter.
> > > > > > > >> >>>> > >> > > > > > >> > >> This
> > > > > > > >> >>>> > >> > > > > > >> > >> > is
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > inconsistent with
> > > everything
> > > > > > else
> > > > > > > >> in the
> > > > > > > >> >>>> > >> protocol
> > > > > > > >> >>>> > >> > > > where
> > > > > > > >> >>>> > >> > > > > > we
> > > > > > > >> >>>> > >> > > > > > >> use
> > > > > > > >> >>>> > >> > > > > > >> > >> -1
> > > > > > > >> >>>> > >> > > > > > >> > >> > or
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > some
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > other marker value. You
> > > > could
> > > > > > > argue
> > > > > > > >> >>>> either way
> > > > > > > >> >>>> > >> > but
> > > > > > > >> >>>> > >> > > > > let's
> > > > > > > >> >>>> > >> > > > > > >> stick
> > > > > > > >> >>>> > >> > > > > > >> > >> with
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > that
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > for consistency. For
> > > clients
> > > > > > that
> > > > > > > >> >>>> implemented
> > > > > > > >> >>>> > >> the
> > > > > > > >> >>>> > >> > > > > > protocol
> > > > > > > >> >>>> > >> > > > > > >> in
> > > > > > > >> >>>> > >> > > > > > >> > a
> > > > > > > >> >>>> > >> > > > > > >> > >> > > better
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > way
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > than our scala code
> > these
> > > > > basic
> > > > > > > >> >>>> primitives are
> > > > > > > >> >>>> > >> > hard
> > > > > > > >> >>>> > >> > > > to
> > > > > > > >> >>>> > >> > > > > > >> change.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This
> > > > seems
> > > > > > to
> > > > > > > >> >>>> duplicate
> > > > > > > >> >>>> > >> > > > > > >> > TopicMetadataRequest
> > > > > > > >> >>>> > >> > > > > > >> > >> > > which
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > brokers, topics, and
> > > > > > partitions. I
> > > > > > > >> think
> > > > > > > >> >>>> we
> > > > > > > >> >>>> > >> > should
> > > > > > > >> >>>> > >> > > > > rename
> > > > > > > >> >>>> > >> > > > > > >> that
> > > > > > > >> >>>> > >> > > > > > >> > >> > > request
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest
> > (or
> > > > > just
> > > > > > > >> >>>> > >> MetadataRequest)
> > > > > > > >> >>>> > >> > > and
> > > > > > > >> >>>> > >> > > > > > >> include
> > > > > > > >> >>>> > >> > > > > > >> > >> the id
> > > > > > > >> >>>> > >> > > > > > >> > >> > > of
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > the
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > controller. Or are there
> > > > other
> > > > > > > >> things we
> > > > > > > >> >>>> could
> > > > > > > >> >>>> > >> > add
> > > > > > > >> >>>> > >> > > > > here?
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to
> > > try
> > > > > to
> > > > > > > >> make a
> > > > > > > >> >>>> lot of
> > > > > > > >> >>>> > >> > > > requests
> > > > > > > >> >>>> > >> > > > > > that
> > > > > > > >> >>>> > >> > > > > > >> > can
> > > > > > > >> >>>> > >> > > > > > >> > >> > only
> > > > > > > >> >>>> > >> > > > > > >> > >> > > go
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > to
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > particular nodes. This
> > > adds
> > > > a
> > > > > > lot
> > > > > > > of
> > > > > > > >> >>>> burden for
> > > > > > > >> >>>> > >> > > > client
> > > > > > > >> >>>> > >> > > > > > >> > >> > > implementations
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > (it
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > sounds easy but each
> > > > discovery
> > > > > > can
> > > > > > > >> fail
> > > > > > > >> >>>> in many
> > > > > > > >> >>>> > >> > > parts
> > > > > > > >> >>>> > >> > > > > so
> > > > > > > >> >>>> > >> > > > > > it
> > > > > > > >> >>>> > >> > > > > > >> > >> ends up
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > being a
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > full state machine to do
> > > > > > right). I
> > > > > > > >> think
> > > > > > > >> >>>> we
> > > > > > > >> >>>> > >> > should
> > > > > > > >> >>>> > >> > > > > > consider
> > > > > > > >> >>>> > >> > > > > > >> > >> making
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > admin
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > commands and ideally as
> > > many
> > > > > of
> > > > > > > the
> > > > > > > >> >>>> other apis
> > > > > > > >> >>>> > >> as
> > > > > > > >> >>>> > >> > > > > > possible
> > > > > > > >> >>>> > >> > > > > > >> > >> > available
> > > > > > > >> >>>> > >> > > > > > >> > >> > > on
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > all
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > brokers and just
> > redirect
> > > to
> > > > > the
> > > > > > > >> >>>> controller on
> > > > > > > >> >>>> > >> > the
> > > > > > > >> >>>> > >> > > > > broker
> > > > > > > >> >>>> > >> > > > > > >> > side.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > Perhaps
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > there would be a general
> > > way
> > > > > to
> > > > > > > >> >>>> encapsulate
> > > > > > > >> >>>> > >> this
> > > > > > > >> >>>> > >> > > > > > re-routing
> > > > > > > >> >>>> > >> > > > > > >> > >> > behavior.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 6. We should probably
> > > > > normalize
> > > > > > > the
> > > > > > > >> key
> > > > > > > >> >>>> value
> > > > > > > >> >>>> > >> > pairs
> > > > > > > >> >>>> > >> > > > > used
> > > > > > > >> >>>> > >> > > > > > >> for
> > > > > > > >> >>>> > >> > > > > > >> > >> > configs
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > rather
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > than embedding a new
> > > > > formatting.
> > > > > > > So
> > > > > > > >> two
> > > > > > > >> >>>> strings
> > > > > > > >> >>>> > >> > > > rather
> > > > > > > >> >>>> > >> > > > > > than
> > > > > > > >> >>>> > >> > > > > > >> > one
> > > > > > > >> >>>> > >> > > > > > >> > >> > with
> > > > > > > >> >>>> > >> > > > > > >> > >> > > an
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > internal equals sign.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition
> > of
> > > > > these
> > > > > > > >> APIs
> > > > > > > >> >>>> that the
> > > > > > > >> >>>> > >> > > > command
> > > > > > > >> >>>> > >> > > > > > has
> > > > > > > >> >>>> > >> > > > > > >> > >> begun or
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > that
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > the command has been
> > > > > completed?
> > > > > > It
> > > > > > > >> is a
> > > > > > > >> >>>> lot
> > > > > > > >> >>>> > >> more
> > > > > > > >> >>>> > >> > > > usable
> > > > > > > >> >>>> > >> > > > > > if
> > > > > > > >> >>>> > >> > > > > > >> the
> > > > > > > >> >>>> > >> > > > > > >> > >> > > command
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > been completed so you
> > know
> > > > > that
> > > > > > if
> > > > > > > >> you
> > > > > > > >> >>>> create a
> > > > > > > >> >>>> > >> > > topic
> > > > > > > >> >>>> > >> > > > > and
> > > > > > > >> >>>> > >> > > > > > >> then
> > > > > > > >> >>>> > >> > > > > > >> > >> > > publish
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > to
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > it you won't get an
> > > > exception
> > > > > > > about
> > > > > > > >> >>>> there being
> > > > > > > >> >>>> > >> > no
> > > > > > > >> >>>> > >> > > > such
> > > > > > > >> >>>> > >> > > > > > >> topic.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and
> > list
> > > > > > topics
> > > > > > > >> >>>> duplicate a
> > > > > > > >> >>>> > >> lot
> > > > > > > >> >>>> > >> > > of
> > > > > > > >> >>>> > >> > > > > > stuff
> > > > > > > >> >>>> > >> > > > > > >> in
> > > > > > > >> >>>> > >> > > > > > >> > >> the
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > metadata
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > request. Is there a
> > reason
> > > > to
> > > > > > give
> > > > > > > >> back
> > > > > > > >> >>>> topics
> > > > > > > >> >>>> > >> > > marked
> > > > > > > >> >>>> > >> > > > > for
> > > > > > > >> >>>> > >> > > > > > >> > >> > deletion? I
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > feel
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > like if we just make the
> > > > > > > >> post-condition
> > > > > > > >> >>>> of the
> > > > > > > >> >>>> > >> > > delete
> > > > > > > >> >>>> > >> > > > > > >> command
> > > > > > > >> >>>> > >> > > > > > >> > be
> > > > > > > >> >>>> > >> > > > > > >> > >> > that
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > the
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > topic is deleted that
> > will
> > > > get
> > > > > > rid
> > > > > > > >> of
> > > > > > > >> >>>> the need
> > > > > > > >> >>>> > >> > for
> > > > > > > >> >>>> > >> > > > this
> > > > > > > >> >>>> > >> > > > > > >> right?
> > > > > > > >> >>>> > >> > > > > > >> > >> And
> > > > > > > >> >>>> > >> > > > > > >> > >> > it
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > will
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 9. Should we consider
> > > > batching
> > > > > > > these
> > > > > > > >> >>>> requests?
> > > > > > > >> >>>> > >> We
> > > > > > > >> >>>> > >> > > > have
> > > > > > > >> >>>> > >> > > > > > >> > generally
> > > > > > > >> >>>> > >> > > > > > >> > >> > > tried
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > to
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > allow multiple
> > operations
> > > to
> > > > > be
> > > > > > > >> batched.
> > > > > > > >> >>>> My
> > > > > > > >> >>>> > >> > > suspicion
> > > > > > > >> >>>> > >> > > > > is
> > > > > > > >> >>>> > >> > > > > > >> that
> > > > > > > >> >>>> > >> > > > > > >> > >> > without
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > this
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > we will get a lot of
> > code
> > > > that
> > > > > > > does
> > > > > > > >> >>>> something
> > > > > > > >> >>>> > >> > like
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >    for(topic:
> > > > > > > >> adminClient.listTopics())
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >>  adminClient.describeTopic(topic)
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > this code will work
> > great
> > > > when
> > > > > > you
> > > > > > > >> test
> > > > > > > >> >>>> on 5
> > > > > > > >> >>>> > >> > topics
> > > > > > > >> >>>> > >> > > > but
> > > > > > > >> >>>> > >> > > > > > >> not do
> > > > > > > >> >>>> > >> > > > > > >> > >> as
> > > > > > > >> >>>> > >> > > > > > >> > >> > > well
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > if
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > you have 50k.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 10. I think we should
> > also
> > > > > > discuss
> > > > > > > >> how
> > > > > > > >> >>>> we want
> > > > > > > >> >>>> > >> to
> > > > > > > >> >>>> > >> > > > > expose
> > > > > > > >> >>>> > >> > > > > > a
> > > > > > > >> >>>> > >> > > > > > >> > >> > > programmatic
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > JVM
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > client api for these
> > > > > operations.
> > > > > > > >> >>>> Currently
> > > > > > > >> >>>> > >> people
> > > > > > > >> >>>> > >> > > > rely
> > > > > > > >> >>>> > >> > > > > on
> > > > > > > >> >>>> > >> > > > > > >> > >> > AdminUtils
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > which
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I
> > > think
> > > > we
> > > > > > > >> probably
> > > > > > > >> >>>> need
> > > > > > > >> >>>> > >> > > another
> > > > > > > >> >>>> > >> > > > > > client
> > > > > > > >> >>>> > >> > > > > > >> > >> under
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > clients/
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > that exposes
> > > administrative
> > > > > > > >> >>>> functionality. We
> > > > > > > >> >>>> > >> > will
> > > > > > > >> >>>> > >> > > > need
> > > > > > > >> >>>> > >> > > > > > >> this
> > > > > > > >> >>>> > >> > > > > > >> > >> just
> > > > > > > >> >>>> > >> > > > > > >> > >> > to
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > properly test the new
> > > apis,
> > > > I
> > > > > > > >> suspect. We
> > > > > > > >> >>>> > >> should
> > > > > > > >> >>>> > >> > > > figure
> > > > > > > >> >>>> > >> > > > > > out
> > > > > > > >> >>>> > >> > > > > > >> > that
> > > > > > > >> >>>> > >> > > > > > >> > >> > API.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 11. The other
> > information
> > > > that
> > > > > > > >> would be
> > > > > > > >> >>>> really
> > > > > > > >> >>>> > >> > > useful
> > > > > > > >> >>>> > >> > > > > to
> > > > > > > >> >>>> > >> > > > > > >> get
> > > > > > > >> >>>> > >> > > > > > >> > >> would
> > > > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > information about
> > > > > > partitions--how
> > > > > > > >> much
> > > > > > > >> >>>> data is
> > > > > > > >> >>>> > >> in
> > > > > > > >> >>>> > >> > > the
> > > > > > > >> >>>> > >> > > > > > >> > partition,
> > > > > > > >> >>>> > >> > > > > > >> > >> > what
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > are
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > the segment offsets,
> > what
> > > is
> > > > > the
> > > > > > > >> log-end
> > > > > > > >> >>>> offset
> > > > > > > >> >>>> > >> > > (i.e.
> > > > > > > >> >>>> > >> > > > > > last
> > > > > > > >> >>>> > >> > > > > > >> > >> offset),
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > what
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > is
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > the compaction point,
> > > etc. I
> > > > > > think
> > > > > > > >> that
> > > > > > > >> >>>> done
> > > > > > > >> >>>> > >> > right
> > > > > > > >> >>>> > >> > > > this
> > > > > > > >> >>>> > >> > > > > > >> would
> > > > > > > >> >>>> > >> > > > > > >> > be
> > > > > > > >> >>>> > >> > > > > > >> > >> > the
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > successor to the very
> > > > awkward
> > > > > > > >> >>>> OffsetRequest we
> > > > > > > >> >>>> > >> > have
> > > > > > > >> >>>> > >> > > > > > today.
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > -Jay
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at
> > > > 10:27
> > > > > > PM,
> > > > > > > >> Joe
> > > > > > > >> >>>> Stein <
> > > > > > > >> >>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > wrote:
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> >>>> > >> > > > >
> > > > > > > >> >>>> > >> > > >
> > > > > > > >> >>>> > >> > >
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >>
> > > > > > > >> >>>>
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > > JIRA
> > > > > > > >> >>>> > >> > > > >
> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > >> >>>> /*******************************************
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal
> > > > > Consultant
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source
> > > > > Security
> > > > > > > LLC
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > http://www.stealth.ly
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Twitter:
> > > > @allthingshadoop <
> > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > http://www.twitter.com/allthingshadoop
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > >> >>>> ********************************************/
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > >> >>>> > >> > > > > > >> > >> --
> > > > > > > >> >>>> > >> > > > > > >> > >> -- Guozhang
> > > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > > >> >>>> > >> > > > > > >> >
> > > > > > > >> >>>> > >> > > > > > >>
> > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > > >
> > > > > > > >> >>>> > >> > > > > >
> > > > > > > >> >>>> > >> > > > >
> > > > > > > >> >>>> > >> > > >
> > > > > > > >> >>>> > >> > >
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >> > --
> > > > > > > >> >>>> > >> > Jeff Holoman
> > > > > > > >> >>>> > >> > Systems Engineer
> > > > > > > >> >>>> > >> >
> > > > > > > >> >>>> > >>
> > > > > > > >> >>>>
> > > > > > > >> >>>>
> > > > > > > >> >>>
> > > > > > > >> >>>
> > > > > > > >> >>> --
> > > > > > > >> >>> -- Guozhang
> > > > > > > >> >>>
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >> >>
> > > > > > > >> >> --
> > > > > > > >> >> -- Guozhang
> > > > > > > >>
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >


Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Jun,

101. Okay, if you say that such use case is important. I also think
using clientId for these purposes is fine - if we already have this field
as part of all Wire protocol messages, why not use that.
I will update KIP-4 page if nobody has other ideas (which may come up
during the call today).

102.1 Agree, I'll update the KIP accordingly. I think we can add new,
fine-grained error codes if some error code received in specific case
won't give enough context to return a descriptive error message for user.

Look forward to discussing all outstanding issues in detail today during
the call.

Thanks,
Andrii Biletskyi



On Mon, Mar 16, 2015 at 10:59 PM, Jun Rao <ju...@confluent.io> wrote:

> 101. There may be a use case where you only want the topics to be created
> manually by admins. Currently, you can do that by disabling auto topic
> creation and issue topic creation from the TopicCommand. If we disable auto
> topic creation completely on the broker and don't have a way to distinguish
> between topic creation requests from the regular clients and the admin, we
> can't support manual topic creation any more. I was thinking that another
> way of distinguishing the clients making the topic creation requests is
> using clientId. For example, the admin tool can set it to something like
> admin and the broker can treat that clientId specially.
>
> Also, there is a related discussion in KAFKA-2020. Currently, we do the
> following in TopicMetadataResponse:
>
> 1. If leader is not available, we set the partition level error code to
> LeaderNotAvailable.
> 2. If a non-leader replica is not available, we take that replica out of
> the assigned replica list and isr in the response. As an indication for
> doing that, we set the partition level error code to ReplicaNotAvailable.
>
> This has a few problems. First, ReplicaNotAvailable probably shouldn't be
> an error, at least for the normal producer/consumer clients that just want
> to find out the leader. Second, it can happen that both the leader and
> another replica are not available at the same time. There is no error code
> to indicate both. Third, even if a replica is not available, it's still
> useful to return its replica id since some clients (e.g. admin tool) may
> still make use of it.
>
> One way to address this issue is to always return the replica id for
> leader, assigned replicas, and isr regardless of whether the corresponding
> broker is live or not. Since we also return the list of live brokers, the
> client can figure out whether a leader or a replica is live or not and act
> accordingly. This way, we don't need to set the partition level error code
> when the leader or a replica is not available. This doesn't change the wire
> protocol, but does change the semantics. Since we are evolving the protocol
> of TopicMetadataRequest here, we can potentially piggyback the change.
>
> 102.1 For those types of errors due to invalid input, shouldn't we just
> guard it at parameter validation time and throw InvalidArgumentException
> without even sending the request to the broker?
>
> Thanks,
>
> Jun
>
>
> On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Jun,
> >
> > Answering your questions:
> >
> > 101. If I understand you correctly, you are saying future producer
> versions
> > (which
> > will be ported to TMR_V1) won't be able to automatically create topic (if
> > we
> > unconditionally remove topic creation from there). But we need to this
> > preserve logic.
> > Ok, about your proposal: I'm not a big fan too, when it comes to
> > differentiating
> > clients directly in protocol schema. And also I'm not sure I understand
> at
> > all why
> > auto.create.topics.enable is a server side configuration. Can we
> deprecate
> > this setting
> > in future versions, add this setting to producer and based on that upon
> > receiving
> > UnknownTopic create topic explicitly by a separate producer call via
> > adminClient?
> >
> > 102.1. Hm, yes. It's because we want to support batching and at the same
> > time we
> > want to give descriptive error messages for clients. Since AdminClient
> > holds the context
> > to construct such messages (e.g. AdminClient layer can know that
> > InvalidArgumentsCode
> > means two cases: either invalid number - e.g. -1; or replication-factor
> was
> > provided while
> > partitions argument wasn't) - I wrapped responses in Exceptions. But I'm
> > open to any
> > other ideas, this was just initial version.
> > 102.2. Yes, I agree. I'll change that to probably some other dto.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> > On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Andrii,
> > >
> > > 101. That's what I was thinking too, but it may not be that simple. In
> > > TopicMetadataRequest_V1,
> > > we can let it not trigger auto topic creation. Then, in the producer
> > side,
> > > if it gets an UnknownTopicException, it can explicitly issue a
> > > createTopicRequest for auto topic creation. On the consumer side, it
> will
> > > never issue createTopicRequest. This works when auto topic creation is
> > > enabled on the broker side. However, I am not sure how things will work
> > > when auto topic creation is disabled on the broker side. In this case,
> we
> > > want to have a way to manually create a topic, potentially through
> admin
> > > commands. However, then we need a way to distinguish createTopicRequest
> > > issued from the producer clients and the admin tools. May be we can
> add a
> > > new field in createTopicRequest and set it differently in the producer
> > > client and the admin client. However, I am not sure if that's the best
> > > approach.
> > >
> > > 2. Yes, refactoring existing requests is a non-trivial amount of work.
> I
> > > posted some comments in KAFKA-1927. We will probably have to fix
> > KAFKA-1927
> > > first, before adding the new logic in KAFKA-1694. Otherwise, the
> changes
> > > will be too big.
> > >
> > > 102. About the AdminClient:
> > > 102.1. It's a bit weird that we return exception in the api. It seems
> > that
> > > we should either return error code or throw an exception when getting
> the
> > > response state.
> > > 102.2. We probably shouldn't explicitly use the request object in the
> > api.
> > > Not every request evolution requires an api change.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Jun,
> > > >
> > > > Thanks for you comments. Answers inline:
> > > >
> > > > 100. There are a few fields such as ReplicaAssignment,
> > > > > ReassignPartitionRequest,
> > > > > and PartitionsSerialized that are represented as a string, but
> > contain
> > > > > composite structures in json. Could we flatten them out directly in
> > the
> > > > > protocol definition as arrays/records?
> > > >
> > > >
> > > > Yes, now with Admin Client this looks a bit weird. My initial
> > motivation
> > > > was:
> > > > ReassignPartitionCommand accepts input in json, we want to remain
> > tools'
> > > > interfaces unchanged, where possible.
> > > > If we port it to deserialized format, in CLI (/tools project) we will
> > > have
> > > > to add some
> > > > json library since /tools is written in java and we'll need to
> > > deserialize
> > > > json file
> > > > provided by a user. Can we quickly agree on what this library should
> be
> > > > (Jackson, GSON, whatever)?
> > > >
> > > > 101. Does TopicMetadataRequest v1 still trigger auto topic creation?
> > This
> > > > > will be a bit weird now that we have a separate topic creation api.
> > > Have
> > > > > you thought about how the new createTopicRequest and
> > > TopicMetadataRequest
> > > > > v1 will be used in the producer/consumer client, in addition to
> admin
> > > > > tools? For example, ideally, we don't want TopicMetadataRequest
> from
> > > the
> > > > > consumer to trigger auto topic creation.
> > > >
> > > >
> > > > I agree, this strange logic should be fixed. I'm not confident in
> this
> > > > Kafka part so
> > > > correct me if I'm wrong, but it doesn't look like a hard thing to
> do, I
> > > > think we can
> > > > leverage AdminClient for that in Producer and unconditionally remove
> > > topic
> > > > creation from the TopicMetadataRequest_V1.
> > > >
> > > > 2. I think Jay meant getting rid of scala classes
> > > > > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We
> did
> > > > that
> > > > > as a stop-gap thing when adding the new requests for the consumers.
> > > > > However, the long term plan is to get rid of all those and just
> reuse
> > > the
> > > > > java request/response in the client. Since this KIP proposes to
> add a
> > > > > significant number of new requests, perhaps we should bite the
> bullet
> > > to
> > > > > clean up the existing scala requests first before adding new ones?
> > > > >
> > > >
> > > > Yes, looks like I misunderstood the point of ...RequestAndHeader.
> > Okay, I
> > > > will
> > > > rework that. The only thing is that I don't see any example how it
> was
> > > done
> > > > for at
> > > > least one existing protocol message. Thus, as I understand, I have to
> > > think
> > > > how we
> > > > are going to do it.
> > > > Re porting all existing RQ/RP in this patch. Sounds reasonable, but
> if
> > > it's
> > > > an *obligatory*
> > > > requirement to have Admin KIP done, I'm afraid this can be a serious
> > > > blocker for us.
> > > > There are 13 protocol messages and all that would require not only
> unit
> > > > tests but quite
> > > > intensive manual testing, no? I'm afraid I'm not the right guy to
> cover
> > > > pretty much all
> > > > Kafka core internals :). Let me know your thoughts on this item. Btw
> > > there
> > > > is a ticket to
> > > > follow-up this issue (
> https://issues.apache.org/jira/browse/KAFKA-2006
> > ).
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > >
> > > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <ju...@confluent.io> wrote:
> > > >
> > > > > Andrii,
> > > > >
> > > > >
> > > > > A few more comments.
> > > > >
> > > > > 100. There are a few fields such as ReplicaAssignment,
> > > > > ReassignPartitionRequest,
> > > > > and PartitionsSerialized that are represented as a string, but
> > contain
> > > > > composite structures in json. Could we flatten them out directly in
> > the
> > > > > protocol definition as arrays/records?
> > > > >
> > > > > 101. Does TopicMetadataRequest v1 still trigger auto topic
> creation?
> > > This
> > > > > will be a bit weird now that we have a separate topic creation api.
> > > Have
> > > > > you thought about how the new createTopicRequest and
> > > TopicMetadataRequest
> > > > > v1 will be used in the producer/consumer client, in addition to
> admin
> > > > > tools? For example, ideally, we don't want TopicMetadataRequest
> from
> > > the
> > > > > consumer to trigger auto topic creation.
> > > > >
> > > > > 2. I think Jay meant getting rid of scala classes
> > > > > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We
> did
> > > > that
> > > > > as a stop-gap thing when adding the new requests for the consumers.
> > > > > However, the long term plan is to get rid of all those and just
> reuse
> > > the
> > > > > java request/response in the client. Since this KIP proposes to
> add a
> > > > > significant number of new requests, perhaps we should bite the
> bullet
> > > to
> > > > > clean up the existing scala requests first before adding new ones?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Jun
> > > > >
> > > > >
> > > > >
> > > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Hi,
> > > > > >
> > > > > > As said above - I list again all comments from this thread so we
> > > > > > can see what's left and finalize all pending issues.
> > > > > >
> > > > > > Comments from Jay:
> > > > > > 1. This is much needed functionality, but there are a lot of the
> so
> > > > let's
> > > > > > really think these protocols through. We really want to end up
> > with a
> > > > set
> > > > > > of well thought-out, orthoganol apis. For this reason I think it
> is
> > > > > really
> > > > > > important to think through the end state even if that includes
> APIs
> > > we
> > > > > > won't implement in the first phase.
> > > > > >
> > > > > > A: Definitely behind this. Would appreciate if there are concrete
> > > > > comments
> > > > > > how this can be improved.
> > > > > >
> > > > > > 2. Let's please please please wait until we have switched the
> > server
> > > > over
> > > > > > to the new java protocol definitions. If we add upteen more ad
> hoc
> > > > scala
> > > > > > objects that is just generating more work for the conversion we
> > know
> > > we
> > > > > > have to do.
> > > > > >
> > > > > > A: Fixed in the latest patch - removed scala protocol classes.
> > > > > >
> > > > > > 3. This proposal introduces a new type of optional parameter.
> This
> > is
> > > > > > inconsistent with everything else in the protocol where we use -1
> > or
> > > > some
> > > > > > other marker value. You could argue either way but let's stick
> with
> > > > that
> > > > > > for consistency. For clients that implemented the protocol in a
> > > better
> > > > > way
> > > > > > than our scala code these basic primitives are hard to change.
> > > > > >
> > > > > > A: Fixed in the latest patch - removed MaybeOf type and changed
> > > > protocol
> > > > > > accordingly.
> > > > > >
> > > > > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
> > > which
> > > > > has
> > > > > > brokers, topics, and partitions. I think we should rename that
> > > request
> > > > > > ClusterMetadataRequest (or just MetadataRequest) and include the
> id
> > > of
> > > > > the
> > > > > > controller. Or are there other things we could add here?
> > > > > >
> > > > > > A: I agree. Updated the KIP. Let's extends TopicMetadata to
> > version 2
> > > > and
> > > > > > include controller.
> > > > > >
> > > > > > 5. We have a tendency to try to make a lot of requests that can
> > only
> > > go
> > > > > to
> > > > > > particular nodes. This adds a lot of burden for client
> > > implementations
> > > > > (it
> > > > > > sounds easy but each discovery can fail in many parts so it ends
> up
> > > > > being a
> > > > > > full state machine to do right). I think we should consider
> making
> > > > admin
> > > > > > commands and ideally as many of the other apis as possible
> > available
> > > on
> > > > > all
> > > > > > brokers and just redirect to the controller on the broker side.
> > > Perhaps
> > > > > > there would be a general way to encapsulate this re-routing
> > behavior.
> > > > > >
> > > > > > A: It's a very interesting idea, but seems there are some
> concerns
> > > > about
> > > > > > this
> > > > > > feature (like performance considerations, how this will
> complicate
> > > > server
> > > > > > etc).
> > > > > > I believe this shouldn't be a blocker. If this feature is
> > implemented
> > > > at
> > > > > > some
> > > > > > point it won't affect Admin changes - at least no changes to
> public
> > > API
> > > > > > will be required.
> > > > > >
> > > > > > 6. We should probably normalize the key value pairs used for
> > configs
> > > > > rather
> > > > > > than embedding a new formatting. So two strings rather than one
> > with
> > > an
> > > > > > internal equals sign.
> > > > > >
> > > > > > A: Fixed in the latest patch - normalized configs and changed
> > > protocol
> > > > > > accordingly.
> > > > > >
> > > > > > 7. Is the postcondition of these APIs that the command has begun
> or
> > > > that
> > > > > > the command has been completed? It is a lot more usable if the
> > > command
> > > > > has
> > > > > > been completed so you know that if you create a topic and then
> > > publish
> > > > to
> > > > > > it you won't get an exception about there being no such topic.
> > > > > >
> > > > > > A: For long running requests (like reassign partitions) - the
> post
> > > > > > condition is
> > > > > > command has begun - so we don't block the client. In case of your
> > > > > example -
> > > > > > topic commands, this will be refactored and topic commands will
> be
> > > > > executed
> > > > > > immediately, since the Controller will serve Admin requests
> > > > > > (follow-up ticket KAFKA-1777).
> > > > > >
> > > > > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > > > > metadata
> > > > > > request. Is there a reason to give back topics marked for
> > deletion? I
> > > > > feel
> > > > > > like if we just make the post-condition of the delete command be
> > that
> > > > the
> > > > > > topic is deleted that will get rid of the need for this right?
> And
> > it
> > > > > will
> > > > > > be much more intuitive.
> > > > > >
> > > > > > A: Fixed in the latest patch - removed topics marked for deletion
> > in
> > > > > > ListTopicsRequest.
> > > > > >
> > > > > > 9. Should we consider batching these requests? We have generally
> > > tried
> > > > to
> > > > > > allow multiple operations to be batched. My suspicion is that
> > without
> > > > > this
> > > > > > we will get a lot of code that does something like
> > > > > >    for(topic: adminClient.listTopics())
> > > > > >       adminClient.describeTopic(topic)
> > > > > > this code will work great when you test on 5 topics but not do as
> > > well
> > > > if
> > > > > > you have 50k.
> > > > > >
> > > > > > A: Updated the KIP - please check "Topic Admin Schema" section.
> > > > > >
> > > > > > 10. I think we should also discuss how we want to expose a
> > > programmatic
> > > > > JVM
> > > > > > client api for these operations. Currently people rely on
> > AdminUtils
> > > > > which
> > > > > > is totally sketchy. I think we probably need another client under
> > > > > clients/
> > > > > > that exposes administrative functionality. We will need this just
> > to
> > > > > > properly test the new apis, I suspect. We should figure out that
> > API.
> > > > > >
> > > > > > A: Updated the KIP - please check "Admin Client" section with an
> > > > initial
> > > > > > API proposal.
> > > > > >
> > > > > > 11. The other information that would be really useful to get
> would
> > be
> > > > > > information about partitions--how much data is in the partition,
> > what
> > > > are
> > > > > > the segment offsets, what is the log-end offset (i.e. last
> offset),
> > > > what
> > > > > is
> > > > > > the compaction point, etc. I think that done right this would be
> > the
> > > > > > successor to the very awkward OffsetRequest we have today.
> > > > > >
> > > > > > A: I removed ConsumerGroupOffsetsRequest in the latest patch. I
> > > believe
> > > > > > this should
> > > > > > be resolved in a separate KIP / jira ticket.
> > > > > >
> > > > > > 12. Generally we can do good error handling without needing
> custom
> > > > > > server-side
> > > > > > messages. I.e. generally the client has the context to know that
> if
> > > it
> > > > > got
> > > > > > an error that the topic doesn't exist to say "Topic X doesn't
> > exist"
> > > > > rather
> > > > > > than "error code 14" (or whatever). Maybe there are specific
> cases
> > > > where
> > > > > > this is hard? If we want to add server-side error messages we
> > really
> > > do
> > > > > > need to do this in a consistent way across the protocol.
> > > > > >
> > > > > > A: Updated the KIP - please check "Protocol Errors" section. I
> > added
> > > > the
> > > > > > comprehensive, fine-grained list of error codes.
> > > > > >
> > > > > > Comments from Guozhang:
> > > > > > 13. Describe topic request: it would be great to go beyond just
> > > > batching
> > > > > on
> > > > > > topic name regex for this request. For example, a very common use
> > > case
> > > > of
> > > > > > the topic command is to list all topics whose config A's value is
> > B.
> > > > With
> > > > > > topic name regex then we have to first retrieve __all__ topics's
> > > > > > description info and then filter at the client end, which will
> be a
> > > > huge
> > > > > > burden on ZK.
> > > > > > AND
> > > > > > 14. Config K-Vs in create topic: this is related to the previous
> > > point;
> > > > > > maybe we can add another metadata K-V or just a metadata string
> > along
> > > > > side
> > > > > > with config K-V in create topic like we did for offset commit
> > > request.
> > > > > This
> > > > > > field can be quite useful in storing information like "owner" of
> > the
> > > > > topic
> > > > > > who issue the create command, etc, which is quite important for a
> > > > > > multi-tenant setting. Then in the describe topic request we can
> > also
> > > > > batch
> > > > > > on regex of the metadata field.
> > > > > >
> > > > > > A: As discussed it is very interesting but can be implemented
> later
> > > > after
> > > > > > we have some basic functionality there.
> > > > > >
> > > > > > 15. Today all the admin operations are async in the sense that
> > > command
> > > > > will
> > > > > > return once it is written in ZK, and that is why we need extra
> > > > > verification
> > > > > > like testUtil.waitForTopicCreated() / verify partition
> reassignment
> > > > > > request, etc. With admin requests we could add a flag to enable /
> > > > disable
> > > > > > synchronous requests; when it is turned on, the response will not
> > > > return
> > > > > > until the request has been completed. And for async requests we
> can
> > > > add a
> > > > > > "token" field in the response, and then only need a general
> "admin
> > > > > > verification request" with the given token to check if the async
> > > > request
> > > > > > has been completed.
> > > > > >
> > > > > > A: I see your point. My idea was to provide specific
> > Verify...Request
> > > > per
> > > > > > each
> > > > > > long running request, where needed. We can do it the way you
> > suggest.
> > > > The
> > > > > > only
> > > > > > concern is that introducing a token we again will make schema
> > > > "dynamic".
> > > > > We
> > > > > > wanted
> > > > > > to do similar thing introducing single AdminRequest for all topic
> > > > > commands
> > > > > > but rejected
> > > > > > this idea because we wanted to have schema defined. So this is
> > more a
> > > > > > choice between:
> > > > > > a) have fixed schema but introduce each time new Verify...Request
> > for
> > > > > > long-running requests
> > > > > > b) use one request for verification but generalize it with token
> > > > > > I'm fine with whatever decision community come to. Just let me
> know
> > > > your
> > > > > > thoughts.
> > > > > >
> > > > > > Comment from Gwen:
> > > > > > 16. Specifically for ownership, I think the plan is to add ACL
> (it
> > > > sounds
> > > > > > like you are describing ACL) via an external system (Argus,
> > Sentry).
> > > > > > I remember KIP-11 described this, but I can't find the KIP any
> > > longer.
> > > > > >
> > > > > > A: Okay, no problem. Not sure though how we are going to handle
> it.
> > > > Wait
> > > > > > which KIP
> > > > > > will be committed first and include changes to TopicMetadata from
> > the
> > > > > later
> > > > > > one?
> > > > > > Anyway, I added this note to "Open Questions" section so we don't
> > > miss
> > > > > this
> > > > > > piece.
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > > > Today I uploaded the patch that covers some of the discussed
> and
> > > > agreed
> > > > > > > items:
> > > > > > > - removed MaybeOf optional type
> > > > > > > - switched to java protocol definitions
> > > > > > > - simplified messages (normalized configs, removed topic marked
> > for
> > > > > > > deletion)
> > > > > > >
> > > > > > > I also updated the KIP-4 with respective changes and wrote down
> > my
> > > > > > > proposal for
> > > > > > > pending items:
> > > > > > > - Batch Admin Operations -> updated Wire Protocol schema
> proposal
> > > > > > > - Remove ClusterMetadata -> changed to extend
> > TopicMetadataRequest
> > > > > > > - Admin Client -> updated my initial proposal to reflect
> batching
> > > > > > > - Error codes -> proposed fine-grained error code instead of
> > > > > > > AdminRequestFailed
> > > > > > >
> > > > > > > I will also send a separate email to cover all comments from
> this
> > > > > thread.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii Biletskyi
> > > > > > >
> > > > > > >
> > > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> > > gshapira@cloudera.com
> > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > >> Found KIP-11 (
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > > >> )
> > > > > > >> It actually specifies changes to the Metadata protocol, so
> > making
> > > > sure
> > > > > > >> both KIPs are consistent in this regard will be good.
> > > > > > >>
> > > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> > > > gshapira@cloudera.com
> > > > > >
> > > > > > >> wrote:
> > > > > > >> > Specifically for ownership, I think the plan is to add ACL
> (it
> > > > > sounds
> > > > > > >> > like you are describing ACL) via an external system (Argus,
> > > > Sentry).
> > > > > > >> > I remember KIP-11 described this, but I can't find the KIP
> any
> > > > > longer.
> > > > > > >> >
> > > > > > >> > Regardless, I think KIP-4 focuses on getting information
> that
> > > > > already
> > > > > > >> > exists from Kafka brokers, not on adding information that
> > > perhaps
> > > > > > >> > should exist but doesn't yet?
> > > > > > >> >
> > > > > > >> > Gwen
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > > > >> wrote:
> > > > > > >> >> Folks,
> > > > > > >> >>
> > > > > > >> >> Just want to elaborate a bit more on the create-topic
> > metadata
> > > > and
> > > > > > >> batching
> > > > > > >> >> describe-topic based on config / metadata in my previous
> > email
> > > as
> > > > > we
> > > > > > >> work
> > > > > > >> >> on KAFKA-1694. The main motivation is to have some sort of
> > > topic
> > > > > > >> management
> > > > > > >> >> mechanisms, which I think is quite important in a
> > multi-tenant
> > > /
> > > > > > cloud
> > > > > > >> >> architecture: today anyone can create topics in a shared
> > Kafka
> > > > > > >> cluster, but
> > > > > > >> >> there is no concept or "ownership" of topics that are
> created
> > > by
> > > > > > >> different
> > > > > > >> >> users. For example, at LinkedIn we basically distinguish
> > topic
> > > > > owners
> > > > > > >> via
> > > > > > >> >> some casual topic name prefix, which is a bit awkward and
> > does
> > > > not
> > > > > > fly
> > > > > > >> as
> > > > > > >> >> we scale our customers. It would be great to use
> > > describe-topics
> > > > > such
> > > > > > >> as:
> > > > > > >> >>
> > > > > > >> >> Describe all topics that is created by me.
> > > > > > >> >>
> > > > > > >> >> Describe all topics whose retention time is overriden to X.
> > > > > > >> >>
> > > > > > >> >> Describe all topics whose writable group include user Y
> (this
> > > is
> > > > > > >> related to
> > > > > > >> >> authorization), etc..
> > > > > > >> >>
> > > > > > >> >> One possible way to achieve this is to add a metadata file
> in
> > > the
> > > > > > >> >> create-topic request, whose value will also be written ZK
> as
> > we
> > > > > > create
> > > > > > >> the
> > > > > > >> >> topic; then describe-topics can choose to batch topics
> based
> > on
> > > > 1)
> > > > > > name
> > > > > > >> >> regex, 2) config K-V matching, 3) metadata regex, etc.
> > > > > > >> >>
> > > > > > >> >> Thoughts?
> > > > > > >> >>
> > > > > > >> >> Guozhang
> > > > > > >> >>
> > > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > > > >> wrote:
> > > > > > >> >>
> > > > > > >> >>> Thanks for the updated wiki. A few comments below:
> > > > > > >> >>>
> > > > > > >> >>> 1. Error description in response: I think if some
> errorCode
> > > > could
> > > > > > >> indicate
> > > > > > >> >>> several different error cases then we should really change
> > it
> > > to
> > > > > > >> multiple
> > > > > > >> >>> codes. In general the errorCode itself would be precise
> and
> > > > > > >> sufficient for
> > > > > > >> >>> describing the server side errors.
> > > > > > >> >>>
> > > > > > >> >>> 2. Describe topic request: it would be great to go beyond
> > just
> > > > > > >> batching on
> > > > > > >> >>> topic name regex for this request. For example, a very
> > common
> > > > use
> > > > > > >> case of
> > > > > > >> >>> the topic command is to list all topics whose config A's
> > value
> > > > is
> > > > > B.
> > > > > > >> With
> > > > > > >> >>> topic name regex then we have to first retrieve __all__
> > > topics's
> > > > > > >> >>> description info and then filter at the client end, which
> > will
> > > > be
> > > > > a
> > > > > > >> huge
> > > > > > >> >>> burden on ZK.
> > > > > > >> >>>
> > > > > > >> >>> 3. Config K-Vs in create topic: this is related to the
> > > previous
> > > > > > point;
> > > > > > >> >>> maybe we can add another metadata K-V or just a metadata
> > > string
> > > > > > along
> > > > > > >> side
> > > > > > >> >>> with config K-V in create topic like we did for offset
> > commit
> > > > > > >> request. This
> > > > > > >> >>> field can be quite useful in storing information like
> > "owner"
> > > of
> > > > > the
> > > > > > >> topic
> > > > > > >> >>> who issue the create command, etc, which is quite
> important
> > > for
> > > > a
> > > > > > >> >>> multi-tenant setting. Then in the describe topic request
> we
> > > can
> > > > > also
> > > > > > >> batch
> > > > > > >> >>> on regex of the metadata field.
> > > > > > >> >>>
> > > > > > >> >>> 4. Today all the admin operations are async in the sense
> > that
> > > > > > command
> > > > > > >> will
> > > > > > >> >>> return once it is written in ZK, and that is why we need
> > extra
> > > > > > >> verification
> > > > > > >> >>> like testUtil.waitForTopicCreated() / verify partition
> > > > > reassignment
> > > > > > >> >>> request, etc. With admin requests we could add a flag to
> > > enable
> > > > /
> > > > > > >> disable
> > > > > > >> >>> synchronous requests; when it is turned on, the response
> > will
> > > > not
> > > > > > >> return
> > > > > > >> >>> until the request has been completed. And for async
> requests
> > > we
> > > > > can
> > > > > > >> add a
> > > > > > >> >>> "token" field in the response, and then only need a
> general
> > > > "admin
> > > > > > >> >>> verification request" with the given token to check if the
> > > async
> > > > > > >> request
> > > > > > >> >>> has been completed.
> > > > > > >> >>>
> > > > > > >> >>> 5. +1 for extending Metadata request to include
> controller /
> > > > > > >> coordinator
> > > > > > >> >>> information, and then we can remove the ConsumerMetadata /
> > > > > > >> ClusterMetadata
> > > > > > >> >>> requests.
> > > > > > >> >>>
> > > > > > >> >>> Guozhang
> > > > > > >> >>>
> > > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> > > > jjkoshy.w@gmail.com>
> > > > > > >> wrote:
> > > > > > >> >>>
> > > > > > >> >>>> Thanks for sending that out Joe - I don't think I will be
> > > able
> > > > to
> > > > > > >> make
> > > > > > >> >>>> it today, so if notes can be sent out afterward that
> would
> > be
> > > > > > great.
> > > > > > >> >>>>
> > > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira
> > wrote:
> > > > > > >> >>>> > Thanks for sending this out Joe. Looking forward to
> > > chatting
> > > > > with
> > > > > > >> >>>> everyone :)
> > > > > > >> >>>> >
> > > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> > > > > joe.stein@stealth.ly>
> > > > > > >> wrote:
> > > > > > >> >>>> > > Hey, I just sent out a google hangout invite to all
> > pmc,
> > > > > > >> committers
> > > > > > >> >>>> and
> > > > > > >> >>>> > > everyone I found working on a KIP. If I missed anyone
> > in
> > > > the
> > > > > > >> invite
> > > > > > >> >>>> please
> > > > > > >> >>>> > > let me know and can update it, np.
> > > > > > >> >>>> > >
> > > > > > >> >>>> > > We should do this every Tuesday @ 2pm Eastern Time.
> > Maybe
> > > > we
> > > > > > can
> > > > > > >> get
> > > > > > >> >>>> INFRA
> > > > > > >> >>>> > > help to make a google account so we can manage
> better?
> > > > > > >> >>>> > >
> > > > > > >> >>>> > > To discuss
> > > > > > >> >>>> > >
> > > > > > >> >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > > >> >>>> > > in progress and related JIRA that are interdependent
> > and
> > > > > common
> > > > > > >> work.
> > > > > > >> >>>> > >
> > > > > > >> >>>> > > ~ Joe Stein
> > > > > > >> >>>> > >
> > > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
> > > > > > jay.kreps@gmail.com>
> > > > > > >> >>>> wrote:
> > > > > > >> >>>> > >
> > > > > > >> >>>> > >> Let's stay on Google hangouts that will also record
> > and
> > > > make
> > > > > > the
> > > > > > >> >>>> sessions
> > > > > > >> >>>> > >> available on youtube.
> > > > > > >> >>>> > >>
> > > > > > >> >>>> > >> -Jay
> > > > > > >> >>>> > >>
> > > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> > > > > > >> >>>> jholoman@cloudera.com>
> > > > > > >> >>>> > >> wrote:
> > > > > > >> >>>> > >>
> > > > > > >> >>>> > >> > Jay / Joe
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >> > We're happy to send out a Webex for this purpose.
> We
> > > > could
> > > > > > >> record
> > > > > > >> >>>> the
> > > > > > >> >>>> > >> > sessions if there is interest and publish them
> out.
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >> > Thanks
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >> > Jeff
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> > > > > > >> jay.kreps@gmail.com>
> > > > > > >> >>>> wrote:
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >> > > Let's try to get the technical hang-ups sorted
> > out,
> > > > > > though.
> > > > > > >> I
> > > > > > >> >>>> really
> > > > > > >> >>>> > >> > think
> > > > > > >> >>>> > >> > > there is some benefit to live discussion vs
> > > writing. I
> > > > > am
> > > > > > >> >>>> hopeful that
> > > > > > >> >>>> > >> if
> > > > > > >> >>>> > >> > > we post instructions and give ourselves a few
> > > attempts
> > > > > we
> > > > > > >> can
> > > > > > >> >>>> get it
> > > > > > >> >>>> > >> > > working.
> > > > > > >> >>>> > >> > >
> > > > > > >> >>>> > >> > > Tuesday at that time would work for me...any
> > > > objections?
> > > > > > >> >>>> > >> > >
> > > > > > >> >>>> > >> > > -Jay
> > > > > > >> >>>> > >> > >
> > > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> > > > > > >> joe.stein@stealth.ly
> > > > > > >> >>>> >
> > > > > > >> >>>> > >> wrote:
> > > > > > >> >>>> > >> > >
> > > > > > >> >>>> > >> > > > Weekly would be great maybe like every
> Tuesday ~
> > > 1pm
> > > > > ET
> > > > > > /
> > > > > > >> 10am
> > > > > > >> >>>> PT
> > > > > > >> >>>> > >> ????
> > > > > > >> >>>> > >> > > >
> > > > > > >> >>>> > >> > > > I don't mind google hangout but there is
> always
> > > some
> > > > > > >> issue or
> > > > > > >> >>>> > >> whatever
> > > > > > >> >>>> > >> > so
> > > > > > >> >>>> > >> > > > we know the apache irc channel works. We can
> > start
> > > > > there
> > > > > > >> and
> > > > > > >> >>>> see how
> > > > > > >> >>>> > >> it
> > > > > > >> >>>> > >> > > > goes? We can pull transcripts too and
> associate
> > to
> > > > > > >> tickets if
> > > > > > >> >>>> need be
> > > > > > >> >>>> > >> > > makes
> > > > > > >> >>>> > >> > > > it helpful for things.
> > > > > > >> >>>> > >> > > >
> > > > > > >> >>>> > >> > > > ~ Joestein
> > > > > > >> >>>> > >> > > >
> > > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> > > > > > >> >>>> jay.kreps@gmail.com>
> > > > > > >> >>>> > >> > wrote:
> > > > > > >> >>>> > >> > > >
> > > > > > >> >>>> > >> > > > > We'd talked about doing a Google Hangout to
> > chat
> > > > > about
> > > > > > >> this.
> > > > > > >> >>>> What
> > > > > > >> >>>> > >> > about
> > > > > > >> >>>> > >> > > > > generalizing that a little further...I
> > actually
> > > > > think
> > > > > > it
> > > > > > >> >>>> would be
> > > > > > >> >>>> > >> > good
> > > > > > >> >>>> > >> > > > for
> > > > > > >> >>>> > >> > > > > everyone spending a reasonable chunk of
> their
> > > week
> > > > > on
> > > > > > >> Kafka
> > > > > > >> >>>> stuff
> > > > > > >> >>>> > >> to
> > > > > > >> >>>> > >> > > > maybe
> > > > > > >> >>>> > >> > > > > sync up once a week. I think we could use
> time
> > > to
> > > > > talk
> > > > > > >> >>>> through
> > > > > > >> >>>> > >> design
> > > > > > >> >>>> > >> > > > > stuff, make sure we are on top of code
> > reviews,
> > > > talk
> > > > > > >> through
> > > > > > >> >>>> any
> > > > > > >> >>>> > >> > tricky
> > > > > > >> >>>> > >> > > > > issues, etc.
> > > > > > >> >>>> > >> > > > >
> > > > > > >> >>>> > >> > > > > We can make it publicly available so that
> any
> > > one
> > > > > can
> > > > > > >> follow
> > > > > > >> >>>> along
> > > > > > >> >>>> > >> > who
> > > > > > >> >>>> > >> > > > > likes.
> > > > > > >> >>>> > >> > > > >
> > > > > > >> >>>> > >> > > > > Any interest in doing this? If so I'll try
> to
> > > set
> > > > it
> > > > > > up
> > > > > > >> >>>> starting
> > > > > > >> >>>> > >> next
> > > > > > >> >>>> > >> > > > week.
> > > > > > >> >>>> > >> > > > >
> > > > > > >> >>>> > >> > > > > -Jay
> > > > > > >> >>>> > >> > > > >
> > > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii
> > > Biletskyi
> > > > <
> > > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >> >>>> > >> > > > >
> > > > > > >> >>>> > >> > > > > > Hi all,
> > > > > > >> >>>> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed / aligned
> > > document
> > > > > > >> structure.
> > > > > > >> >>>> Also I
> > > > > > >> >>>> > >> > > added
> > > > > > >> >>>> > >> > > > > > some
> > > > > > >> >>>> > >> > > > > > very initial proposal for AdminClient so
> we
> > > have
> > > > > > >> something
> > > > > > >> >>>> to
> > > > > > >> >>>> > >> start
> > > > > > >> >>>> > >> > > > from
> > > > > > >> >>>> > >> > > > > > while
> > > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > > > >> >>>> > >> > > > > >
> > > > > > >> >>>> > >> > > > > >
> > > > > > >> >>>> > >> > > > > >
> > > > > > >> >>>> > >> > > > >
> > > > > > >> >>>> > >> > > >
> > > > > > >> >>>> > >> > >
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >>
> > > > > > >> >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > >> >>>> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > Thanks,
> > > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > > > >> >>>> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii
> > > > Biletskyi
> > > > > <
> > > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >> >>>> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > > Jay,
> > > > > > >> >>>> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > > > Re error messages: you are right, in
> most
> > > > cases
> > > > > > >> client
> > > > > > >> >>>> will
> > > > > > >> >>>> > >> have
> > > > > > >> >>>> > >> > > > enough
> > > > > > >> >>>> > >> > > > > > > context to show descriptive error
> message.
> > > My
> > > > > > >> concern is
> > > > > > >> >>>> that
> > > > > > >> >>>> > >> we
> > > > > > >> >>>> > >> > > will
> > > > > > >> >>>> > >> > > > > > have
> > > > > > >> >>>> > >> > > > > > > to
> > > > > > >> >>>> > >> > > > > > > add lots of new error codes for each
> > > possible
> > > > > > >> error. Of
> > > > > > >> >>>> course,
> > > > > > >> >>>> > >> > we
> > > > > > >> >>>> > >> > > > > could
> > > > > > >> >>>> > >> > > > > > > reuse
> > > > > > >> >>>> > >> > > > > > > some of existing like
> > > > > UknownTopicOrPartitionCode,
> > > > > > >> but we
> > > > > > >> >>>> will
> > > > > > >> >>>> > >> > also
> > > > > > >> >>>> > >> > > > need
> > > > > > >> >>>> > >> > > > > > to
> > > > > > >> >>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> > > > > > >> >>>> TopicConfigInvalid (both
> > > > > > >> >>>> > >> > for
> > > > > > >> >>>> > >> > > > > topic
> > > > > > >> >>>> > >> > > > > > > name and config, and probably user would
> > > like
> > > > to
> > > > > > >> know
> > > > > > >> >>>> what
> > > > > > >> >>>> > >> > exactly
> > > > > > >> >>>> > >> > > > > > > is wrong in his config),
> > > > > InvalidReplicaAssignment,
> > > > > > >> >>>> > >> InternalError
> > > > > > >> >>>> > >> > > > (e.g.
> > > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > > > > >> >>>> > >> > > > > > > And this is only for TopicCommand, we
> will
> > > > also
> > > > > > >> need to
> > > > > > >> >>>> add
> > > > > > >> >>>> > >> > similar
> > > > > > >> >>>> > >> > > > > stuff
> > > > > > >> >>>> > >> > > > > > > for
> > > > > > >> >>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So
> > > we'll
> > > > > end
> > > > > > >> up
> > > > > > >> >>>> with a
> > > > > > >> >>>> > >> > large
> > > > > > >> >>>> > >> > > > list
> > > > > > >> >>>> > >> > > > > > of
> > > > > > >> >>>> > >> > > > > > > error codes, used only in Admin
> protocol.
> > > > > > >> >>>> > >> > > > > > > Having said that, I agree my proposal is
> > not
> > > > > > >> consistent
> > > > > > >> >>>> with
> > > > > > >> >>>> > >> > other
> > > > > > >> >>>> > >> > > > > cases.
> > > > > > >> >>>> > >> > > > > > > Maybe we can find better solution or
> > > something
> > > > > > >> >>>> in-between.
> > > > > > >> >>>> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it is a great
> > idea.
> > > > > This
> > > > > > >> way we
> > > > > > >> >>>> can
> > > > > > >> >>>> > >> move
> > > > > > >> >>>> > >> > > on
> > > > > > >> >>>> > >> > > > > > > faster.
> > > > > > >> >>>> > >> > > > > > > Let's agree somehow on date/time so
> people
> > > can
> > > > > > join.
> > > > > > >> >>>> Will work
> > > > > > >> >>>> > >> > for
> > > > > > >> >>>> > >> > > me
> > > > > > >> >>>> > >> > > > > > this
> > > > > > >> >>>> > >> > > > > > > and
> > > > > > >> >>>> > >> > > > > > > next week almost anytime if agreed in
> > > advance.
> > > > > > >> >>>> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > > > Thanks,
> > > > > > >> >>>> > >> > > > > > > Andrii
> > > > > > >> >>>> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay
> > Kreps <
> > > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > > > >> >>>> > >> > > > > wrote:
> > > > > > >> >>>> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> >>>> > >> > > > > > >> Generally we can do good error handling
> > > > without
> > > > > > >> needing
> > > > > > >> >>>> custom
> > > > > > >> >>>> > >> > > > > > server-side
> > > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the client has
> > the
> > > > > > >> context to
> > > > > > >> >>>> know
> > > > > > >> >>>> > >> that
> > > > > > >> >>>> > >> > > if
> > > > > > >> >>>> > >> > > > it
> > > > > > >> >>>> > >> > > > > > got
> > > > > > >> >>>> > >> > > > > > >> an error that the topic doesn't exist
> to
> > > say
> > > > > > >> "Topic X
> > > > > > >> >>>> doesn't
> > > > > > >> >>>> > >> > > exist"
> > > > > > >> >>>> > >> > > > > > >> rather
> > > > > > >> >>>> > >> > > > > > >> than "error code 14" (or whatever).
> Maybe
> > > > there
> > > > > > are
> > > > > > >> >>>> specific
> > > > > > >> >>>> > >> > cases
> > > > > > >> >>>> > >> > > > > where
> > > > > > >> >>>> > >> > > > > > >> this is hard? If we want to add
> > server-side
> > > > > error
> > > > > > >> >>>> messages we
> > > > > > >> >>>> > >> > > really
> > > > > > >> >>>> > >> > > > > do
> > > > > > >> >>>> > >> > > > > > >> need to do this in a consistent way
> > across
> > > > the
> > > > > > >> protocol.
> > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> >>>> > >> > > > > > >> I still have a bunch of open questions
> > here
> > > > > from
> > > > > > my
> > > > > > >> >>>> previous
> > > > > > >> >>>> > >> > > list. I
> > > > > > >> >>>> > >> > > > > > will
> > > > > > >> >>>> > >> > > > > > >> be out for the next few days for Strata
> > > > though.
> > > > > > >> Maybe
> > > > > > >> >>>> we could
> > > > > > >> >>>> > >> > do
> > > > > > >> >>>> > >> > > a
> > > > > > >> >>>> > >> > > > > > Google
> > > > > > >> >>>> > >> > > > > > >> Hangout chat on any open issues some
> time
> > > > > towards
> > > > > > >> the
> > > > > > >> >>>> end of
> > > > > > >> >>>> > >> > next
> > > > > > >> >>>> > >> > > > week
> > > > > > >> >>>> > >> > > > > > for
> > > > > > >> >>>> > >> > > > > > >> anyone interested in this ticket? I
> have
> > a
> > > > > > feeling
> > > > > > >> that
> > > > > > >> >>>> might
> > > > > > >> >>>> > >> > > > progress
> > > > > > >> >>>> > >> > > > > > >> things a little faster than email--I
> > think
> > > we
> > > > > > >> could talk
> > > > > > >> >>>> > >> through
> > > > > > >> >>>> > >> > > > those
> > > > > > >> >>>> > >> > > > > > >> issues I brought up fairly quickly...
> > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> >>>> > >> > > > > > >> -Jay
> > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii
> > > > > > Biletskyi <
> > > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >> > I'm trying to address some of the
> > issues
> > > > > which
> > > > > > >> were
> > > > > > >> >>>> > >> mentioned
> > > > > > >> >>>> > >> > > > > earlier
> > > > > > >> >>>> > >> > > > > > >> about
> > > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of those was
> > > about
> > > > > > >> batching
> > > > > > >> >>>> > >> > operations.
> > > > > > >> >>>> > >> > > > What
> > > > > > >> >>>> > >> > > > > > if
> > > > > > >> >>>> > >> > > > > > >> we
> > > > > > >> >>>> > >> > > > > > >> > follow TopicCommand approach and let
> > > people
> > > > > > >> specify
> > > > > > >> >>>> > >> topic-name
> > > > > > >> >>>> > >> > > by
> > > > > > >> >>>> > >> > > > > > >> regexp -
> > > > > > >> >>>> > >> > > > > > >> > would that cover most of the use
> cases?
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >> > Secondly, is what information should
> we
> > > > > > generally
> > > > > > >> >>>> provide in
> > > > > > >> >>>> > >> > > Admin
> > > > > > >> >>>> > >> > > > > > >> > responses.
> > > > > > >> >>>> > >> > > > > > >> > I realize that Admin commands don't
> > imply
> > > > > they
> > > > > > >> will
> > > > > > >> >>>> be used
> > > > > > >> >>>> > >> > only
> > > > > > >> >>>> > >> > > > in
> > > > > > >> >>>> > >> > > > > > CLI
> > > > > > >> >>>> > >> > > > > > >> > but,
> > > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a very
> important
> > > > > client
> > > > > > >> of this
> > > > > > >> >>>> > >> > feature.
> > > > > > >> >>>> > >> > > In
> > > > > > >> >>>> > >> > > > > > this
> > > > > > >> >>>> > >> > > > > > >> > case,
> > > > > > >> >>>> > >> > > > > > >> > seems logical, we would like to
> provide
> > > > users
> > > > > > >> with
> > > > > > >> >>>> rich
> > > > > > >> >>>> > >> > > experience
> > > > > > >> >>>> > >> > > > > in
> > > > > > >> >>>> > >> > > > > > >> terms
> > > > > > >> >>>> > >> > > > > > >> > of
> > > > > > >> >>>> > >> > > > > > >> > getting results / errors of the
> > executed
> > > > > > >> commands.
> > > > > > >> >>>> Usually
> > > > > > >> >>>> > >> we
> > > > > > >> >>>> > >> > > > supply
> > > > > > >> >>>> > >> > > > > > >> with
> > > > > > >> >>>> > >> > > > > > >> > responses only errorCode, which looks
> > > very
> > > > > > >> limiting,
> > > > > > >> >>>> in case
> > > > > > >> >>>> > >> > of
> > > > > > >> >>>> > >> > > > CLI
> > > > > > >> >>>> > >> > > > > we
> > > > > > >> >>>> > >> > > > > > >> may
> > > > > > >> >>>> > >> > > > > > >> > want to print human readable error
> > > > > description.
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >> > So, taking into account previous item
> > > about
> > > > > > >> batching,
> > > > > > >> >>>> what
> > > > > > >> >>>> > >> do
> > > > > > >> >>>> > >> > > you
> > > > > > >> >>>> > >> > > > > > think
> > > > > > >> >>>> > >> > > > > > >> > about
> > > > > > >> >>>> > >> > > > > > >> > having smth like:
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support regexp)
> > > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName
> > > Partitions
> > > > > > >> Replicas
> > > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > > > > >> >>>> > >> > > > > > >> > [Config]
> > > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode
> > > > > > ErrorDescription
> > > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > > > > >> >>>> > >> > > > > > >> >   ErrorDescription => string (empty
> if
> > > > > > >> successful)
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp
> > > > > Partitions
> > > > > > >> >>>> > >> > > ReplicaAssignment
> > > > > > >> >>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> > > > > > >> >>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName
> > > ErrorCode
> > > > > > >> >>>> ErrorDescription]
> > > > > > >> >>>> > >> > > > > > >> > CommandErrorCode
> > CommandErrorDescription
> > > > > > >> >>>> > >> > > > > > >> >   CommandErrorCode => int16
> > > > > > >> >>>> > >> > > > > > >> >   CommandErrorDescription => string
> > > > (nonempty
> > > > > > in
> > > > > > >> case
> > > > > > >> >>>> of
> > > > > > >> >>>> > >> fatal
> > > > > > >> >>>> > >> > > > > error,
> > > > > > >> >>>> > >> > > > > > >> e.g.
> > > > > > >> >>>> > >> > > > > > >> > we couldn't get topics by regexp)
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >> > DescribeTopicRequest ->
> TopicNameRegexp
> > > > > > >> >>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> > > > > > >> TopicDescription
> > > > > > >> >>>> > >> ErrorCode
> > > > > > >> >>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> > > > > > >> >>>> CommandErrorDescription
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >> > Also, any thoughts about our
> discussion
> > > > > > regarding
> > > > > > >> >>>> re-routing
> > > > > > >> >>>> > >> > > > > facility?
> > > > > > >> >>>> > >> > > > > > >> In
> > > > > > >> >>>> > >> > > > > > >> > my
> > > > > > >> >>>> > >> > > > > > >> > understanding, it is like between
> > > > augmenting
> > > > > > >> >>>> > >> > > TopicMetadataRequest
> > > > > > >> >>>> > >> > > > > > >> > (to include at least controllerId)
> and
> > > > > > >> implementing
> > > > > > >> >>>> new
> > > > > > >> >>>> > >> > generic
> > > > > > >> >>>> > >> > > > > > >> re-routing
> > > > > > >> >>>> > >> > > > > > >> > facility so sending messages to
> > > controller
> > > > > will
> > > > > > >> be
> > > > > > >> >>>> handled
> > > > > > >> >>>> > >> by
> > > > > > >> >>>> > >> > > it.
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >> > Thanks,
> > > > > > >> >>>> > >> > > > > > >> > Andrii Biletskyi
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM,
> Andrii
> > > > > > >> Biletskyi <
> > > > > > >> >>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >> > > @Guozhang:
> > > > > > >> >>>> > >> > > > > > >> > > Thanks for your comments, I've
> > answered
> > > > > some
> > > > > > of
> > > > > > >> >>>> those. The
> > > > > > >> >>>> > >> > > main
> > > > > > >> >>>> > >> > > > > > thing
> > > > > > >> >>>> > >> > > > > > >> is
> > > > > > >> >>>> > >> > > > > > >> > > having merged request for
> > > > > > >> >>>> create-alter-delete-describe - I
> > > > > > >> >>>> > >> > > have
> > > > > > >> >>>> > >> > > > > some
> > > > > > >> >>>> > >> > > > > > >> > > concerns about this approach.
> > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > >> >>>> > >> > > > > > >> > > @*Jay*:
> > > > > > >> >>>> > >> > > > > > >> > > I see that introduced
> > > > ClusterMetadaRequest
> > > > > is
> > > > > > >> also
> > > > > > >> >>>> one of
> > > > > > >> >>>> > >> > the
> > > > > > >> >>>> > >> > > > > > >> concerns.
> > > > > > >> >>>> > >> > > > > > >> > We
> > > > > > >> >>>> > >> > > > > > >> > > can solve it if we implement
> > re-routing
> > > > > > >> facility.
> > > > > > >> >>>> But I
> > > > > > >> >>>> > >> > agree
> > > > > > >> >>>> > >> > > > with
> > > > > > >> >>>> > >> > > > > > >> > > Guozhang - it will make clients'
> > > > internals
> > > > > a
> > > > > > >> little
> > > > > > >> >>>> bit
> > > > > > >> >>>> > >> > easier
> > > > > > >> >>>> > >> > > > but
> > > > > > >> >>>> > >> > > > > > >> this
> > > > > > >> >>>> > >> > > > > > >> > > seems to be a complex logic to
> > > implement
> > > > > and
> > > > > > >> >>>> support then.
> > > > > > >> >>>> > >> > > > > > Especially
> > > > > > >> >>>> > >> > > > > > >> for
> > > > > > >> >>>> > >> > > > > > >> > > Fetch and Produce (even if we add
> > > > > re-routing
> > > > > > >> later
> > > > > > >> >>>> for
> > > > > > >> >>>> > >> these
> > > > > > >> >>>> > >> > > > > > >> requests).
> > > > > > >> >>>> > >> > > > > > >> > > Also people will tend to avoid this
> > > > > > re-routing
> > > > > > >> >>>> facility
> > > > > > >> >>>> > >> and
> > > > > > >> >>>> > >> > > hold
> > > > > > >> >>>> > >> > > > > > local
> > > > > > >> >>>> > >> > > > > > >> > > cluster cache to ensure their
> > > > high-priority
> > > > > > >> requests
> > > > > > >> >>>> > >> (which
> > > > > > >> >>>> > >> > > some
> > > > > > >> >>>> > >> > > > > of
> > > > > > >> >>>> > >> > > > > > >> the
> > > > > > >> >>>> > >> > > > > > >> > > admin requests are) not sent to
> some
> > > busy
> > > > > > >> broker
> > > > > > >> >>>> where
> > > > > > >> >>>> > >> they
> > > > > > >> >>>> > >> > > wait
> > > > > > >> >>>> > >> > > > > to
> > > > > > >> >>>> > >> > > > > > be
> > > > > > >> >>>> > >> > > > > > >> > > routed to the correct one.
> > > > > > >> >>>> > >> > > > > > >> > > As pointed out by Jun here (
> > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> >>>> > >> > > > > >
> > > > > > >> >>>> > >> > > > >
> > > > > > >> >>>> > >> > > >
> > > > > > >> >>>> > >> > >
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >>
> > > > > > >> >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > > > > >> >>>> > >> > > > > > >> > )
> > > > > > >> >>>> > >> > > > > > >> > > to solve the issue we might
> > introduce a
> > > > > > message
> > > > > > >> >>>> type to
> > > > > > >> >>>> > >> get
> > > > > > >> >>>> > >> > > > > cluster
> > > > > > >> >>>> > >> > > > > > >> > state.
> > > > > > >> >>>> > >> > > > > > >> > > But I agree we can just update
> > > > > > >> >>>> TopicMetadataResponse to
> > > > > > >> >>>> > >> > > include
> > > > > > >> >>>> > >> > > > > > >> > > controllerId (and probably smth
> > else).
> > > > > > >> >>>> > >> > > > > > >> > > What are you thougths?
> > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > >> >>>> > >> > > > > > >> > > Thanks,
> > > > > > >> >>>> > >> > > > > > >> > > Andrii
> > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > >> >>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM,
> > > Guozhang
> > > > > > Wang
> > > > > > >> <
> > > > > > >> >>>> > >> > > > > wangguoz@gmail.com>
> > > > > > >> >>>> > >> > > > > > >> > wrote:
> > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > >> >>>> > >> > > > > > >> > >> I think for the topics commands we
> > can
> > > > > > >> actually
> > > > > > >> >>>> merge
> > > > > > >> >>>> > >> > > > > > >> > >> create/alter/delete/describe as
> one
> > > > > request
> > > > > > >> type
> > > > > > >> >>>> since
> > > > > > >> >>>> > >> > their
> > > > > > >> >>>> > >> > > > > > formats
> > > > > > >> >>>> > >> > > > > > >> are
> > > > > > >> >>>> > >> > > > > > >> > >> very much similar, and keep
> > > list-topics
> > > > > and
> > > > > > >> others
> > > > > > >> >>>> like
> > > > > > >> >>>> > >> > > > > > >> > >> partition-reassignment /
> > > > > > >> preferred-leader-election
> > > > > > >> >>>> as
> > > > > > >> >>>> > >> > > separate
> > > > > > >> >>>> > >> > > > > > >> request
> > > > > > >> >>>> > >> > > > > > >> > >> types, I also left some other
> > comments
> > > > on
> > > > > > the
> > > > > > >> RB (
> > > > > > >> >>>> > >> > > > > > >> > >>
> https://reviews.apache.org/r/29301/
> > ).
> > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > >> >>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM,
> Jay
> > > > > Kreps <
> > > > > > >> >>>> > >> > > > jay.kreps@gmail.com>
> > > > > > >> >>>> > >> > > > > > >> wrote:
> > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > >> >>>> > >> > > > > > >> > >> > Yeah I totally agree that we
> don't
> > > > want
> > > > > to
> > > > > > >> just
> > > > > > >> >>>> have
> > > > > > >> >>>> > >> one
> > > > > > >> >>>> > >> > > "do
> > > > > > >> >>>> > >> > > > > > admin
> > > > > > >> >>>> > >> > > > > > >> > >> stuff"
> > > > > > >> >>>> > >> > > > > > >> > >> > command that has the union of
> all
> > > > > > >> parameters.
> > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > >> >>>> > >> > > > > > >> > >> > What I am saying is that command
> > > line
> > > > > > tools
> > > > > > >> are
> > > > > > >> >>>> one
> > > > > > >> >>>> > >> > client
> > > > > > >> >>>> > >> > > of
> > > > > > >> >>>> > >> > > > > the
> > > > > > >> >>>> > >> > > > > > >> > >> > administrative apis, but these
> > will
> > > be
> > > > > > used
> > > > > > >> in a
> > > > > > >> >>>> number
> > > > > > >> >>>> > >> > of
> > > > > > >> >>>> > >> > > > > > >> scenarios
> > > > > > >> >>>> > >> > > > > > >> > so
> > > > > > >> >>>> > >> > > > > > >> > >> > they should make logical sense
> > even
> > > in
> > > > > the
> > > > > > >> >>>> absence of
> > > > > > >> >>>> > >> the
> > > > > > >> >>>> > >> > > > > command
> > > > > > >> >>>> > >> > > > > > >> line
> > > > > > >> >>>> > >> > > > > > >> > >> > tool. Hence comments like trying
> > to
> > > > > > clarify
> > > > > > >> the
> > > > > > >> >>>> > >> > > relationship
> > > > > > >> >>>> > >> > > > > > >> between
> > > > > > >> >>>> > >> > > > > > >> > >> > ClusterMetadata and
> > > > > TopicMetadata...these
> > > > > > >> kinds
> > > > > > >> >>>> of
> > > > > > >> >>>> > >> things
> > > > > > >> >>>> > >> > > > > really
> > > > > > >> >>>> > >> > > > > > >> need
> > > > > > >> >>>> > >> > > > > > >> > >> to be
> > > > > > >> >>>> > >> > > > > > >> > >> > thought through.
> > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > >> >>>> > >> > > > > > >> > >> > Hope that makes sense.
> > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > >> >>>> > >> > > > > > >> > >> > -Jay
> > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > >> >>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM,
> > > > Andrii
> > > > > > >> >>>> Biletskyi <
> > > > > > >> >>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly>
> > wrote:
> > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > >> >>>> > >> > > > > > >> > >> > > Jay,
> > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > Thanks for answering. You
> > > understood
> > > > > > >> >>>> correctly, most
> > > > > > >> >>>> > >> of
> > > > > > >> >>>> > >> > > my
> > > > > > >> >>>> > >> > > > > > >> comments
> > > > > > >> >>>> > >> > > > > > >> > >> were
> > > > > > >> >>>> > >> > > > > > >> > >> > > related to your point 1) -
> about
> > > > "well
> > > > > > >> >>>> thought-out"
> > > > > > >> >>>> > >> > apis.
> > > > > > >> >>>> > >> > > > > Also,
> > > > > > >> >>>> > >> > > > > > >> yes,
> > > > > > >> >>>> > >> > > > > > >> > >> as I
> > > > > > >> >>>> > >> > > > > > >> > >> > > understood we would like to
> > > > introduce
> > > > > a
> > > > > > >> single
> > > > > > >> >>>> > >> unified
> > > > > > >> >>>> > >> > > CLI
> > > > > > >> >>>> > >> > > > > tool
> > > > > > >> >>>> > >> > > > > > >> with
> > > > > > >> >>>> > >> > > > > > >> > >> > > centralized server-side
> request
> > > > > handling
> > > > > > >> for
> > > > > > >> >>>> lots of
> > > > > > >> >>>> > >> > > > existing
> > > > > > >> >>>> > >> > > > > > >> ones
> > > > > > >> >>>> > >> > > > > > >> > >> (incl.
> > > > > > >> >>>> > >> > > > > > >> > >> > > TopicCommand,
> > CommitOffsetChecker,
> > > > > > >> >>>> > >> ReassignPartitions,
> > > > > > >> >>>> > >> > > smth
> > > > > > >> >>>> > >> > > > > > else
> > > > > > >> >>>> > >> > > > > > >> if
> > > > > > >> >>>> > >> > > > > > >> > >> added
> > > > > > >> >>>> > >> > > > > > >> > >> > > in future). In our previous
> > > > > discussion (
> > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694)
> > > > > > >> >>>> > >> > people
> > > > > > >> >>>> > >> > > > > said
> > > > > > >> >>>> > >> > > > > > >> > they'd
> > > > > > >> >>>> > >> > > > > > >> > >> > > rather
> > > > > > >> >>>> > >> > > > > > >> > >> > > have a separate message for
> each
> > > > > > command,
> > > > > > >> so,
> > > > > > >> >>>> yes,
> > > > > > >> >>>> > >> this
> > > > > > >> >>>> > >> > > > way I
> > > > > > >> >>>> > >> > > > > > >> came
> > > > > > >> >>>> > >> > > > > > >> > to
> > > > > > >> >>>> > >> > > > > > >> > >> 1-1
> > > > > > >> >>>> > >> > > > > > >> > >> > > mapping between commands in
> the
> > > tool
> > > > > and
> > > > > > >> >>>> protocol
> > > > > > >> >>>> > >> > > > additions.
> > > > > > >> >>>> > >> > > > > > But
> > > > > > >> >>>> > >> > > > > > >> I
> > > > > > >> >>>> > >> > > > > > >> > >> might
> > > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > > >> >>>> > >> > > > > > >> > >> > > wrong.
> > > > > > >> >>>> > >> > > > > > >> > >> > > At the end I just try to start
> > > > > > discussion
> > > > > > >> how
> > > > > > >> >>>> at
> > > > > > >> >>>> > >> least
> > > > > > >> >>>> > >> > > > > > generally
> > > > > > >> >>>> > >> > > > > > >> > this
> > > > > > >> >>>> > >> > > > > > >> > >> > > protocol should look like.
> > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > Thanks,
> > > > > > >> >>>> > >> > > > > > >> > >> > > Andrii
> > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10
> > PM,
> > > > Jay
> > > > > > >> Kreps <
> > > > > > >> >>>> > >> > > > > > jay.kreps@gmail.com
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >> > >> wrote:
> > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > Hey Andrii,
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > To answer your earlier
> > question
> > > we
> > > > > > just
> > > > > > >> >>>> really
> > > > > > >> >>>> > >> can't
> > > > > > >> >>>> > >> > be
> > > > > > >> >>>> > >> > > > > > adding
> > > > > > >> >>>> > >> > > > > > >> any
> > > > > > >> >>>> > >> > > > > > >> > >> more
> > > > > > >> >>>> > >> > > > > > >> > >> > > > scala protocol objects.
> These
> > > > things
> > > > > > are
> > > > > > >> >>>> super hard
> > > > > > >> >>>> > >> > to
> > > > > > >> >>>> > >> > > > > > maintain
> > > > > > >> >>>> > >> > > > > > >> > >> because
> > > > > > >> >>>> > >> > > > > > >> > >> > > > they hand code the byte
> > parsing
> > > > and
> > > > > > >> don't
> > > > > > >> >>>> have good
> > > > > > >> >>>> > >> > > > > > versioning
> > > > > > >> >>>> > >> > > > > > >> > >> support.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > Since we are already
> planning
> > on
> > > > > > >> converting
> > > > > > >> >>>> we
> > > > > > >> >>>> > >> > > definitely
> > > > > > >> >>>> > >> > > > > > don't
> > > > > > >> >>>> > >> > > > > > >> > >> want to
> > > > > > >> >>>> > >> > > > > > >> > >> > > add
> > > > > > >> >>>> > >> > > > > > >> > >> > > > a ton more of these--they
> are
> > > > total
> > > > > > tech
> > > > > > >> >>>> debt.
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > What does it mean that the
> > > changes
> > > > > are
> > > > > > >> >>>> isolated
> > > > > > >> >>>> > >> from
> > > > > > >> >>>> > >> > > the
> > > > > > >> >>>> > >> > > > > > >> current
> > > > > > >> >>>> > >> > > > > > >> > >> code
> > > > > > >> >>>> > >> > > > > > >> > >> > > base?
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > I actually didn't understand
> > the
> > > > > > >> remaining
> > > > > > >> >>>> > >> comments,
> > > > > > >> >>>> > >> > > > which
> > > > > > >> >>>> > >> > > > > of
> > > > > > >> >>>> > >> > > > > > >> the
> > > > > > >> >>>> > >> > > > > > >> > >> > points
> > > > > > >> >>>> > >> > > > > > >> > >> > > > are you responding to?
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > Maybe one sticking point
> here
> > is
> > > > > that
> > > > > > it
> > > > > > >> >>>> seems like
> > > > > > >> >>>> > >> > you
> > > > > > >> >>>> > >> > > > > want
> > > > > > >> >>>> > >> > > > > > to
> > > > > > >> >>>> > >> > > > > > >> > make
> > > > > > >> >>>> > >> > > > > > >> > >> > some
> > > > > > >> >>>> > >> > > > > > >> > >> > > > kind of tool, and you have
> > made
> > > a
> > > > > 1-1
> > > > > > >> mapping
> > > > > > >> >>>> > >> between
> > > > > > >> >>>> > >> > > > > > commands
> > > > > > >> >>>> > >> > > > > > >> you
> > > > > > >> >>>> > >> > > > > > >> > >> > > imagine
> > > > > > >> >>>> > >> > > > > > >> > >> > > > in the tool and protocol
> > > > additions.
> > > > > I
> > > > > > >> want
> > > > > > >> >>>> to make
> > > > > > >> >>>> > >> > sure
> > > > > > >> >>>> > >> > > > we
> > > > > > >> >>>> > >> > > > > > >> don't
> > > > > > >> >>>> > >> > > > > > >> > do
> > > > > > >> >>>> > >> > > > > > >> > >> > that.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > The protocol needs to be
> > really
> > > > > really
> > > > > > >> well
> > > > > > >> >>>> thought
> > > > > > >> >>>> > >> > out
> > > > > > >> >>>> > >> > > > > > against
> > > > > > >> >>>> > >> > > > > > >> > many
> > > > > > >> >>>> > >> > > > > > >> > >> > use
> > > > > > >> >>>> > >> > > > > > >> > >> > > > cases so it should make
> > perfect
> > > > > > logical
> > > > > > >> >>>> sense in
> > > > > > >> >>>> > >> the
> > > > > > >> >>>> > >> > > > > absence
> > > > > > >> >>>> > >> > > > > > of
> > > > > > >> >>>> > >> > > > > > >> > >> knowing
> > > > > > >> >>>> > >> > > > > > >> > >> > > the
> > > > > > >> >>>> > >> > > > > > >> > >> > > > command line tool, right?
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > -Jay
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at
> 11:57
> > > AM,
> > > > > > Andrii
> > > > > > >> >>>> Biletskyi
> > > > > > >> >>>> > >> <
> > > > > > >> >>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly
> >
> > > > wrote:
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > Hey Jay,
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > I would like to continue
> > this
> > > > > > >> discussion
> > > > > > >> >>>> as it
> > > > > > >> >>>> > >> seem
> > > > > > >> >>>> > >> > > > there
> > > > > > >> >>>> > >> > > > > > is
> > > > > > >> >>>> > >> > > > > > >> no
> > > > > > >> >>>> > >> > > > > > >> > >> > > progress
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > here.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > First of all, could you
> > please
> > > > > > explain
> > > > > > >> >>>> what did
> > > > > > >> >>>> > >> you
> > > > > > >> >>>> > >> > > > mean
> > > > > > >> >>>> > >> > > > > in
> > > > > > >> >>>> > >> > > > > > >> 2?
> > > > > > >> >>>> > >> > > > > > >> > How
> > > > > > >> >>>> > >> > > > > > >> > >> > > > exactly
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > are we going to migrate to
> > the
> > > > new
> > > > > > >> java
> > > > > > >> >>>> protocol
> > > > > > >> >>>> > >> > > > > > definitions.
> > > > > > >> >>>> > >> > > > > > >> > And
> > > > > > >> >>>> > >> > > > > > >> > >> why
> > > > > > >> >>>> > >> > > > > > >> > >> > > > it's
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > a blocker for centralized
> > CLI?
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > I agree with you, this
> > feature
> > > > > > >> includes
> > > > > > >> >>>> lots of
> > > > > > >> >>>> > >> > > stuff,
> > > > > > >> >>>> > >> > > > > but
> > > > > > >> >>>> > >> > > > > > >> > >> thankfully
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > almost all changes are
> > > isolated
> > > > > from
> > > > > > >> the
> > > > > > >> >>>> current
> > > > > > >> >>>> > >> > code
> > > > > > >> >>>> > >> > > > > base,
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > so the main thing, I
> think,
> > we
> > > > > need
> > > > > > to
> > > > > > >> >>>> agree is
> > > > > > >> >>>> > >> > RQ/RP
> > > > > > >> >>>> > >> > > > > > format.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > So how can we start
> > discussion
> > > > > about
> > > > > > >> the
> > > > > > >> >>>> concrete
> > > > > > >> >>>> > >> > > > > messages
> > > > > > >> >>>> > >> > > > > > >> > format?
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > Can we take (
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> >>>> > >> > > > > >
> > > > > > >> >>>> > >> > > > >
> > > > > > >> >>>> > >> > > >
> > > > > > >> >>>> > >> > >
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >>
> > > > > > >> >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > as starting point?
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > We had some doubts earlier
> > > > whether
> > > > > > it
> > > > > > >> worth
> > > > > > >> >>>> > >> > > introducing
> > > > > > >> >>>> > >> > > > > one
> > > > > > >> >>>> > >> > > > > > >> > >> generic
> > > > > > >> >>>> > >> > > > > > >> > >> > > Admin
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > Request for all commands (
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > but then everybody agreed
> it
> > > > would
> > > > > > be
> > > > > > >> >>>> better to
> > > > > > >> >>>> > >> > have
> > > > > > >> >>>> > >> > > > > > separate
> > > > > > >> >>>> > >> > > > > > >> > >> message
> > > > > > >> >>>> > >> > > > > > >> > >> > > for
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > each admin command. The
> > > Request
> > > > > part
> > > > > > >> is
> > > > > > >> >>>> really
> > > > > > >> >>>> > >> > > dictated
> > > > > > >> >>>> > >> > > > > > from
> > > > > > >> >>>> > >> > > > > > >> the
> > > > > > >> >>>> > >> > > > > > >> > >> > > command
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand)
> > arguments
> > > > > > itself,
> > > > > > >> so
> > > > > > >> >>>> the
> > > > > > >> >>>> > >> > proposed
> > > > > > >> >>>> > >> > > > > > version
> > > > > > >> >>>> > >> > > > > > >> > >> should
> > > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > fine (let's put aside for
> > now
> > > > > > remarks
> > > > > > >> about
> > > > > > >> >>>> > >> > Optional
> > > > > > >> >>>> > >> > > > > type,
> > > > > > >> >>>> > >> > > > > > >> > >> batching,
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > configs normalization - I
> > > agree
> > > > > with
> > > > > > >> all of
> > > > > > >> >>>> > >> them).
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > So the second part is
> > > Response.
> > > > I
> > > > > > see
> > > > > > >> >>>> there are
> > > > > > >> >>>> > >> two
> > > > > > >> >>>> > >> > > > cases
> > > > > > >> >>>> > >> > > > > > >> here.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> > > > > > >> Create/Alter/... ;
> > > > > > >> >>>> b)
> > > > > > >> >>>> > >> "Get"
> > > > > > >> >>>> > >> > > > > > requests -
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > List/Describe...
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > a) should only hold
> request
> > > > result
> > > > > > >> >>>> (regardless
> > > > > > >> >>>> > >> what
> > > > > > >> >>>> > >> > > we
> > > > > > >> >>>> > >> > > > > > decide
> > > > > > >> >>>> > >> > > > > > >> > >> about
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > blocking/non-blocking
> > commands
> > > > > > >> execution).
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > Usually we provide error
> > code
> > > in
> > > > > > >> response
> > > > > > >> >>>> but
> > > > > > >> >>>> > >> since
> > > > > > >> >>>> > >> > > we
> > > > > > >> >>>> > >> > > > > will
> > > > > > >> >>>> > >> > > > > > >> use
> > > > > > >> >>>> > >> > > > > > >> > >> this
> > > > > > >> >>>> > >> > > > > > >> > >> > in
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > interactive shell we need
> > some
> > > > > human
> > > > > > >> >>>> readable
> > > > > > >> >>>> > >> error
> > > > > > >> >>>> > >> > > > > > >> description
> > > > > > >> >>>> > >> > > > > > >> > -
> > > > > > >> >>>> > >> > > > > > >> > >> so
> > > > > > >> >>>> > >> > > > > > >> > >> > I
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > added errorDesription
> field
> > > > where
> > > > > > you
> > > > > > >> can
> > > > > > >> >>>> at
> > > > > > >> >>>> > >> least
> > > > > > >> >>>> > >> > > > leave
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > exception.getMessage.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > b) in addition to previous
> > > item
> > > > > > >> message
> > > > > > >> >>>> should
> > > > > > >> >>>> > >> hold
> > > > > > >> >>>> > >> > > > > command
> > > > > > >> >>>> > >> > > > > > >> > >> specific
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > response data. We can
> > discuss
> > > in
> > > > > > >> detail
> > > > > > >> >>>> each of
> > > > > > >> >>>> > >> > them
> > > > > > >> >>>> > >> > > > but
> > > > > > >> >>>> > >> > > > > > >> let's
> > > > > > >> >>>> > >> > > > > > >> > for
> > > > > > >> >>>> > >> > > > > > >> > >> > now
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > agree about the overall
> > > pattern.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > Thanks,
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at
> 6:59
> > > AM,
> > > > > Jay
> > > > > > >> Kreps
> > > > > > >> >>>> <
> > > > > > >> >>>> > >> > > > > > >> jay.kreps@gmail.com
> > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > wrote:
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > Hey Joe,
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > This is great. A few
> > > comments
> > > > on
> > > > > > >> KIP-4
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 1. This is much needed
> > > > > > >> functionality,
> > > > > > >> >>>> but there
> > > > > > >> >>>> > >> > > are a
> > > > > > >> >>>> > >> > > > > lot
> > > > > > >> >>>> > >> > > > > > >> of
> > > > > > >> >>>> > >> > > > > > >> > >> the so
> > > > > > >> >>>> > >> > > > > > >> > >> > > > let's
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > really think these
> > protocols
> > > > > > >> through. We
> > > > > > >> >>>> really
> > > > > > >> >>>> > >> > > want
> > > > > > >> >>>> > >> > > > to
> > > > > > >> >>>> > >> > > > > > >> end up
> > > > > > >> >>>> > >> > > > > > >> > >> > with a
> > > > > > >> >>>> > >> > > > > > >> > >> > > > set
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > of well thought-out,
> > > > orthoganol
> > > > > > >> apis.
> > > > > > >> >>>> For this
> > > > > > >> >>>> > >> > > > reason I
> > > > > > >> >>>> > >> > > > > > >> think
> > > > > > >> >>>> > >> > > > > > >> > >> it is
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > really
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > important to think
> through
> > > the
> > > > > end
> > > > > > >> state
> > > > > > >> >>>> even
> > > > > > >> >>>> > >> if
> > > > > > >> >>>> > >> > > that
> > > > > > >> >>>> > >> > > > > > >> includes
> > > > > > >> >>>> > >> > > > > > >> > >> APIs
> > > > > > >> >>>> > >> > > > > > >> > >> > > we
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > won't implement in the
> > first
> > > > > > phase.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 2. Let's please please
> > > please
> > > > > wait
> > > > > > >> until
> > > > > > >> >>>> we
> > > > > > >> >>>> > >> have
> > > > > > >> >>>> > >> > > > > switched
> > > > > > >> >>>> > >> > > > > > >> the
> > > > > > >> >>>> > >> > > > > > >> > >> > server
> > > > > > >> >>>> > >> > > > > > >> > >> > > > over
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > to the new java protocol
> > > > > > >> definitions. If
> > > > > > >> >>>> we add
> > > > > > >> >>>> > >> > > > upteen
> > > > > > >> >>>> > >> > > > > > >> more ad
> > > > > > >> >>>> > >> > > > > > >> > >> hoc
> > > > > > >> >>>> > >> > > > > > >> > >> > > > scala
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > objects that is just
> > > > generating
> > > > > > more
> > > > > > >> >>>> work for
> > > > > > >> >>>> > >> the
> > > > > > >> >>>> > >> > > > > > >> conversion
> > > > > > >> >>>> > >> > > > > > >> > we
> > > > > > >> >>>> > >> > > > > > >> > >> > know
> > > > > > >> >>>> > >> > > > > > >> > >> > > we
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > have to do.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 3. This proposal
> > introduces
> > > a
> > > > > new
> > > > > > >> type of
> > > > > > >> >>>> > >> > optional
> > > > > > >> >>>> > >> > > > > > >> parameter.
> > > > > > >> >>>> > >> > > > > > >> > >> This
> > > > > > >> >>>> > >> > > > > > >> > >> > is
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > inconsistent with
> > everything
> > > > > else
> > > > > > >> in the
> > > > > > >> >>>> > >> protocol
> > > > > > >> >>>> > >> > > > where
> > > > > > >> >>>> > >> > > > > > we
> > > > > > >> >>>> > >> > > > > > >> use
> > > > > > >> >>>> > >> > > > > > >> > >> -1
> > > > > > >> >>>> > >> > > > > > >> > >> > or
> > > > > > >> >>>> > >> > > > > > >> > >> > > > some
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > other marker value. You
> > > could
> > > > > > argue
> > > > > > >> >>>> either way
> > > > > > >> >>>> > >> > but
> > > > > > >> >>>> > >> > > > > let's
> > > > > > >> >>>> > >> > > > > > >> stick
> > > > > > >> >>>> > >> > > > > > >> > >> with
> > > > > > >> >>>> > >> > > > > > >> > >> > > > that
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > for consistency. For
> > clients
> > > > > that
> > > > > > >> >>>> implemented
> > > > > > >> >>>> > >> the
> > > > > > >> >>>> > >> > > > > > protocol
> > > > > > >> >>>> > >> > > > > > >> in
> > > > > > >> >>>> > >> > > > > > >> > a
> > > > > > >> >>>> > >> > > > > > >> > >> > > better
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > way
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > than our scala code
> these
> > > > basic
> > > > > > >> >>>> primitives are
> > > > > > >> >>>> > >> > hard
> > > > > > >> >>>> > >> > > > to
> > > > > > >> >>>> > >> > > > > > >> change.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This
> > > seems
> > > > > to
> > > > > > >> >>>> duplicate
> > > > > > >> >>>> > >> > > > > > >> > TopicMetadataRequest
> > > > > > >> >>>> > >> > > > > > >> > >> > > which
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > brokers, topics, and
> > > > > partitions. I
> > > > > > >> think
> > > > > > >> >>>> we
> > > > > > >> >>>> > >> > should
> > > > > > >> >>>> > >> > > > > rename
> > > > > > >> >>>> > >> > > > > > >> that
> > > > > > >> >>>> > >> > > > > > >> > >> > > request
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest
> (or
> > > > just
> > > > > > >> >>>> > >> MetadataRequest)
> > > > > > >> >>>> > >> > > and
> > > > > > >> >>>> > >> > > > > > >> include
> > > > > > >> >>>> > >> > > > > > >> > >> the id
> > > > > > >> >>>> > >> > > > > > >> > >> > > of
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > the
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > controller. Or are there
> > > other
> > > > > > >> things we
> > > > > > >> >>>> could
> > > > > > >> >>>> > >> > add
> > > > > > >> >>>> > >> > > > > here?
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to
> > try
> > > > to
> > > > > > >> make a
> > > > > > >> >>>> lot of
> > > > > > >> >>>> > >> > > > requests
> > > > > > >> >>>> > >> > > > > > that
> > > > > > >> >>>> > >> > > > > > >> > can
> > > > > > >> >>>> > >> > > > > > >> > >> > only
> > > > > > >> >>>> > >> > > > > > >> > >> > > go
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > to
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > particular nodes. This
> > adds
> > > a
> > > > > lot
> > > > > > of
> > > > > > >> >>>> burden for
> > > > > > >> >>>> > >> > > > client
> > > > > > >> >>>> > >> > > > > > >> > >> > > implementations
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > (it
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > sounds easy but each
> > > discovery
> > > > > can
> > > > > > >> fail
> > > > > > >> >>>> in many
> > > > > > >> >>>> > >> > > parts
> > > > > > >> >>>> > >> > > > > so
> > > > > > >> >>>> > >> > > > > > it
> > > > > > >> >>>> > >> > > > > > >> > >> ends up
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > being a
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > full state machine to do
> > > > > right). I
> > > > > > >> think
> > > > > > >> >>>> we
> > > > > > >> >>>> > >> > should
> > > > > > >> >>>> > >> > > > > > consider
> > > > > > >> >>>> > >> > > > > > >> > >> making
> > > > > > >> >>>> > >> > > > > > >> > >> > > > admin
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > commands and ideally as
> > many
> > > > of
> > > > > > the
> > > > > > >> >>>> other apis
> > > > > > >> >>>> > >> as
> > > > > > >> >>>> > >> > > > > > possible
> > > > > > >> >>>> > >> > > > > > >> > >> > available
> > > > > > >> >>>> > >> > > > > > >> > >> > > on
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > all
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > brokers and just
> redirect
> > to
> > > > the
> > > > > > >> >>>> controller on
> > > > > > >> >>>> > >> > the
> > > > > > >> >>>> > >> > > > > broker
> > > > > > >> >>>> > >> > > > > > >> > side.
> > > > > > >> >>>> > >> > > > > > >> > >> > > Perhaps
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > there would be a general
> > way
> > > > to
> > > > > > >> >>>> encapsulate
> > > > > > >> >>>> > >> this
> > > > > > >> >>>> > >> > > > > > re-routing
> > > > > > >> >>>> > >> > > > > > >> > >> > behavior.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 6. We should probably
> > > > normalize
> > > > > > the
> > > > > > >> key
> > > > > > >> >>>> value
> > > > > > >> >>>> > >> > pairs
> > > > > > >> >>>> > >> > > > > used
> > > > > > >> >>>> > >> > > > > > >> for
> > > > > > >> >>>> > >> > > > > > >> > >> > configs
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > rather
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > than embedding a new
> > > > formatting.
> > > > > > So
> > > > > > >> two
> > > > > > >> >>>> strings
> > > > > > >> >>>> > >> > > > rather
> > > > > > >> >>>> > >> > > > > > than
> > > > > > >> >>>> > >> > > > > > >> > one
> > > > > > >> >>>> > >> > > > > > >> > >> > with
> > > > > > >> >>>> > >> > > > > > >> > >> > > an
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > internal equals sign.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition
> of
> > > > these
> > > > > > >> APIs
> > > > > > >> >>>> that the
> > > > > > >> >>>> > >> > > > command
> > > > > > >> >>>> > >> > > > > > has
> > > > > > >> >>>> > >> > > > > > >> > >> begun or
> > > > > > >> >>>> > >> > > > > > >> > >> > > > that
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > the command has been
> > > > completed?
> > > > > It
> > > > > > >> is a
> > > > > > >> >>>> lot
> > > > > > >> >>>> > >> more
> > > > > > >> >>>> > >> > > > usable
> > > > > > >> >>>> > >> > > > > > if
> > > > > > >> >>>> > >> > > > > > >> the
> > > > > > >> >>>> > >> > > > > > >> > >> > > command
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > been completed so you
> know
> > > > that
> > > > > if
> > > > > > >> you
> > > > > > >> >>>> create a
> > > > > > >> >>>> > >> > > topic
> > > > > > >> >>>> > >> > > > > and
> > > > > > >> >>>> > >> > > > > > >> then
> > > > > > >> >>>> > >> > > > > > >> > >> > > publish
> > > > > > >> >>>> > >> > > > > > >> > >> > > > to
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > it you won't get an
> > > exception
> > > > > > about
> > > > > > >> >>>> there being
> > > > > > >> >>>> > >> > no
> > > > > > >> >>>> > >> > > > such
> > > > > > >> >>>> > >> > > > > > >> topic.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and
> list
> > > > > topics
> > > > > > >> >>>> duplicate a
> > > > > > >> >>>> > >> lot
> > > > > > >> >>>> > >> > > of
> > > > > > >> >>>> > >> > > > > > stuff
> > > > > > >> >>>> > >> > > > > > >> in
> > > > > > >> >>>> > >> > > > > > >> > >> the
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > metadata
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > request. Is there a
> reason
> > > to
> > > > > give
> > > > > > >> back
> > > > > > >> >>>> topics
> > > > > > >> >>>> > >> > > marked
> > > > > > >> >>>> > >> > > > > for
> > > > > > >> >>>> > >> > > > > > >> > >> > deletion? I
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > feel
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > like if we just make the
> > > > > > >> post-condition
> > > > > > >> >>>> of the
> > > > > > >> >>>> > >> > > delete
> > > > > > >> >>>> > >> > > > > > >> command
> > > > > > >> >>>> > >> > > > > > >> > be
> > > > > > >> >>>> > >> > > > > > >> > >> > that
> > > > > > >> >>>> > >> > > > > > >> > >> > > > the
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > topic is deleted that
> will
> > > get
> > > > > rid
> > > > > > >> of
> > > > > > >> >>>> the need
> > > > > > >> >>>> > >> > for
> > > > > > >> >>>> > >> > > > this
> > > > > > >> >>>> > >> > > > > > >> right?
> > > > > > >> >>>> > >> > > > > > >> > >> And
> > > > > > >> >>>> > >> > > > > > >> > >> > it
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > will
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 9. Should we consider
> > > batching
> > > > > > these
> > > > > > >> >>>> requests?
> > > > > > >> >>>> > >> We
> > > > > > >> >>>> > >> > > > have
> > > > > > >> >>>> > >> > > > > > >> > generally
> > > > > > >> >>>> > >> > > > > > >> > >> > > tried
> > > > > > >> >>>> > >> > > > > > >> > >> > > > to
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > allow multiple
> operations
> > to
> > > > be
> > > > > > >> batched.
> > > > > > >> >>>> My
> > > > > > >> >>>> > >> > > suspicion
> > > > > > >> >>>> > >> > > > > is
> > > > > > >> >>>> > >> > > > > > >> that
> > > > > > >> >>>> > >> > > > > > >> > >> > without
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > this
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > we will get a lot of
> code
> > > that
> > > > > > does
> > > > > > >> >>>> something
> > > > > > >> >>>> > >> > like
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >    for(topic:
> > > > > > >> adminClient.listTopics())
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >>  adminClient.describeTopic(topic)
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > this code will work
> great
> > > when
> > > > > you
> > > > > > >> test
> > > > > > >> >>>> on 5
> > > > > > >> >>>> > >> > topics
> > > > > > >> >>>> > >> > > > but
> > > > > > >> >>>> > >> > > > > > >> not do
> > > > > > >> >>>> > >> > > > > > >> > >> as
> > > > > > >> >>>> > >> > > > > > >> > >> > > well
> > > > > > >> >>>> > >> > > > > > >> > >> > > > if
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > you have 50k.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 10. I think we should
> also
> > > > > discuss
> > > > > > >> how
> > > > > > >> >>>> we want
> > > > > > >> >>>> > >> to
> > > > > > >> >>>> > >> > > > > expose
> > > > > > >> >>>> > >> > > > > > a
> > > > > > >> >>>> > >> > > > > > >> > >> > > programmatic
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > JVM
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > client api for these
> > > > operations.
> > > > > > >> >>>> Currently
> > > > > > >> >>>> > >> people
> > > > > > >> >>>> > >> > > > rely
> > > > > > >> >>>> > >> > > > > on
> > > > > > >> >>>> > >> > > > > > >> > >> > AdminUtils
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > which
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I
> > think
> > > we
> > > > > > >> probably
> > > > > > >> >>>> need
> > > > > > >> >>>> > >> > > another
> > > > > > >> >>>> > >> > > > > > client
> > > > > > >> >>>> > >> > > > > > >> > >> under
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > clients/
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > that exposes
> > administrative
> > > > > > >> >>>> functionality. We
> > > > > > >> >>>> > >> > will
> > > > > > >> >>>> > >> > > > need
> > > > > > >> >>>> > >> > > > > > >> this
> > > > > > >> >>>> > >> > > > > > >> > >> just
> > > > > > >> >>>> > >> > > > > > >> > >> > to
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > properly test the new
> > apis,
> > > I
> > > > > > >> suspect. We
> > > > > > >> >>>> > >> should
> > > > > > >> >>>> > >> > > > figure
> > > > > > >> >>>> > >> > > > > > out
> > > > > > >> >>>> > >> > > > > > >> > that
> > > > > > >> >>>> > >> > > > > > >> > >> > API.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > 11. The other
> information
> > > that
> > > > > > >> would be
> > > > > > >> >>>> really
> > > > > > >> >>>> > >> > > useful
> > > > > > >> >>>> > >> > > > > to
> > > > > > >> >>>> > >> > > > > > >> get
> > > > > > >> >>>> > >> > > > > > >> > >> would
> > > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > information about
> > > > > partitions--how
> > > > > > >> much
> > > > > > >> >>>> data is
> > > > > > >> >>>> > >> in
> > > > > > >> >>>> > >> > > the
> > > > > > >> >>>> > >> > > > > > >> > partition,
> > > > > > >> >>>> > >> > > > > > >> > >> > what
> > > > > > >> >>>> > >> > > > > > >> > >> > > > are
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > the segment offsets,
> what
> > is
> > > > the
> > > > > > >> log-end
> > > > > > >> >>>> offset
> > > > > > >> >>>> > >> > > (i.e.
> > > > > > >> >>>> > >> > > > > > last
> > > > > > >> >>>> > >> > > > > > >> > >> offset),
> > > > > > >> >>>> > >> > > > > > >> > >> > > > what
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > is
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > the compaction point,
> > etc. I
> > > > > think
> > > > > > >> that
> > > > > > >> >>>> done
> > > > > > >> >>>> > >> > right
> > > > > > >> >>>> > >> > > > this
> > > > > > >> >>>> > >> > > > > > >> would
> > > > > > >> >>>> > >> > > > > > >> > be
> > > > > > >> >>>> > >> > > > > > >> > >> > the
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > successor to the very
> > > awkward
> > > > > > >> >>>> OffsetRequest we
> > > > > > >> >>>> > >> > have
> > > > > > >> >>>> > >> > > > > > today.
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > -Jay
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at
> > > 10:27
> > > > > PM,
> > > > > > >> Joe
> > > > > > >> >>>> Stein <
> > > > > > >> >>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > wrote:
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> >>>> > >> > > > > >
> > > > > > >> >>>> > >> > > > >
> > > > > > >> >>>> > >> > > >
> > > > > > >> >>>> > >> > >
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >>
> > > > > > >> >>>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > > JIRA
> > > > > > >> >>>> > >> > > > >
> > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > >> >>>> /*******************************************
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal
> > > > Consultant
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source
> > > > Security
> > > > > > LLC
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> http://www.stealth.ly
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Twitter:
> > > @allthingshadoop <
> > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > http://www.twitter.com/allthingshadoop
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > >> >>>> ********************************************/
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > >> >>>> > >> > > > > > >> > >> --
> > > > > > >> >>>> > >> > > > > > >> > >> -- Guozhang
> > > > > > >> >>>> > >> > > > > > >> > >>
> > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > >> >>>> > >> > > > > > >> > >
> > > > > > >> >>>> > >> > > > > > >> >
> > > > > > >> >>>> > >> > > > > > >>
> > > > > > >> >>>> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > > >
> > > > > > >> >>>> > >> > > > > >
> > > > > > >> >>>> > >> > > > >
> > > > > > >> >>>> > >> > > >
> > > > > > >> >>>> > >> > >
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >> > --
> > > > > > >> >>>> > >> > Jeff Holoman
> > > > > > >> >>>> > >> > Systems Engineer
> > > > > > >> >>>> > >> >
> > > > > > >> >>>> > >>
> > > > > > >> >>>>
> > > > > > >> >>>>
> > > > > > >> >>>
> > > > > > >> >>>
> > > > > > >> >>> --
> > > > > > >> >>> -- Guozhang
> > > > > > >> >>>
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >> --
> > > > > > >> >> -- Guozhang
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
101. There may be a use case where you only want the topics to be created
manually by admins. Currently, you can do that by disabling auto topic
creation and issue topic creation from the TopicCommand. If we disable auto
topic creation completely on the broker and don't have a way to distinguish
between topic creation requests from the regular clients and the admin, we
can't support manual topic creation any more. I was thinking that another
way of distinguishing the clients making the topic creation requests is
using clientId. For example, the admin tool can set it to something like
admin and the broker can treat that clientId specially.

Also, there is a related discussion in KAFKA-2020. Currently, we do the
following in TopicMetadataResponse:

1. If leader is not available, we set the partition level error code to
LeaderNotAvailable.
2. If a non-leader replica is not available, we take that replica out of
the assigned replica list and isr in the response. As an indication for
doing that, we set the partition level error code to ReplicaNotAvailable.

This has a few problems. First, ReplicaNotAvailable probably shouldn't be
an error, at least for the normal producer/consumer clients that just want
to find out the leader. Second, it can happen that both the leader and
another replica are not available at the same time. There is no error code
to indicate both. Third, even if a replica is not available, it's still
useful to return its replica id since some clients (e.g. admin tool) may
still make use of it.

One way to address this issue is to always return the replica id for
leader, assigned replicas, and isr regardless of whether the corresponding
broker is live or not. Since we also return the list of live brokers, the
client can figure out whether a leader or a replica is live or not and act
accordingly. This way, we don't need to set the partition level error code
when the leader or a replica is not available. This doesn't change the wire
protocol, but does change the semantics. Since we are evolving the protocol
of TopicMetadataRequest here, we can potentially piggyback the change.

102.1 For those types of errors due to invalid input, shouldn't we just
guard it at parameter validation time and throw InvalidArgumentException
without even sending the request to the broker?

Thanks,

Jun


On Mon, Mar 16, 2015 at 10:37 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Jun,
>
> Answering your questions:
>
> 101. If I understand you correctly, you are saying future producer versions
> (which
> will be ported to TMR_V1) won't be able to automatically create topic (if
> we
> unconditionally remove topic creation from there). But we need to this
> preserve logic.
> Ok, about your proposal: I'm not a big fan too, when it comes to
> differentiating
> clients directly in protocol schema. And also I'm not sure I understand at
> all why
> auto.create.topics.enable is a server side configuration. Can we deprecate
> this setting
> in future versions, add this setting to producer and based on that upon
> receiving
> UnknownTopic create topic explicitly by a separate producer call via
> adminClient?
>
> 102.1. Hm, yes. It's because we want to support batching and at the same
> time we
> want to give descriptive error messages for clients. Since AdminClient
> holds the context
> to construct such messages (e.g. AdminClient layer can know that
> InvalidArgumentsCode
> means two cases: either invalid number - e.g. -1; or replication-factor was
> provided while
> partitions argument wasn't) - I wrapped responses in Exceptions. But I'm
> open to any
> other ideas, this was just initial version.
> 102.2. Yes, I agree. I'll change that to probably some other dto.
>
> Thanks,
> Andrii Biletskyi
>
> On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io> wrote:
>
> > Andrii,
> >
> > 101. That's what I was thinking too, but it may not be that simple. In
> > TopicMetadataRequest_V1,
> > we can let it not trigger auto topic creation. Then, in the producer
> side,
> > if it gets an UnknownTopicException, it can explicitly issue a
> > createTopicRequest for auto topic creation. On the consumer side, it will
> > never issue createTopicRequest. This works when auto topic creation is
> > enabled on the broker side. However, I am not sure how things will work
> > when auto topic creation is disabled on the broker side. In this case, we
> > want to have a way to manually create a topic, potentially through admin
> > commands. However, then we need a way to distinguish createTopicRequest
> > issued from the producer clients and the admin tools. May be we can add a
> > new field in createTopicRequest and set it differently in the producer
> > client and the admin client. However, I am not sure if that's the best
> > approach.
> >
> > 2. Yes, refactoring existing requests is a non-trivial amount of work. I
> > posted some comments in KAFKA-1927. We will probably have to fix
> KAFKA-1927
> > first, before adding the new logic in KAFKA-1694. Otherwise, the changes
> > will be too big.
> >
> > 102. About the AdminClient:
> > 102.1. It's a bit weird that we return exception in the api. It seems
> that
> > we should either return error code or throw an exception when getting the
> > response state.
> > 102.2. We probably shouldn't explicitly use the request object in the
> api.
> > Not every request evolution requires an api change.
> >
> > Thanks,
> >
> > Jun
> >
> >
> > On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Jun,
> > >
> > > Thanks for you comments. Answers inline:
> > >
> > > 100. There are a few fields such as ReplicaAssignment,
> > > > ReassignPartitionRequest,
> > > > and PartitionsSerialized that are represented as a string, but
> contain
> > > > composite structures in json. Could we flatten them out directly in
> the
> > > > protocol definition as arrays/records?
> > >
> > >
> > > Yes, now with Admin Client this looks a bit weird. My initial
> motivation
> > > was:
> > > ReassignPartitionCommand accepts input in json, we want to remain
> tools'
> > > interfaces unchanged, where possible.
> > > If we port it to deserialized format, in CLI (/tools project) we will
> > have
> > > to add some
> > > json library since /tools is written in java and we'll need to
> > deserialize
> > > json file
> > > provided by a user. Can we quickly agree on what this library should be
> > > (Jackson, GSON, whatever)?
> > >
> > > 101. Does TopicMetadataRequest v1 still trigger auto topic creation?
> This
> > > > will be a bit weird now that we have a separate topic creation api.
> > Have
> > > > you thought about how the new createTopicRequest and
> > TopicMetadataRequest
> > > > v1 will be used in the producer/consumer client, in addition to admin
> > > > tools? For example, ideally, we don't want TopicMetadataRequest from
> > the
> > > > consumer to trigger auto topic creation.
> > >
> > >
> > > I agree, this strange logic should be fixed. I'm not confident in this
> > > Kafka part so
> > > correct me if I'm wrong, but it doesn't look like a hard thing to do, I
> > > think we can
> > > leverage AdminClient for that in Producer and unconditionally remove
> > topic
> > > creation from the TopicMetadataRequest_V1.
> > >
> > > 2. I think Jay meant getting rid of scala classes
> > > > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We did
> > > that
> > > > as a stop-gap thing when adding the new requests for the consumers.
> > > > However, the long term plan is to get rid of all those and just reuse
> > the
> > > > java request/response in the client. Since this KIP proposes to add a
> > > > significant number of new requests, perhaps we should bite the bullet
> > to
> > > > clean up the existing scala requests first before adding new ones?
> > > >
> > >
> > > Yes, looks like I misunderstood the point of ...RequestAndHeader.
> Okay, I
> > > will
> > > rework that. The only thing is that I don't see any example how it was
> > done
> > > for at
> > > least one existing protocol message. Thus, as I understand, I have to
> > think
> > > how we
> > > are going to do it.
> > > Re porting all existing RQ/RP in this patch. Sounds reasonable, but if
> > it's
> > > an *obligatory*
> > > requirement to have Admin KIP done, I'm afraid this can be a serious
> > > blocker for us.
> > > There are 13 protocol messages and all that would require not only unit
> > > tests but quite
> > > intensive manual testing, no? I'm afraid I'm not the right guy to cover
> > > pretty much all
> > > Kafka core internals :). Let me know your thoughts on this item. Btw
> > there
> > > is a ticket to
> > > follow-up this issue (https://issues.apache.org/jira/browse/KAFKA-2006
> ).
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > >
> > > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <ju...@confluent.io> wrote:
> > >
> > > > Andrii,
> > > >
> > > >
> > > > A few more comments.
> > > >
> > > > 100. There are a few fields such as ReplicaAssignment,
> > > > ReassignPartitionRequest,
> > > > and PartitionsSerialized that are represented as a string, but
> contain
> > > > composite structures in json. Could we flatten them out directly in
> the
> > > > protocol definition as arrays/records?
> > > >
> > > > 101. Does TopicMetadataRequest v1 still trigger auto topic creation?
> > This
> > > > will be a bit weird now that we have a separate topic creation api.
> > Have
> > > > you thought about how the new createTopicRequest and
> > TopicMetadataRequest
> > > > v1 will be used in the producer/consumer client, in addition to admin
> > > > tools? For example, ideally, we don't want TopicMetadataRequest from
> > the
> > > > consumer to trigger auto topic creation.
> > > >
> > > > 2. I think Jay meant getting rid of scala classes
> > > > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We did
> > > that
> > > > as a stop-gap thing when adding the new requests for the consumers.
> > > > However, the long term plan is to get rid of all those and just reuse
> > the
> > > > java request/response in the client. Since this KIP proposes to add a
> > > > significant number of new requests, perhaps we should bite the bullet
> > to
> > > > clean up the existing scala requests first before adding new ones?
> > > >
> > > > Thanks,
> > > >
> > > > Jun
> > > >
> > > >
> > > >
> > > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > As said above - I list again all comments from this thread so we
> > > > > can see what's left and finalize all pending issues.
> > > > >
> > > > > Comments from Jay:
> > > > > 1. This is much needed functionality, but there are a lot of the so
> > > let's
> > > > > really think these protocols through. We really want to end up
> with a
> > > set
> > > > > of well thought-out, orthoganol apis. For this reason I think it is
> > > > really
> > > > > important to think through the end state even if that includes APIs
> > we
> > > > > won't implement in the first phase.
> > > > >
> > > > > A: Definitely behind this. Would appreciate if there are concrete
> > > > comments
> > > > > how this can be improved.
> > > > >
> > > > > 2. Let's please please please wait until we have switched the
> server
> > > over
> > > > > to the new java protocol definitions. If we add upteen more ad hoc
> > > scala
> > > > > objects that is just generating more work for the conversion we
> know
> > we
> > > > > have to do.
> > > > >
> > > > > A: Fixed in the latest patch - removed scala protocol classes.
> > > > >
> > > > > 3. This proposal introduces a new type of optional parameter. This
> is
> > > > > inconsistent with everything else in the protocol where we use -1
> or
> > > some
> > > > > other marker value. You could argue either way but let's stick with
> > > that
> > > > > for consistency. For clients that implemented the protocol in a
> > better
> > > > way
> > > > > than our scala code these basic primitives are hard to change.
> > > > >
> > > > > A: Fixed in the latest patch - removed MaybeOf type and changed
> > > protocol
> > > > > accordingly.
> > > > >
> > > > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
> > which
> > > > has
> > > > > brokers, topics, and partitions. I think we should rename that
> > request
> > > > > ClusterMetadataRequest (or just MetadataRequest) and include the id
> > of
> > > > the
> > > > > controller. Or are there other things we could add here?
> > > > >
> > > > > A: I agree. Updated the KIP. Let's extends TopicMetadata to
> version 2
> > > and
> > > > > include controller.
> > > > >
> > > > > 5. We have a tendency to try to make a lot of requests that can
> only
> > go
> > > > to
> > > > > particular nodes. This adds a lot of burden for client
> > implementations
> > > > (it
> > > > > sounds easy but each discovery can fail in many parts so it ends up
> > > > being a
> > > > > full state machine to do right). I think we should consider making
> > > admin
> > > > > commands and ideally as many of the other apis as possible
> available
> > on
> > > > all
> > > > > brokers and just redirect to the controller on the broker side.
> > Perhaps
> > > > > there would be a general way to encapsulate this re-routing
> behavior.
> > > > >
> > > > > A: It's a very interesting idea, but seems there are some concerns
> > > about
> > > > > this
> > > > > feature (like performance considerations, how this will complicate
> > > server
> > > > > etc).
> > > > > I believe this shouldn't be a blocker. If this feature is
> implemented
> > > at
> > > > > some
> > > > > point it won't affect Admin changes - at least no changes to public
> > API
> > > > > will be required.
> > > > >
> > > > > 6. We should probably normalize the key value pairs used for
> configs
> > > > rather
> > > > > than embedding a new formatting. So two strings rather than one
> with
> > an
> > > > > internal equals sign.
> > > > >
> > > > > A: Fixed in the latest patch - normalized configs and changed
> > protocol
> > > > > accordingly.
> > > > >
> > > > > 7. Is the postcondition of these APIs that the command has begun or
> > > that
> > > > > the command has been completed? It is a lot more usable if the
> > command
> > > > has
> > > > > been completed so you know that if you create a topic and then
> > publish
> > > to
> > > > > it you won't get an exception about there being no such topic.
> > > > >
> > > > > A: For long running requests (like reassign partitions) - the post
> > > > > condition is
> > > > > command has begun - so we don't block the client. In case of your
> > > > example -
> > > > > topic commands, this will be refactored and topic commands will be
> > > > executed
> > > > > immediately, since the Controller will serve Admin requests
> > > > > (follow-up ticket KAFKA-1777).
> > > > >
> > > > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > > > metadata
> > > > > request. Is there a reason to give back topics marked for
> deletion? I
> > > > feel
> > > > > like if we just make the post-condition of the delete command be
> that
> > > the
> > > > > topic is deleted that will get rid of the need for this right? And
> it
> > > > will
> > > > > be much more intuitive.
> > > > >
> > > > > A: Fixed in the latest patch - removed topics marked for deletion
> in
> > > > > ListTopicsRequest.
> > > > >
> > > > > 9. Should we consider batching these requests? We have generally
> > tried
> > > to
> > > > > allow multiple operations to be batched. My suspicion is that
> without
> > > > this
> > > > > we will get a lot of code that does something like
> > > > >    for(topic: adminClient.listTopics())
> > > > >       adminClient.describeTopic(topic)
> > > > > this code will work great when you test on 5 topics but not do as
> > well
> > > if
> > > > > you have 50k.
> > > > >
> > > > > A: Updated the KIP - please check "Topic Admin Schema" section.
> > > > >
> > > > > 10. I think we should also discuss how we want to expose a
> > programmatic
> > > > JVM
> > > > > client api for these operations. Currently people rely on
> AdminUtils
> > > > which
> > > > > is totally sketchy. I think we probably need another client under
> > > > clients/
> > > > > that exposes administrative functionality. We will need this just
> to
> > > > > properly test the new apis, I suspect. We should figure out that
> API.
> > > > >
> > > > > A: Updated the KIP - please check "Admin Client" section with an
> > > initial
> > > > > API proposal.
> > > > >
> > > > > 11. The other information that would be really useful to get would
> be
> > > > > information about partitions--how much data is in the partition,
> what
> > > are
> > > > > the segment offsets, what is the log-end offset (i.e. last offset),
> > > what
> > > > is
> > > > > the compaction point, etc. I think that done right this would be
> the
> > > > > successor to the very awkward OffsetRequest we have today.
> > > > >
> > > > > A: I removed ConsumerGroupOffsetsRequest in the latest patch. I
> > believe
> > > > > this should
> > > > > be resolved in a separate KIP / jira ticket.
> > > > >
> > > > > 12. Generally we can do good error handling without needing custom
> > > > > server-side
> > > > > messages. I.e. generally the client has the context to know that if
> > it
> > > > got
> > > > > an error that the topic doesn't exist to say "Topic X doesn't
> exist"
> > > > rather
> > > > > than "error code 14" (or whatever). Maybe there are specific cases
> > > where
> > > > > this is hard? If we want to add server-side error messages we
> really
> > do
> > > > > need to do this in a consistent way across the protocol.
> > > > >
> > > > > A: Updated the KIP - please check "Protocol Errors" section. I
> added
> > > the
> > > > > comprehensive, fine-grained list of error codes.
> > > > >
> > > > > Comments from Guozhang:
> > > > > 13. Describe topic request: it would be great to go beyond just
> > > batching
> > > > on
> > > > > topic name regex for this request. For example, a very common use
> > case
> > > of
> > > > > the topic command is to list all topics whose config A's value is
> B.
> > > With
> > > > > topic name regex then we have to first retrieve __all__ topics's
> > > > > description info and then filter at the client end, which will be a
> > > huge
> > > > > burden on ZK.
> > > > > AND
> > > > > 14. Config K-Vs in create topic: this is related to the previous
> > point;
> > > > > maybe we can add another metadata K-V or just a metadata string
> along
> > > > side
> > > > > with config K-V in create topic like we did for offset commit
> > request.
> > > > This
> > > > > field can be quite useful in storing information like "owner" of
> the
> > > > topic
> > > > > who issue the create command, etc, which is quite important for a
> > > > > multi-tenant setting. Then in the describe topic request we can
> also
> > > > batch
> > > > > on regex of the metadata field.
> > > > >
> > > > > A: As discussed it is very interesting but can be implemented later
> > > after
> > > > > we have some basic functionality there.
> > > > >
> > > > > 15. Today all the admin operations are async in the sense that
> > command
> > > > will
> > > > > return once it is written in ZK, and that is why we need extra
> > > > verification
> > > > > like testUtil.waitForTopicCreated() / verify partition reassignment
> > > > > request, etc. With admin requests we could add a flag to enable /
> > > disable
> > > > > synchronous requests; when it is turned on, the response will not
> > > return
> > > > > until the request has been completed. And for async requests we can
> > > add a
> > > > > "token" field in the response, and then only need a general "admin
> > > > > verification request" with the given token to check if the async
> > > request
> > > > > has been completed.
> > > > >
> > > > > A: I see your point. My idea was to provide specific
> Verify...Request
> > > per
> > > > > each
> > > > > long running request, where needed. We can do it the way you
> suggest.
> > > The
> > > > > only
> > > > > concern is that introducing a token we again will make schema
> > > "dynamic".
> > > > We
> > > > > wanted
> > > > > to do similar thing introducing single AdminRequest for all topic
> > > > commands
> > > > > but rejected
> > > > > this idea because we wanted to have schema defined. So this is
> more a
> > > > > choice between:
> > > > > a) have fixed schema but introduce each time new Verify...Request
> for
> > > > > long-running requests
> > > > > b) use one request for verification but generalize it with token
> > > > > I'm fine with whatever decision community come to. Just let me know
> > > your
> > > > > thoughts.
> > > > >
> > > > > Comment from Gwen:
> > > > > 16. Specifically for ownership, I think the plan is to add ACL (it
> > > sounds
> > > > > like you are describing ACL) via an external system (Argus,
> Sentry).
> > > > > I remember KIP-11 described this, but I can't find the KIP any
> > longer.
> > > > >
> > > > > A: Okay, no problem. Not sure though how we are going to handle it.
> > > Wait
> > > > > which KIP
> > > > > will be committed first and include changes to TopicMetadata from
> the
> > > > later
> > > > > one?
> > > > > Anyway, I added this note to "Open Questions" section so we don't
> > miss
> > > > this
> > > > > piece.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > Today I uploaded the patch that covers some of the discussed and
> > > agreed
> > > > > > items:
> > > > > > - removed MaybeOf optional type
> > > > > > - switched to java protocol definitions
> > > > > > - simplified messages (normalized configs, removed topic marked
> for
> > > > > > deletion)
> > > > > >
> > > > > > I also updated the KIP-4 with respective changes and wrote down
> my
> > > > > > proposal for
> > > > > > pending items:
> > > > > > - Batch Admin Operations -> updated Wire Protocol schema proposal
> > > > > > - Remove ClusterMetadata -> changed to extend
> TopicMetadataRequest
> > > > > > - Admin Client -> updated my initial proposal to reflect batching
> > > > > > - Error codes -> proposed fine-grained error code instead of
> > > > > > AdminRequestFailed
> > > > > >
> > > > > > I will also send a separate email to cover all comments from this
> > > > thread.
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > >
> > > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> > gshapira@cloudera.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > >> Found KIP-11 (
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > > >> )
> > > > > >> It actually specifies changes to the Metadata protocol, so
> making
> > > sure
> > > > > >> both KIPs are consistent in this regard will be good.
> > > > > >>
> > > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> > > gshapira@cloudera.com
> > > > >
> > > > > >> wrote:
> > > > > >> > Specifically for ownership, I think the plan is to add ACL (it
> > > > sounds
> > > > > >> > like you are describing ACL) via an external system (Argus,
> > > Sentry).
> > > > > >> > I remember KIP-11 described this, but I can't find the KIP any
> > > > longer.
> > > > > >> >
> > > > > >> > Regardless, I think KIP-4 focuses on getting information that
> > > > already
> > > > > >> > exists from Kafka brokers, not on adding information that
> > perhaps
> > > > > >> > should exist but doesn't yet?
> > > > > >> >
> > > > > >> > Gwen
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> > > wangguoz@gmail.com>
> > > > > >> wrote:
> > > > > >> >> Folks,
> > > > > >> >>
> > > > > >> >> Just want to elaborate a bit more on the create-topic
> metadata
> > > and
> > > > > >> batching
> > > > > >> >> describe-topic based on config / metadata in my previous
> email
> > as
> > > > we
> > > > > >> work
> > > > > >> >> on KAFKA-1694. The main motivation is to have some sort of
> > topic
> > > > > >> management
> > > > > >> >> mechanisms, which I think is quite important in a
> multi-tenant
> > /
> > > > > cloud
> > > > > >> >> architecture: today anyone can create topics in a shared
> Kafka
> > > > > >> cluster, but
> > > > > >> >> there is no concept or "ownership" of topics that are created
> > by
> > > > > >> different
> > > > > >> >> users. For example, at LinkedIn we basically distinguish
> topic
> > > > owners
> > > > > >> via
> > > > > >> >> some casual topic name prefix, which is a bit awkward and
> does
> > > not
> > > > > fly
> > > > > >> as
> > > > > >> >> we scale our customers. It would be great to use
> > describe-topics
> > > > such
> > > > > >> as:
> > > > > >> >>
> > > > > >> >> Describe all topics that is created by me.
> > > > > >> >>
> > > > > >> >> Describe all topics whose retention time is overriden to X.
> > > > > >> >>
> > > > > >> >> Describe all topics whose writable group include user Y (this
> > is
> > > > > >> related to
> > > > > >> >> authorization), etc..
> > > > > >> >>
> > > > > >> >> One possible way to achieve this is to add a metadata file in
> > the
> > > > > >> >> create-topic request, whose value will also be written ZK as
> we
> > > > > create
> > > > > >> the
> > > > > >> >> topic; then describe-topics can choose to batch topics based
> on
> > > 1)
> > > > > name
> > > > > >> >> regex, 2) config K-V matching, 3) metadata regex, etc.
> > > > > >> >>
> > > > > >> >> Thoughts?
> > > > > >> >>
> > > > > >> >> Guozhang
> > > > > >> >>
> > > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> > > wangguoz@gmail.com>
> > > > > >> wrote:
> > > > > >> >>
> > > > > >> >>> Thanks for the updated wiki. A few comments below:
> > > > > >> >>>
> > > > > >> >>> 1. Error description in response: I think if some errorCode
> > > could
> > > > > >> indicate
> > > > > >> >>> several different error cases then we should really change
> it
> > to
> > > > > >> multiple
> > > > > >> >>> codes. In general the errorCode itself would be precise and
> > > > > >> sufficient for
> > > > > >> >>> describing the server side errors.
> > > > > >> >>>
> > > > > >> >>> 2. Describe topic request: it would be great to go beyond
> just
> > > > > >> batching on
> > > > > >> >>> topic name regex for this request. For example, a very
> common
> > > use
> > > > > >> case of
> > > > > >> >>> the topic command is to list all topics whose config A's
> value
> > > is
> > > > B.
> > > > > >> With
> > > > > >> >>> topic name regex then we have to first retrieve __all__
> > topics's
> > > > > >> >>> description info and then filter at the client end, which
> will
> > > be
> > > > a
> > > > > >> huge
> > > > > >> >>> burden on ZK.
> > > > > >> >>>
> > > > > >> >>> 3. Config K-Vs in create topic: this is related to the
> > previous
> > > > > point;
> > > > > >> >>> maybe we can add another metadata K-V or just a metadata
> > string
> > > > > along
> > > > > >> side
> > > > > >> >>> with config K-V in create topic like we did for offset
> commit
> > > > > >> request. This
> > > > > >> >>> field can be quite useful in storing information like
> "owner"
> > of
> > > > the
> > > > > >> topic
> > > > > >> >>> who issue the create command, etc, which is quite important
> > for
> > > a
> > > > > >> >>> multi-tenant setting. Then in the describe topic request we
> > can
> > > > also
> > > > > >> batch
> > > > > >> >>> on regex of the metadata field.
> > > > > >> >>>
> > > > > >> >>> 4. Today all the admin operations are async in the sense
> that
> > > > > command
> > > > > >> will
> > > > > >> >>> return once it is written in ZK, and that is why we need
> extra
> > > > > >> verification
> > > > > >> >>> like testUtil.waitForTopicCreated() / verify partition
> > > > reassignment
> > > > > >> >>> request, etc. With admin requests we could add a flag to
> > enable
> > > /
> > > > > >> disable
> > > > > >> >>> synchronous requests; when it is turned on, the response
> will
> > > not
> > > > > >> return
> > > > > >> >>> until the request has been completed. And for async requests
> > we
> > > > can
> > > > > >> add a
> > > > > >> >>> "token" field in the response, and then only need a general
> > > "admin
> > > > > >> >>> verification request" with the given token to check if the
> > async
> > > > > >> request
> > > > > >> >>> has been completed.
> > > > > >> >>>
> > > > > >> >>> 5. +1 for extending Metadata request to include controller /
> > > > > >> coordinator
> > > > > >> >>> information, and then we can remove the ConsumerMetadata /
> > > > > >> ClusterMetadata
> > > > > >> >>> requests.
> > > > > >> >>>
> > > > > >> >>> Guozhang
> > > > > >> >>>
> > > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> > > jjkoshy.w@gmail.com>
> > > > > >> wrote:
> > > > > >> >>>
> > > > > >> >>>> Thanks for sending that out Joe - I don't think I will be
> > able
> > > to
> > > > > >> make
> > > > > >> >>>> it today, so if notes can be sent out afterward that would
> be
> > > > > great.
> > > > > >> >>>>
> > > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira
> wrote:
> > > > > >> >>>> > Thanks for sending this out Joe. Looking forward to
> > chatting
> > > > with
> > > > > >> >>>> everyone :)
> > > > > >> >>>> >
> > > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> > > > joe.stein@stealth.ly>
> > > > > >> wrote:
> > > > > >> >>>> > > Hey, I just sent out a google hangout invite to all
> pmc,
> > > > > >> committers
> > > > > >> >>>> and
> > > > > >> >>>> > > everyone I found working on a KIP. If I missed anyone
> in
> > > the
> > > > > >> invite
> > > > > >> >>>> please
> > > > > >> >>>> > > let me know and can update it, np.
> > > > > >> >>>> > >
> > > > > >> >>>> > > We should do this every Tuesday @ 2pm Eastern Time.
> Maybe
> > > we
> > > > > can
> > > > > >> get
> > > > > >> >>>> INFRA
> > > > > >> >>>> > > help to make a google account so we can manage better?
> > > > > >> >>>> > >
> > > > > >> >>>> > > To discuss
> > > > > >> >>>> > >
> > > > > >> >>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > > >> >>>> > > in progress and related JIRA that are interdependent
> and
> > > > common
> > > > > >> work.
> > > > > >> >>>> > >
> > > > > >> >>>> > > ~ Joe Stein
> > > > > >> >>>> > >
> > > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
> > > > > jay.kreps@gmail.com>
> > > > > >> >>>> wrote:
> > > > > >> >>>> > >
> > > > > >> >>>> > >> Let's stay on Google hangouts that will also record
> and
> > > make
> > > > > the
> > > > > >> >>>> sessions
> > > > > >> >>>> > >> available on youtube.
> > > > > >> >>>> > >>
> > > > > >> >>>> > >> -Jay
> > > > > >> >>>> > >>
> > > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> > > > > >> >>>> jholoman@cloudera.com>
> > > > > >> >>>> > >> wrote:
> > > > > >> >>>> > >>
> > > > > >> >>>> > >> > Jay / Joe
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >> > We're happy to send out a Webex for this purpose. We
> > > could
> > > > > >> record
> > > > > >> >>>> the
> > > > > >> >>>> > >> > sessions if there is interest and publish them out.
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >> > Thanks
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >> > Jeff
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> > > > > >> jay.kreps@gmail.com>
> > > > > >> >>>> wrote:
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >> > > Let's try to get the technical hang-ups sorted
> out,
> > > > > though.
> > > > > >> I
> > > > > >> >>>> really
> > > > > >> >>>> > >> > think
> > > > > >> >>>> > >> > > there is some benefit to live discussion vs
> > writing. I
> > > > am
> > > > > >> >>>> hopeful that
> > > > > >> >>>> > >> if
> > > > > >> >>>> > >> > > we post instructions and give ourselves a few
> > attempts
> > > > we
> > > > > >> can
> > > > > >> >>>> get it
> > > > > >> >>>> > >> > > working.
> > > > > >> >>>> > >> > >
> > > > > >> >>>> > >> > > Tuesday at that time would work for me...any
> > > objections?
> > > > > >> >>>> > >> > >
> > > > > >> >>>> > >> > > -Jay
> > > > > >> >>>> > >> > >
> > > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> > > > > >> joe.stein@stealth.ly
> > > > > >> >>>> >
> > > > > >> >>>> > >> wrote:
> > > > > >> >>>> > >> > >
> > > > > >> >>>> > >> > > > Weekly would be great maybe like every Tuesday ~
> > 1pm
> > > > ET
> > > > > /
> > > > > >> 10am
> > > > > >> >>>> PT
> > > > > >> >>>> > >> ????
> > > > > >> >>>> > >> > > >
> > > > > >> >>>> > >> > > > I don't mind google hangout but there is always
> > some
> > > > > >> issue or
> > > > > >> >>>> > >> whatever
> > > > > >> >>>> > >> > so
> > > > > >> >>>> > >> > > > we know the apache irc channel works. We can
> start
> > > > there
> > > > > >> and
> > > > > >> >>>> see how
> > > > > >> >>>> > >> it
> > > > > >> >>>> > >> > > > goes? We can pull transcripts too and associate
> to
> > > > > >> tickets if
> > > > > >> >>>> need be
> > > > > >> >>>> > >> > > makes
> > > > > >> >>>> > >> > > > it helpful for things.
> > > > > >> >>>> > >> > > >
> > > > > >> >>>> > >> > > > ~ Joestein
> > > > > >> >>>> > >> > > >
> > > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> > > > > >> >>>> jay.kreps@gmail.com>
> > > > > >> >>>> > >> > wrote:
> > > > > >> >>>> > >> > > >
> > > > > >> >>>> > >> > > > > We'd talked about doing a Google Hangout to
> chat
> > > > about
> > > > > >> this.
> > > > > >> >>>> What
> > > > > >> >>>> > >> > about
> > > > > >> >>>> > >> > > > > generalizing that a little further...I
> actually
> > > > think
> > > > > it
> > > > > >> >>>> would be
> > > > > >> >>>> > >> > good
> > > > > >> >>>> > >> > > > for
> > > > > >> >>>> > >> > > > > everyone spending a reasonable chunk of their
> > week
> > > > on
> > > > > >> Kafka
> > > > > >> >>>> stuff
> > > > > >> >>>> > >> to
> > > > > >> >>>> > >> > > > maybe
> > > > > >> >>>> > >> > > > > sync up once a week. I think we could use time
> > to
> > > > talk
> > > > > >> >>>> through
> > > > > >> >>>> > >> design
> > > > > >> >>>> > >> > > > > stuff, make sure we are on top of code
> reviews,
> > > talk
> > > > > >> through
> > > > > >> >>>> any
> > > > > >> >>>> > >> > tricky
> > > > > >> >>>> > >> > > > > issues, etc.
> > > > > >> >>>> > >> > > > >
> > > > > >> >>>> > >> > > > > We can make it publicly available so that any
> > one
> > > > can
> > > > > >> follow
> > > > > >> >>>> along
> > > > > >> >>>> > >> > who
> > > > > >> >>>> > >> > > > > likes.
> > > > > >> >>>> > >> > > > >
> > > > > >> >>>> > >> > > > > Any interest in doing this? If so I'll try to
> > set
> > > it
> > > > > up
> > > > > >> >>>> starting
> > > > > >> >>>> > >> next
> > > > > >> >>>> > >> > > > week.
> > > > > >> >>>> > >> > > > >
> > > > > >> >>>> > >> > > > > -Jay
> > > > > >> >>>> > >> > > > >
> > > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii
> > Biletskyi
> > > <
> > > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >> >>>> > >> > > > >
> > > > > >> >>>> > >> > > > > > Hi all,
> > > > > >> >>>> > >> > > > > >
> > > > > >> >>>> > >> > > > > > I've updated KIP page, fixed / aligned
> > document
> > > > > >> structure.
> > > > > >> >>>> Also I
> > > > > >> >>>> > >> > > added
> > > > > >> >>>> > >> > > > > > some
> > > > > >> >>>> > >> > > > > > very initial proposal for AdminClient so we
> > have
> > > > > >> something
> > > > > >> >>>> to
> > > > > >> >>>> > >> start
> > > > > >> >>>> > >> > > > from
> > > > > >> >>>> > >> > > > > > while
> > > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > > >> >>>> > >> > > > > >
> > > > > >> >>>> > >> > > > > >
> > > > > >> >>>> > >> > > > > >
> > > > > >> >>>> > >> > > > >
> > > > > >> >>>> > >> > > >
> > > > > >> >>>> > >> > >
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >>
> > > > > >> >>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > >> >>>> > >> > > > > >
> > > > > >> >>>> > >> > > > > > Thanks,
> > > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > > >> >>>> > >> > > > > >
> > > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii
> > > Biletskyi
> > > > <
> > > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >> >>>> > >> > > > > >
> > > > > >> >>>> > >> > > > > > > Jay,
> > > > > >> >>>> > >> > > > > > >
> > > > > >> >>>> > >> > > > > > > Re error messages: you are right, in most
> > > cases
> > > > > >> client
> > > > > >> >>>> will
> > > > > >> >>>> > >> have
> > > > > >> >>>> > >> > > > enough
> > > > > >> >>>> > >> > > > > > > context to show descriptive error message.
> > My
> > > > > >> concern is
> > > > > >> >>>> that
> > > > > >> >>>> > >> we
> > > > > >> >>>> > >> > > will
> > > > > >> >>>> > >> > > > > > have
> > > > > >> >>>> > >> > > > > > > to
> > > > > >> >>>> > >> > > > > > > add lots of new error codes for each
> > possible
> > > > > >> error. Of
> > > > > >> >>>> course,
> > > > > >> >>>> > >> > we
> > > > > >> >>>> > >> > > > > could
> > > > > >> >>>> > >> > > > > > > reuse
> > > > > >> >>>> > >> > > > > > > some of existing like
> > > > UknownTopicOrPartitionCode,
> > > > > >> but we
> > > > > >> >>>> will
> > > > > >> >>>> > >> > also
> > > > > >> >>>> > >> > > > need
> > > > > >> >>>> > >> > > > > > to
> > > > > >> >>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> > > > > >> >>>> TopicConfigInvalid (both
> > > > > >> >>>> > >> > for
> > > > > >> >>>> > >> > > > > topic
> > > > > >> >>>> > >> > > > > > > name and config, and probably user would
> > like
> > > to
> > > > > >> know
> > > > > >> >>>> what
> > > > > >> >>>> > >> > exactly
> > > > > >> >>>> > >> > > > > > > is wrong in his config),
> > > > InvalidReplicaAssignment,
> > > > > >> >>>> > >> InternalError
> > > > > >> >>>> > >> > > > (e.g.
> > > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > > > >> >>>> > >> > > > > > > And this is only for TopicCommand, we will
> > > also
> > > > > >> need to
> > > > > >> >>>> add
> > > > > >> >>>> > >> > similar
> > > > > >> >>>> > >> > > > > stuff
> > > > > >> >>>> > >> > > > > > > for
> > > > > >> >>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So
> > we'll
> > > > end
> > > > > >> up
> > > > > >> >>>> with a
> > > > > >> >>>> > >> > large
> > > > > >> >>>> > >> > > > list
> > > > > >> >>>> > >> > > > > > of
> > > > > >> >>>> > >> > > > > > > error codes, used only in Admin protocol.
> > > > > >> >>>> > >> > > > > > > Having said that, I agree my proposal is
> not
> > > > > >> consistent
> > > > > >> >>>> with
> > > > > >> >>>> > >> > other
> > > > > >> >>>> > >> > > > > cases.
> > > > > >> >>>> > >> > > > > > > Maybe we can find better solution or
> > something
> > > > > >> >>>> in-between.
> > > > > >> >>>> > >> > > > > > >
> > > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it is a great
> idea.
> > > > This
> > > > > >> way we
> > > > > >> >>>> can
> > > > > >> >>>> > >> move
> > > > > >> >>>> > >> > > on
> > > > > >> >>>> > >> > > > > > > faster.
> > > > > >> >>>> > >> > > > > > > Let's agree somehow on date/time so people
> > can
> > > > > join.
> > > > > >> >>>> Will work
> > > > > >> >>>> > >> > for
> > > > > >> >>>> > >> > > me
> > > > > >> >>>> > >> > > > > > this
> > > > > >> >>>> > >> > > > > > > and
> > > > > >> >>>> > >> > > > > > > next week almost anytime if agreed in
> > advance.
> > > > > >> >>>> > >> > > > > > >
> > > > > >> >>>> > >> > > > > > > Thanks,
> > > > > >> >>>> > >> > > > > > > Andrii
> > > > > >> >>>> > >> > > > > > >
> > > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay
> Kreps <
> > > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > > >> >>>> > >> > > > > wrote:
> > > > > >> >>>> > >> > > > > > >
> > > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > > >> >>>> > >> > > > > > >>
> > > > > >> >>>> > >> > > > > > >> Generally we can do good error handling
> > > without
> > > > > >> needing
> > > > > >> >>>> custom
> > > > > >> >>>> > >> > > > > > server-side
> > > > > >> >>>> > >> > > > > > >> messages. I.e. generally the client has
> the
> > > > > >> context to
> > > > > >> >>>> know
> > > > > >> >>>> > >> that
> > > > > >> >>>> > >> > > if
> > > > > >> >>>> > >> > > > it
> > > > > >> >>>> > >> > > > > > got
> > > > > >> >>>> > >> > > > > > >> an error that the topic doesn't exist to
> > say
> > > > > >> "Topic X
> > > > > >> >>>> doesn't
> > > > > >> >>>> > >> > > exist"
> > > > > >> >>>> > >> > > > > > >> rather
> > > > > >> >>>> > >> > > > > > >> than "error code 14" (or whatever). Maybe
> > > there
> > > > > are
> > > > > >> >>>> specific
> > > > > >> >>>> > >> > cases
> > > > > >> >>>> > >> > > > > where
> > > > > >> >>>> > >> > > > > > >> this is hard? If we want to add
> server-side
> > > > error
> > > > > >> >>>> messages we
> > > > > >> >>>> > >> > > really
> > > > > >> >>>> > >> > > > > do
> > > > > >> >>>> > >> > > > > > >> need to do this in a consistent way
> across
> > > the
> > > > > >> protocol.
> > > > > >> >>>> > >> > > > > > >>
> > > > > >> >>>> > >> > > > > > >> I still have a bunch of open questions
> here
> > > > from
> > > > > my
> > > > > >> >>>> previous
> > > > > >> >>>> > >> > > list. I
> > > > > >> >>>> > >> > > > > > will
> > > > > >> >>>> > >> > > > > > >> be out for the next few days for Strata
> > > though.
> > > > > >> Maybe
> > > > > >> >>>> we could
> > > > > >> >>>> > >> > do
> > > > > >> >>>> > >> > > a
> > > > > >> >>>> > >> > > > > > Google
> > > > > >> >>>> > >> > > > > > >> Hangout chat on any open issues some time
> > > > towards
> > > > > >> the
> > > > > >> >>>> end of
> > > > > >> >>>> > >> > next
> > > > > >> >>>> > >> > > > week
> > > > > >> >>>> > >> > > > > > for
> > > > > >> >>>> > >> > > > > > >> anyone interested in this ticket? I have
> a
> > > > > feeling
> > > > > >> that
> > > > > >> >>>> might
> > > > > >> >>>> > >> > > > progress
> > > > > >> >>>> > >> > > > > > >> things a little faster than email--I
> think
> > we
> > > > > >> could talk
> > > > > >> >>>> > >> through
> > > > > >> >>>> > >> > > > those
> > > > > >> >>>> > >> > > > > > >> issues I brought up fairly quickly...
> > > > > >> >>>> > >> > > > > > >>
> > > > > >> >>>> > >> > > > > > >> -Jay
> > > > > >> >>>> > >> > > > > > >>
> > > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii
> > > > > Biletskyi <
> > > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > > > > >> >>>> > >> > > > > > >>
> > > > > >> >>>> > >> > > > > > >> > Hi all,
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >> > I'm trying to address some of the
> issues
> > > > which
> > > > > >> were
> > > > > >> >>>> > >> mentioned
> > > > > >> >>>> > >> > > > > earlier
> > > > > >> >>>> > >> > > > > > >> about
> > > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of those was
> > about
> > > > > >> batching
> > > > > >> >>>> > >> > operations.
> > > > > >> >>>> > >> > > > What
> > > > > >> >>>> > >> > > > > > if
> > > > > >> >>>> > >> > > > > > >> we
> > > > > >> >>>> > >> > > > > > >> > follow TopicCommand approach and let
> > people
> > > > > >> specify
> > > > > >> >>>> > >> topic-name
> > > > > >> >>>> > >> > > by
> > > > > >> >>>> > >> > > > > > >> regexp -
> > > > > >> >>>> > >> > > > > > >> > would that cover most of the use cases?
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >> > Secondly, is what information should we
> > > > > generally
> > > > > >> >>>> provide in
> > > > > >> >>>> > >> > > Admin
> > > > > >> >>>> > >> > > > > > >> > responses.
> > > > > >> >>>> > >> > > > > > >> > I realize that Admin commands don't
> imply
> > > > they
> > > > > >> will
> > > > > >> >>>> be used
> > > > > >> >>>> > >> > only
> > > > > >> >>>> > >> > > > in
> > > > > >> >>>> > >> > > > > > CLI
> > > > > >> >>>> > >> > > > > > >> > but,
> > > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a very important
> > > > client
> > > > > >> of this
> > > > > >> >>>> > >> > feature.
> > > > > >> >>>> > >> > > In
> > > > > >> >>>> > >> > > > > > this
> > > > > >> >>>> > >> > > > > > >> > case,
> > > > > >> >>>> > >> > > > > > >> > seems logical, we would like to provide
> > > users
> > > > > >> with
> > > > > >> >>>> rich
> > > > > >> >>>> > >> > > experience
> > > > > >> >>>> > >> > > > > in
> > > > > >> >>>> > >> > > > > > >> terms
> > > > > >> >>>> > >> > > > > > >> > of
> > > > > >> >>>> > >> > > > > > >> > getting results / errors of the
> executed
> > > > > >> commands.
> > > > > >> >>>> Usually
> > > > > >> >>>> > >> we
> > > > > >> >>>> > >> > > > supply
> > > > > >> >>>> > >> > > > > > >> with
> > > > > >> >>>> > >> > > > > > >> > responses only errorCode, which looks
> > very
> > > > > >> limiting,
> > > > > >> >>>> in case
> > > > > >> >>>> > >> > of
> > > > > >> >>>> > >> > > > CLI
> > > > > >> >>>> > >> > > > > we
> > > > > >> >>>> > >> > > > > > >> may
> > > > > >> >>>> > >> > > > > > >> > want to print human readable error
> > > > description.
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >> > So, taking into account previous item
> > about
> > > > > >> batching,
> > > > > >> >>>> what
> > > > > >> >>>> > >> do
> > > > > >> >>>> > >> > > you
> > > > > >> >>>> > >> > > > > > think
> > > > > >> >>>> > >> > > > > > >> > about
> > > > > >> >>>> > >> > > > > > >> > having smth like:
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >> > ('create' doesn't support regexp)
> > > > > >> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName
> > Partitions
> > > > > >> Replicas
> > > > > >> >>>> > >> > > > > ReplicaAssignment
> > > > > >> >>>> > >> > > > > > >> > [Config]
> > > > > >> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode
> > > > > ErrorDescription
> > > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > > > >> >>>> > >> > > > > > >> >   ErrorDescription => string (empty if
> > > > > >> successful)
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp
> > > > Partitions
> > > > > >> >>>> > >> > > ReplicaAssignment
> > > > > >> >>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> > > > > >> >>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName
> > ErrorCode
> > > > > >> >>>> ErrorDescription]
> > > > > >> >>>> > >> > > > > > >> > CommandErrorCode
> CommandErrorDescription
> > > > > >> >>>> > >> > > > > > >> >   CommandErrorCode => int16
> > > > > >> >>>> > >> > > > > > >> >   CommandErrorDescription => string
> > > (nonempty
> > > > > in
> > > > > >> case
> > > > > >> >>>> of
> > > > > >> >>>> > >> fatal
> > > > > >> >>>> > >> > > > > error,
> > > > > >> >>>> > >> > > > > > >> e.g.
> > > > > >> >>>> > >> > > > > > >> > we couldn't get topics by regexp)
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> > > > > >> >>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> > > > > >> TopicDescription
> > > > > >> >>>> > >> ErrorCode
> > > > > >> >>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> > > > > >> >>>> CommandErrorDescription
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >> > Also, any thoughts about our discussion
> > > > > regarding
> > > > > >> >>>> re-routing
> > > > > >> >>>> > >> > > > > facility?
> > > > > >> >>>> > >> > > > > > >> In
> > > > > >> >>>> > >> > > > > > >> > my
> > > > > >> >>>> > >> > > > > > >> > understanding, it is like between
> > > augmenting
> > > > > >> >>>> > >> > > TopicMetadataRequest
> > > > > >> >>>> > >> > > > > > >> > (to include at least controllerId) and
> > > > > >> implementing
> > > > > >> >>>> new
> > > > > >> >>>> > >> > generic
> > > > > >> >>>> > >> > > > > > >> re-routing
> > > > > >> >>>> > >> > > > > > >> > facility so sending messages to
> > controller
> > > > will
> > > > > >> be
> > > > > >> >>>> handled
> > > > > >> >>>> > >> by
> > > > > >> >>>> > >> > > it.
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >> > Thanks,
> > > > > >> >>>> > >> > > > > > >> > Andrii Biletskyi
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii
> > > > > >> Biletskyi <
> > > > > >> >>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >> > > @Guozhang:
> > > > > >> >>>> > >> > > > > > >> > > Thanks for your comments, I've
> answered
> > > > some
> > > > > of
> > > > > >> >>>> those. The
> > > > > >> >>>> > >> > > main
> > > > > >> >>>> > >> > > > > > thing
> > > > > >> >>>> > >> > > > > > >> is
> > > > > >> >>>> > >> > > > > > >> > > having merged request for
> > > > > >> >>>> create-alter-delete-describe - I
> > > > > >> >>>> > >> > > have
> > > > > >> >>>> > >> > > > > some
> > > > > >> >>>> > >> > > > > > >> > > concerns about this approach.
> > > > > >> >>>> > >> > > > > > >> > >
> > > > > >> >>>> > >> > > > > > >> > > @*Jay*:
> > > > > >> >>>> > >> > > > > > >> > > I see that introduced
> > > ClusterMetadaRequest
> > > > is
> > > > > >> also
> > > > > >> >>>> one of
> > > > > >> >>>> > >> > the
> > > > > >> >>>> > >> > > > > > >> concerns.
> > > > > >> >>>> > >> > > > > > >> > We
> > > > > >> >>>> > >> > > > > > >> > > can solve it if we implement
> re-routing
> > > > > >> facility.
> > > > > >> >>>> But I
> > > > > >> >>>> > >> > agree
> > > > > >> >>>> > >> > > > with
> > > > > >> >>>> > >> > > > > > >> > > Guozhang - it will make clients'
> > > internals
> > > > a
> > > > > >> little
> > > > > >> >>>> bit
> > > > > >> >>>> > >> > easier
> > > > > >> >>>> > >> > > > but
> > > > > >> >>>> > >> > > > > > >> this
> > > > > >> >>>> > >> > > > > > >> > > seems to be a complex logic to
> > implement
> > > > and
> > > > > >> >>>> support then.
> > > > > >> >>>> > >> > > > > > Especially
> > > > > >> >>>> > >> > > > > > >> for
> > > > > >> >>>> > >> > > > > > >> > > Fetch and Produce (even if we add
> > > > re-routing
> > > > > >> later
> > > > > >> >>>> for
> > > > > >> >>>> > >> these
> > > > > >> >>>> > >> > > > > > >> requests).
> > > > > >> >>>> > >> > > > > > >> > > Also people will tend to avoid this
> > > > > re-routing
> > > > > >> >>>> facility
> > > > > >> >>>> > >> and
> > > > > >> >>>> > >> > > hold
> > > > > >> >>>> > >> > > > > > local
> > > > > >> >>>> > >> > > > > > >> > > cluster cache to ensure their
> > > high-priority
> > > > > >> requests
> > > > > >> >>>> > >> (which
> > > > > >> >>>> > >> > > some
> > > > > >> >>>> > >> > > > > of
> > > > > >> >>>> > >> > > > > > >> the
> > > > > >> >>>> > >> > > > > > >> > > admin requests are) not sent to some
> > busy
> > > > > >> broker
> > > > > >> >>>> where
> > > > > >> >>>> > >> they
> > > > > >> >>>> > >> > > wait
> > > > > >> >>>> > >> > > > > to
> > > > > >> >>>> > >> > > > > > be
> > > > > >> >>>> > >> > > > > > >> > > routed to the correct one.
> > > > > >> >>>> > >> > > > > > >> > > As pointed out by Jun here (
> > > > > >> >>>> > >> > > > > > >> > >
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >>
> > > > > >> >>>> > >> > > > > >
> > > > > >> >>>> > >> > > > >
> > > > > >> >>>> > >> > > >
> > > > > >> >>>> > >> > >
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >>
> > > > > >> >>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > > > >> >>>> > >> > > > > > >> > )
> > > > > >> >>>> > >> > > > > > >> > > to solve the issue we might
> introduce a
> > > > > message
> > > > > >> >>>> type to
> > > > > >> >>>> > >> get
> > > > > >> >>>> > >> > > > > cluster
> > > > > >> >>>> > >> > > > > > >> > state.
> > > > > >> >>>> > >> > > > > > >> > > But I agree we can just update
> > > > > >> >>>> TopicMetadataResponse to
> > > > > >> >>>> > >> > > include
> > > > > >> >>>> > >> > > > > > >> > > controllerId (and probably smth
> else).
> > > > > >> >>>> > >> > > > > > >> > > What are you thougths?
> > > > > >> >>>> > >> > > > > > >> > >
> > > > > >> >>>> > >> > > > > > >> > > Thanks,
> > > > > >> >>>> > >> > > > > > >> > > Andrii
> > > > > >> >>>> > >> > > > > > >> > >
> > > > > >> >>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM,
> > Guozhang
> > > > > Wang
> > > > > >> <
> > > > > >> >>>> > >> > > > > wangguoz@gmail.com>
> > > > > >> >>>> > >> > > > > > >> > wrote:
> > > > > >> >>>> > >> > > > > > >> > >
> > > > > >> >>>> > >> > > > > > >> > >> I think for the topics commands we
> can
> > > > > >> actually
> > > > > >> >>>> merge
> > > > > >> >>>> > >> > > > > > >> > >> create/alter/delete/describe as one
> > > > request
> > > > > >> type
> > > > > >> >>>> since
> > > > > >> >>>> > >> > their
> > > > > >> >>>> > >> > > > > > formats
> > > > > >> >>>> > >> > > > > > >> are
> > > > > >> >>>> > >> > > > > > >> > >> very much similar, and keep
> > list-topics
> > > > and
> > > > > >> others
> > > > > >> >>>> like
> > > > > >> >>>> > >> > > > > > >> > >> partition-reassignment /
> > > > > >> preferred-leader-election
> > > > > >> >>>> as
> > > > > >> >>>> > >> > > separate
> > > > > >> >>>> > >> > > > > > >> request
> > > > > >> >>>> > >> > > > > > >> > >> types, I also left some other
> comments
> > > on
> > > > > the
> > > > > >> RB (
> > > > > >> >>>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/
> ).
> > > > > >> >>>> > >> > > > > > >> > >>
> > > > > >> >>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay
> > > > Kreps <
> > > > > >> >>>> > >> > > > jay.kreps@gmail.com>
> > > > > >> >>>> > >> > > > > > >> wrote:
> > > > > >> >>>> > >> > > > > > >> > >>
> > > > > >> >>>> > >> > > > > > >> > >> > Yeah I totally agree that we don't
> > > want
> > > > to
> > > > > >> just
> > > > > >> >>>> have
> > > > > >> >>>> > >> one
> > > > > >> >>>> > >> > > "do
> > > > > >> >>>> > >> > > > > > admin
> > > > > >> >>>> > >> > > > > > >> > >> stuff"
> > > > > >> >>>> > >> > > > > > >> > >> > command that has the union of all
> > > > > >> parameters.
> > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > >> >>>> > >> > > > > > >> > >> > What I am saying is that command
> > line
> > > > > tools
> > > > > >> are
> > > > > >> >>>> one
> > > > > >> >>>> > >> > client
> > > > > >> >>>> > >> > > of
> > > > > >> >>>> > >> > > > > the
> > > > > >> >>>> > >> > > > > > >> > >> > administrative apis, but these
> will
> > be
> > > > > used
> > > > > >> in a
> > > > > >> >>>> number
> > > > > >> >>>> > >> > of
> > > > > >> >>>> > >> > > > > > >> scenarios
> > > > > >> >>>> > >> > > > > > >> > so
> > > > > >> >>>> > >> > > > > > >> > >> > they should make logical sense
> even
> > in
> > > > the
> > > > > >> >>>> absence of
> > > > > >> >>>> > >> the
> > > > > >> >>>> > >> > > > > command
> > > > > >> >>>> > >> > > > > > >> line
> > > > > >> >>>> > >> > > > > > >> > >> > tool. Hence comments like trying
> to
> > > > > clarify
> > > > > >> the
> > > > > >> >>>> > >> > > relationship
> > > > > >> >>>> > >> > > > > > >> between
> > > > > >> >>>> > >> > > > > > >> > >> > ClusterMetadata and
> > > > TopicMetadata...these
> > > > > >> kinds
> > > > > >> >>>> of
> > > > > >> >>>> > >> things
> > > > > >> >>>> > >> > > > > really
> > > > > >> >>>> > >> > > > > > >> need
> > > > > >> >>>> > >> > > > > > >> > >> to be
> > > > > >> >>>> > >> > > > > > >> > >> > thought through.
> > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > >> >>>> > >> > > > > > >> > >> > Hope that makes sense.
> > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > >> >>>> > >> > > > > > >> > >> > -Jay
> > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > >> >>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM,
> > > Andrii
> > > > > >> >>>> Biletskyi <
> > > > > >> >>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly>
> wrote:
> > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > >> >>>> > >> > > > > > >> > >> > > Jay,
> > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > >> >>>> > >> > > > > > >> > >> > > Thanks for answering. You
> > understood
> > > > > >> >>>> correctly, most
> > > > > >> >>>> > >> of
> > > > > >> >>>> > >> > > my
> > > > > >> >>>> > >> > > > > > >> comments
> > > > > >> >>>> > >> > > > > > >> > >> were
> > > > > >> >>>> > >> > > > > > >> > >> > > related to your point 1) - about
> > > "well
> > > > > >> >>>> thought-out"
> > > > > >> >>>> > >> > apis.
> > > > > >> >>>> > >> > > > > Also,
> > > > > >> >>>> > >> > > > > > >> yes,
> > > > > >> >>>> > >> > > > > > >> > >> as I
> > > > > >> >>>> > >> > > > > > >> > >> > > understood we would like to
> > > introduce
> > > > a
> > > > > >> single
> > > > > >> >>>> > >> unified
> > > > > >> >>>> > >> > > CLI
> > > > > >> >>>> > >> > > > > tool
> > > > > >> >>>> > >> > > > > > >> with
> > > > > >> >>>> > >> > > > > > >> > >> > > centralized server-side request
> > > > handling
> > > > > >> for
> > > > > >> >>>> lots of
> > > > > >> >>>> > >> > > > existing
> > > > > >> >>>> > >> > > > > > >> ones
> > > > > >> >>>> > >> > > > > > >> > >> (incl.
> > > > > >> >>>> > >> > > > > > >> > >> > > TopicCommand,
> CommitOffsetChecker,
> > > > > >> >>>> > >> ReassignPartitions,
> > > > > >> >>>> > >> > > smth
> > > > > >> >>>> > >> > > > > > else
> > > > > >> >>>> > >> > > > > > >> if
> > > > > >> >>>> > >> > > > > > >> > >> added
> > > > > >> >>>> > >> > > > > > >> > >> > > in future). In our previous
> > > > discussion (
> > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694)
> > > > > >> >>>> > >> > people
> > > > > >> >>>> > >> > > > > said
> > > > > >> >>>> > >> > > > > > >> > they'd
> > > > > >> >>>> > >> > > > > > >> > >> > > rather
> > > > > >> >>>> > >> > > > > > >> > >> > > have a separate message for each
> > > > > command,
> > > > > >> so,
> > > > > >> >>>> yes,
> > > > > >> >>>> > >> this
> > > > > >> >>>> > >> > > > way I
> > > > > >> >>>> > >> > > > > > >> came
> > > > > >> >>>> > >> > > > > > >> > to
> > > > > >> >>>> > >> > > > > > >> > >> 1-1
> > > > > >> >>>> > >> > > > > > >> > >> > > mapping between commands in the
> > tool
> > > > and
> > > > > >> >>>> protocol
> > > > > >> >>>> > >> > > > additions.
> > > > > >> >>>> > >> > > > > > But
> > > > > >> >>>> > >> > > > > > >> I
> > > > > >> >>>> > >> > > > > > >> > >> might
> > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > >> >>>> > >> > > > > > >> > >> > > wrong.
> > > > > >> >>>> > >> > > > > > >> > >> > > At the end I just try to start
> > > > > discussion
> > > > > >> how
> > > > > >> >>>> at
> > > > > >> >>>> > >> least
> > > > > >> >>>> > >> > > > > > generally
> > > > > >> >>>> > >> > > > > > >> > this
> > > > > >> >>>> > >> > > > > > >> > >> > > protocol should look like.
> > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > >> >>>> > >> > > > > > >> > >> > > Thanks,
> > > > > >> >>>> > >> > > > > > >> > >> > > Andrii
> > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > >> >>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10
> PM,
> > > Jay
> > > > > >> Kreps <
> > > > > >> >>>> > >> > > > > > jay.kreps@gmail.com
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >> > >> wrote:
> > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > Hey Andrii,
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > To answer your earlier
> question
> > we
> > > > > just
> > > > > >> >>>> really
> > > > > >> >>>> > >> can't
> > > > > >> >>>> > >> > be
> > > > > >> >>>> > >> > > > > > adding
> > > > > >> >>>> > >> > > > > > >> any
> > > > > >> >>>> > >> > > > > > >> > >> more
> > > > > >> >>>> > >> > > > > > >> > >> > > > scala protocol objects. These
> > > things
> > > > > are
> > > > > >> >>>> super hard
> > > > > >> >>>> > >> > to
> > > > > >> >>>> > >> > > > > > maintain
> > > > > >> >>>> > >> > > > > > >> > >> because
> > > > > >> >>>> > >> > > > > > >> > >> > > > they hand code the byte
> parsing
> > > and
> > > > > >> don't
> > > > > >> >>>> have good
> > > > > >> >>>> > >> > > > > > versioning
> > > > > >> >>>> > >> > > > > > >> > >> support.
> > > > > >> >>>> > >> > > > > > >> > >> > > > Since we are already planning
> on
> > > > > >> converting
> > > > > >> >>>> we
> > > > > >> >>>> > >> > > definitely
> > > > > >> >>>> > >> > > > > > don't
> > > > > >> >>>> > >> > > > > > >> > >> want to
> > > > > >> >>>> > >> > > > > > >> > >> > > add
> > > > > >> >>>> > >> > > > > > >> > >> > > > a ton more of these--they are
> > > total
> > > > > tech
> > > > > >> >>>> debt.
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > What does it mean that the
> > changes
> > > > are
> > > > > >> >>>> isolated
> > > > > >> >>>> > >> from
> > > > > >> >>>> > >> > > the
> > > > > >> >>>> > >> > > > > > >> current
> > > > > >> >>>> > >> > > > > > >> > >> code
> > > > > >> >>>> > >> > > > > > >> > >> > > base?
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > I actually didn't understand
> the
> > > > > >> remaining
> > > > > >> >>>> > >> comments,
> > > > > >> >>>> > >> > > > which
> > > > > >> >>>> > >> > > > > of
> > > > > >> >>>> > >> > > > > > >> the
> > > > > >> >>>> > >> > > > > > >> > >> > points
> > > > > >> >>>> > >> > > > > > >> > >> > > > are you responding to?
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > Maybe one sticking point here
> is
> > > > that
> > > > > it
> > > > > >> >>>> seems like
> > > > > >> >>>> > >> > you
> > > > > >> >>>> > >> > > > > want
> > > > > >> >>>> > >> > > > > > to
> > > > > >> >>>> > >> > > > > > >> > make
> > > > > >> >>>> > >> > > > > > >> > >> > some
> > > > > >> >>>> > >> > > > > > >> > >> > > > kind of tool, and you have
> made
> > a
> > > > 1-1
> > > > > >> mapping
> > > > > >> >>>> > >> between
> > > > > >> >>>> > >> > > > > > commands
> > > > > >> >>>> > >> > > > > > >> you
> > > > > >> >>>> > >> > > > > > >> > >> > > imagine
> > > > > >> >>>> > >> > > > > > >> > >> > > > in the tool and protocol
> > > additions.
> > > > I
> > > > > >> want
> > > > > >> >>>> to make
> > > > > >> >>>> > >> > sure
> > > > > >> >>>> > >> > > > we
> > > > > >> >>>> > >> > > > > > >> don't
> > > > > >> >>>> > >> > > > > > >> > do
> > > > > >> >>>> > >> > > > > > >> > >> > that.
> > > > > >> >>>> > >> > > > > > >> > >> > > > The protocol needs to be
> really
> > > > really
> > > > > >> well
> > > > > >> >>>> thought
> > > > > >> >>>> > >> > out
> > > > > >> >>>> > >> > > > > > against
> > > > > >> >>>> > >> > > > > > >> > many
> > > > > >> >>>> > >> > > > > > >> > >> > use
> > > > > >> >>>> > >> > > > > > >> > >> > > > cases so it should make
> perfect
> > > > > logical
> > > > > >> >>>> sense in
> > > > > >> >>>> > >> the
> > > > > >> >>>> > >> > > > > absence
> > > > > >> >>>> > >> > > > > > of
> > > > > >> >>>> > >> > > > > > >> > >> knowing
> > > > > >> >>>> > >> > > > > > >> > >> > > the
> > > > > >> >>>> > >> > > > > > >> > >> > > > command line tool, right?
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > -Jay
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57
> > AM,
> > > > > Andrii
> > > > > >> >>>> Biletskyi
> > > > > >> >>>> > >> <
> > > > > >> >>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly>
> > > wrote:
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > Hey Jay,
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > I would like to continue
> this
> > > > > >> discussion
> > > > > >> >>>> as it
> > > > > >> >>>> > >> seem
> > > > > >> >>>> > >> > > > there
> > > > > >> >>>> > >> > > > > > is
> > > > > >> >>>> > >> > > > > > >> no
> > > > > >> >>>> > >> > > > > > >> > >> > > progress
> > > > > >> >>>> > >> > > > > > >> > >> > > > > here.
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > First of all, could you
> please
> > > > > explain
> > > > > >> >>>> what did
> > > > > >> >>>> > >> you
> > > > > >> >>>> > >> > > > mean
> > > > > >> >>>> > >> > > > > in
> > > > > >> >>>> > >> > > > > > >> 2?
> > > > > >> >>>> > >> > > > > > >> > How
> > > > > >> >>>> > >> > > > > > >> > >> > > > exactly
> > > > > >> >>>> > >> > > > > > >> > >> > > > > are we going to migrate to
> the
> > > new
> > > > > >> java
> > > > > >> >>>> protocol
> > > > > >> >>>> > >> > > > > > definitions.
> > > > > >> >>>> > >> > > > > > >> > And
> > > > > >> >>>> > >> > > > > > >> > >> why
> > > > > >> >>>> > >> > > > > > >> > >> > > > it's
> > > > > >> >>>> > >> > > > > > >> > >> > > > > a blocker for centralized
> CLI?
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > I agree with you, this
> feature
> > > > > >> includes
> > > > > >> >>>> lots of
> > > > > >> >>>> > >> > > stuff,
> > > > > >> >>>> > >> > > > > but
> > > > > >> >>>> > >> > > > > > >> > >> thankfully
> > > > > >> >>>> > >> > > > > > >> > >> > > > > almost all changes are
> > isolated
> > > > from
> > > > > >> the
> > > > > >> >>>> current
> > > > > >> >>>> > >> > code
> > > > > >> >>>> > >> > > > > base,
> > > > > >> >>>> > >> > > > > > >> > >> > > > > so the main thing, I think,
> we
> > > > need
> > > > > to
> > > > > >> >>>> agree is
> > > > > >> >>>> > >> > RQ/RP
> > > > > >> >>>> > >> > > > > > format.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > So how can we start
> discussion
> > > > about
> > > > > >> the
> > > > > >> >>>> concrete
> > > > > >> >>>> > >> > > > > messages
> > > > > >> >>>> > >> > > > > > >> > format?
> > > > > >> >>>> > >> > > > > > >> > >> > > > > Can we take (
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > >> >>>> > >> > > > > > >> > >>
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >>
> > > > > >> >>>> > >> > > > > >
> > > > > >> >>>> > >> > > > >
> > > > > >> >>>> > >> > > >
> > > > > >> >>>> > >> > >
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >>
> > > > > >> >>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > > > >> >>>> > >> > > > > > >> > >> > > > > as starting point?
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > We had some doubts earlier
> > > whether
> > > > > it
> > > > > >> worth
> > > > > >> >>>> > >> > > introducing
> > > > > >> >>>> > >> > > > > one
> > > > > >> >>>> > >> > > > > > >> > >> generic
> > > > > >> >>>> > >> > > > > > >> > >> > > Admin
> > > > > >> >>>> > >> > > > > > >> > >> > > > > Request for all commands (
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > > > >> >>>> > >> > > > > > >> > >> > > > > but then everybody agreed it
> > > would
> > > > > be
> > > > > >> >>>> better to
> > > > > >> >>>> > >> > have
> > > > > >> >>>> > >> > > > > > separate
> > > > > >> >>>> > >> > > > > > >> > >> message
> > > > > >> >>>> > >> > > > > > >> > >> > > for
> > > > > >> >>>> > >> > > > > > >> > >> > > > > each admin command. The
> > Request
> > > > part
> > > > > >> is
> > > > > >> >>>> really
> > > > > >> >>>> > >> > > dictated
> > > > > >> >>>> > >> > > > > > from
> > > > > >> >>>> > >> > > > > > >> the
> > > > > >> >>>> > >> > > > > > >> > >> > > command
> > > > > >> >>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand)
> arguments
> > > > > itself,
> > > > > >> so
> > > > > >> >>>> the
> > > > > >> >>>> > >> > proposed
> > > > > >> >>>> > >> > > > > > version
> > > > > >> >>>> > >> > > > > > >> > >> should
> > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > >> >>>> > >> > > > > > >> > >> > > > > fine (let's put aside for
> now
> > > > > remarks
> > > > > >> about
> > > > > >> >>>> > >> > Optional
> > > > > >> >>>> > >> > > > > type,
> > > > > >> >>>> > >> > > > > > >> > >> batching,
> > > > > >> >>>> > >> > > > > > >> > >> > > > > configs normalization - I
> > agree
> > > > with
> > > > > >> all of
> > > > > >> >>>> > >> them).
> > > > > >> >>>> > >> > > > > > >> > >> > > > > So the second part is
> > Response.
> > > I
> > > > > see
> > > > > >> >>>> there are
> > > > > >> >>>> > >> two
> > > > > >> >>>> > >> > > > cases
> > > > > >> >>>> > >> > > > > > >> here.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> > > > > >> Create/Alter/... ;
> > > > > >> >>>> b)
> > > > > >> >>>> > >> "Get"
> > > > > >> >>>> > >> > > > > > requests -
> > > > > >> >>>> > >> > > > > > >> > >> > > > > List/Describe...
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > a) should only hold request
> > > result
> > > > > >> >>>> (regardless
> > > > > >> >>>> > >> what
> > > > > >> >>>> > >> > > we
> > > > > >> >>>> > >> > > > > > decide
> > > > > >> >>>> > >> > > > > > >> > >> about
> > > > > >> >>>> > >> > > > > > >> > >> > > > > blocking/non-blocking
> commands
> > > > > >> execution).
> > > > > >> >>>> > >> > > > > > >> > >> > > > > Usually we provide error
> code
> > in
> > > > > >> response
> > > > > >> >>>> but
> > > > > >> >>>> > >> since
> > > > > >> >>>> > >> > > we
> > > > > >> >>>> > >> > > > > will
> > > > > >> >>>> > >> > > > > > >> use
> > > > > >> >>>> > >> > > > > > >> > >> this
> > > > > >> >>>> > >> > > > > > >> > >> > in
> > > > > >> >>>> > >> > > > > > >> > >> > > > > interactive shell we need
> some
> > > > human
> > > > > >> >>>> readable
> > > > > >> >>>> > >> error
> > > > > >> >>>> > >> > > > > > >> description
> > > > > >> >>>> > >> > > > > > >> > -
> > > > > >> >>>> > >> > > > > > >> > >> so
> > > > > >> >>>> > >> > > > > > >> > >> > I
> > > > > >> >>>> > >> > > > > > >> > >> > > > > added errorDesription field
> > > where
> > > > > you
> > > > > >> can
> > > > > >> >>>> at
> > > > > >> >>>> > >> least
> > > > > >> >>>> > >> > > > leave
> > > > > >> >>>> > >> > > > > > >> > >> > > > > exception.getMessage.
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > b) in addition to previous
> > item
> > > > > >> message
> > > > > >> >>>> should
> > > > > >> >>>> > >> hold
> > > > > >> >>>> > >> > > > > command
> > > > > >> >>>> > >> > > > > > >> > >> specific
> > > > > >> >>>> > >> > > > > > >> > >> > > > > response data. We can
> discuss
> > in
> > > > > >> detail
> > > > > >> >>>> each of
> > > > > >> >>>> > >> > them
> > > > > >> >>>> > >> > > > but
> > > > > >> >>>> > >> > > > > > >> let's
> > > > > >> >>>> > >> > > > > > >> > for
> > > > > >> >>>> > >> > > > > > >> > >> > now
> > > > > >> >>>> > >> > > > > > >> > >> > > > > agree about the overall
> > pattern.
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > Thanks,
> > > > > >> >>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59
> > AM,
> > > > Jay
> > > > > >> Kreps
> > > > > >> >>>> <
> > > > > >> >>>> > >> > > > > > >> jay.kreps@gmail.com
> > > > > >> >>>> > >> > > > > > >> > >
> > > > > >> >>>> > >> > > > > > >> > >> > > wrote:
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > Hey Joe,
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > This is great. A few
> > comments
> > > on
> > > > > >> KIP-4
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > 1. This is much needed
> > > > > >> functionality,
> > > > > >> >>>> but there
> > > > > >> >>>> > >> > > are a
> > > > > >> >>>> > >> > > > > lot
> > > > > >> >>>> > >> > > > > > >> of
> > > > > >> >>>> > >> > > > > > >> > >> the so
> > > > > >> >>>> > >> > > > > > >> > >> > > > let's
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > really think these
> protocols
> > > > > >> through. We
> > > > > >> >>>> really
> > > > > >> >>>> > >> > > want
> > > > > >> >>>> > >> > > > to
> > > > > >> >>>> > >> > > > > > >> end up
> > > > > >> >>>> > >> > > > > > >> > >> > with a
> > > > > >> >>>> > >> > > > > > >> > >> > > > set
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > of well thought-out,
> > > orthoganol
> > > > > >> apis.
> > > > > >> >>>> For this
> > > > > >> >>>> > >> > > > reason I
> > > > > >> >>>> > >> > > > > > >> think
> > > > > >> >>>> > >> > > > > > >> > >> it is
> > > > > >> >>>> > >> > > > > > >> > >> > > > > really
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > important to think through
> > the
> > > > end
> > > > > >> state
> > > > > >> >>>> even
> > > > > >> >>>> > >> if
> > > > > >> >>>> > >> > > that
> > > > > >> >>>> > >> > > > > > >> includes
> > > > > >> >>>> > >> > > > > > >> > >> APIs
> > > > > >> >>>> > >> > > > > > >> > >> > > we
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > won't implement in the
> first
> > > > > phase.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > 2. Let's please please
> > please
> > > > wait
> > > > > >> until
> > > > > >> >>>> we
> > > > > >> >>>> > >> have
> > > > > >> >>>> > >> > > > > switched
> > > > > >> >>>> > >> > > > > > >> the
> > > > > >> >>>> > >> > > > > > >> > >> > server
> > > > > >> >>>> > >> > > > > > >> > >> > > > over
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > to the new java protocol
> > > > > >> definitions. If
> > > > > >> >>>> we add
> > > > > >> >>>> > >> > > > upteen
> > > > > >> >>>> > >> > > > > > >> more ad
> > > > > >> >>>> > >> > > > > > >> > >> hoc
> > > > > >> >>>> > >> > > > > > >> > >> > > > scala
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > objects that is just
> > > generating
> > > > > more
> > > > > >> >>>> work for
> > > > > >> >>>> > >> the
> > > > > >> >>>> > >> > > > > > >> conversion
> > > > > >> >>>> > >> > > > > > >> > we
> > > > > >> >>>> > >> > > > > > >> > >> > know
> > > > > >> >>>> > >> > > > > > >> > >> > > we
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > have to do.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > 3. This proposal
> introduces
> > a
> > > > new
> > > > > >> type of
> > > > > >> >>>> > >> > optional
> > > > > >> >>>> > >> > > > > > >> parameter.
> > > > > >> >>>> > >> > > > > > >> > >> This
> > > > > >> >>>> > >> > > > > > >> > >> > is
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > inconsistent with
> everything
> > > > else
> > > > > >> in the
> > > > > >> >>>> > >> protocol
> > > > > >> >>>> > >> > > > where
> > > > > >> >>>> > >> > > > > > we
> > > > > >> >>>> > >> > > > > > >> use
> > > > > >> >>>> > >> > > > > > >> > >> -1
> > > > > >> >>>> > >> > > > > > >> > >> > or
> > > > > >> >>>> > >> > > > > > >> > >> > > > some
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > other marker value. You
> > could
> > > > > argue
> > > > > >> >>>> either way
> > > > > >> >>>> > >> > but
> > > > > >> >>>> > >> > > > > let's
> > > > > >> >>>> > >> > > > > > >> stick
> > > > > >> >>>> > >> > > > > > >> > >> with
> > > > > >> >>>> > >> > > > > > >> > >> > > > that
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > for consistency. For
> clients
> > > > that
> > > > > >> >>>> implemented
> > > > > >> >>>> > >> the
> > > > > >> >>>> > >> > > > > > protocol
> > > > > >> >>>> > >> > > > > > >> in
> > > > > >> >>>> > >> > > > > > >> > a
> > > > > >> >>>> > >> > > > > > >> > >> > > better
> > > > > >> >>>> > >> > > > > > >> > >> > > > > way
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > than our scala code these
> > > basic
> > > > > >> >>>> primitives are
> > > > > >> >>>> > >> > hard
> > > > > >> >>>> > >> > > > to
> > > > > >> >>>> > >> > > > > > >> change.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This
> > seems
> > > > to
> > > > > >> >>>> duplicate
> > > > > >> >>>> > >> > > > > > >> > TopicMetadataRequest
> > > > > >> >>>> > >> > > > > > >> > >> > > which
> > > > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > brokers, topics, and
> > > > partitions. I
> > > > > >> think
> > > > > >> >>>> we
> > > > > >> >>>> > >> > should
> > > > > >> >>>> > >> > > > > rename
> > > > > >> >>>> > >> > > > > > >> that
> > > > > >> >>>> > >> > > > > > >> > >> > > request
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or
> > > just
> > > > > >> >>>> > >> MetadataRequest)
> > > > > >> >>>> > >> > > and
> > > > > >> >>>> > >> > > > > > >> include
> > > > > >> >>>> > >> > > > > > >> > >> the id
> > > > > >> >>>> > >> > > > > > >> > >> > > of
> > > > > >> >>>> > >> > > > > > >> > >> > > > > the
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > controller. Or are there
> > other
> > > > > >> things we
> > > > > >> >>>> could
> > > > > >> >>>> > >> > add
> > > > > >> >>>> > >> > > > > here?
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to
> try
> > > to
> > > > > >> make a
> > > > > >> >>>> lot of
> > > > > >> >>>> > >> > > > requests
> > > > > >> >>>> > >> > > > > > that
> > > > > >> >>>> > >> > > > > > >> > can
> > > > > >> >>>> > >> > > > > > >> > >> > only
> > > > > >> >>>> > >> > > > > > >> > >> > > go
> > > > > >> >>>> > >> > > > > > >> > >> > > > > to
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > particular nodes. This
> adds
> > a
> > > > lot
> > > > > of
> > > > > >> >>>> burden for
> > > > > >> >>>> > >> > > > client
> > > > > >> >>>> > >> > > > > > >> > >> > > implementations
> > > > > >> >>>> > >> > > > > > >> > >> > > > > (it
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > sounds easy but each
> > discovery
> > > > can
> > > > > >> fail
> > > > > >> >>>> in many
> > > > > >> >>>> > >> > > parts
> > > > > >> >>>> > >> > > > > so
> > > > > >> >>>> > >> > > > > > it
> > > > > >> >>>> > >> > > > > > >> > >> ends up
> > > > > >> >>>> > >> > > > > > >> > >> > > > > being a
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > full state machine to do
> > > > right). I
> > > > > >> think
> > > > > >> >>>> we
> > > > > >> >>>> > >> > should
> > > > > >> >>>> > >> > > > > > consider
> > > > > >> >>>> > >> > > > > > >> > >> making
> > > > > >> >>>> > >> > > > > > >> > >> > > > admin
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > commands and ideally as
> many
> > > of
> > > > > the
> > > > > >> >>>> other apis
> > > > > >> >>>> > >> as
> > > > > >> >>>> > >> > > > > > possible
> > > > > >> >>>> > >> > > > > > >> > >> > available
> > > > > >> >>>> > >> > > > > > >> > >> > > on
> > > > > >> >>>> > >> > > > > > >> > >> > > > > all
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > brokers and just redirect
> to
> > > the
> > > > > >> >>>> controller on
> > > > > >> >>>> > >> > the
> > > > > >> >>>> > >> > > > > broker
> > > > > >> >>>> > >> > > > > > >> > side.
> > > > > >> >>>> > >> > > > > > >> > >> > > Perhaps
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > there would be a general
> way
> > > to
> > > > > >> >>>> encapsulate
> > > > > >> >>>> > >> this
> > > > > >> >>>> > >> > > > > > re-routing
> > > > > >> >>>> > >> > > > > > >> > >> > behavior.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > 6. We should probably
> > > normalize
> > > > > the
> > > > > >> key
> > > > > >> >>>> value
> > > > > >> >>>> > >> > pairs
> > > > > >> >>>> > >> > > > > used
> > > > > >> >>>> > >> > > > > > >> for
> > > > > >> >>>> > >> > > > > > >> > >> > configs
> > > > > >> >>>> > >> > > > > > >> > >> > > > > rather
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > than embedding a new
> > > formatting.
> > > > > So
> > > > > >> two
> > > > > >> >>>> strings
> > > > > >> >>>> > >> > > > rather
> > > > > >> >>>> > >> > > > > > than
> > > > > >> >>>> > >> > > > > > >> > one
> > > > > >> >>>> > >> > > > > > >> > >> > with
> > > > > >> >>>> > >> > > > > > >> > >> > > an
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > internal equals sign.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of
> > > these
> > > > > >> APIs
> > > > > >> >>>> that the
> > > > > >> >>>> > >> > > > command
> > > > > >> >>>> > >> > > > > > has
> > > > > >> >>>> > >> > > > > > >> > >> begun or
> > > > > >> >>>> > >> > > > > > >> > >> > > > that
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > the command has been
> > > completed?
> > > > It
> > > > > >> is a
> > > > > >> >>>> lot
> > > > > >> >>>> > >> more
> > > > > >> >>>> > >> > > > usable
> > > > > >> >>>> > >> > > > > > if
> > > > > >> >>>> > >> > > > > > >> the
> > > > > >> >>>> > >> > > > > > >> > >> > > command
> > > > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > been completed so you know
> > > that
> > > > if
> > > > > >> you
> > > > > >> >>>> create a
> > > > > >> >>>> > >> > > topic
> > > > > >> >>>> > >> > > > > and
> > > > > >> >>>> > >> > > > > > >> then
> > > > > >> >>>> > >> > > > > > >> > >> > > publish
> > > > > >> >>>> > >> > > > > > >> > >> > > > to
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > it you won't get an
> > exception
> > > > > about
> > > > > >> >>>> there being
> > > > > >> >>>> > >> > no
> > > > > >> >>>> > >> > > > such
> > > > > >> >>>> > >> > > > > > >> topic.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list
> > > > topics
> > > > > >> >>>> duplicate a
> > > > > >> >>>> > >> lot
> > > > > >> >>>> > >> > > of
> > > > > >> >>>> > >> > > > > > stuff
> > > > > >> >>>> > >> > > > > > >> in
> > > > > >> >>>> > >> > > > > > >> > >> the
> > > > > >> >>>> > >> > > > > > >> > >> > > > > metadata
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > request. Is there a reason
> > to
> > > > give
> > > > > >> back
> > > > > >> >>>> topics
> > > > > >> >>>> > >> > > marked
> > > > > >> >>>> > >> > > > > for
> > > > > >> >>>> > >> > > > > > >> > >> > deletion? I
> > > > > >> >>>> > >> > > > > > >> > >> > > > > feel
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > like if we just make the
> > > > > >> post-condition
> > > > > >> >>>> of the
> > > > > >> >>>> > >> > > delete
> > > > > >> >>>> > >> > > > > > >> command
> > > > > >> >>>> > >> > > > > > >> > be
> > > > > >> >>>> > >> > > > > > >> > >> > that
> > > > > >> >>>> > >> > > > > > >> > >> > > > the
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > topic is deleted that will
> > get
> > > > rid
> > > > > >> of
> > > > > >> >>>> the need
> > > > > >> >>>> > >> > for
> > > > > >> >>>> > >> > > > this
> > > > > >> >>>> > >> > > > > > >> right?
> > > > > >> >>>> > >> > > > > > >> > >> And
> > > > > >> >>>> > >> > > > > > >> > >> > it
> > > > > >> >>>> > >> > > > > > >> > >> > > > > will
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > 9. Should we consider
> > batching
> > > > > these
> > > > > >> >>>> requests?
> > > > > >> >>>> > >> We
> > > > > >> >>>> > >> > > > have
> > > > > >> >>>> > >> > > > > > >> > generally
> > > > > >> >>>> > >> > > > > > >> > >> > > tried
> > > > > >> >>>> > >> > > > > > >> > >> > > > to
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > allow multiple operations
> to
> > > be
> > > > > >> batched.
> > > > > >> >>>> My
> > > > > >> >>>> > >> > > suspicion
> > > > > >> >>>> > >> > > > > is
> > > > > >> >>>> > >> > > > > > >> that
> > > > > >> >>>> > >> > > > > > >> > >> > without
> > > > > >> >>>> > >> > > > > > >> > >> > > > > this
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > we will get a lot of code
> > that
> > > > > does
> > > > > >> >>>> something
> > > > > >> >>>> > >> > like
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >    for(topic:
> > > > > >> adminClient.listTopics())
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >>  adminClient.describeTopic(topic)
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > this code will work great
> > when
> > > > you
> > > > > >> test
> > > > > >> >>>> on 5
> > > > > >> >>>> > >> > topics
> > > > > >> >>>> > >> > > > but
> > > > > >> >>>> > >> > > > > > >> not do
> > > > > >> >>>> > >> > > > > > >> > >> as
> > > > > >> >>>> > >> > > > > > >> > >> > > well
> > > > > >> >>>> > >> > > > > > >> > >> > > > if
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > you have 50k.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > 10. I think we should also
> > > > discuss
> > > > > >> how
> > > > > >> >>>> we want
> > > > > >> >>>> > >> to
> > > > > >> >>>> > >> > > > > expose
> > > > > >> >>>> > >> > > > > > a
> > > > > >> >>>> > >> > > > > > >> > >> > > programmatic
> > > > > >> >>>> > >> > > > > > >> > >> > > > > JVM
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > client api for these
> > > operations.
> > > > > >> >>>> Currently
> > > > > >> >>>> > >> people
> > > > > >> >>>> > >> > > > rely
> > > > > >> >>>> > >> > > > > on
> > > > > >> >>>> > >> > > > > > >> > >> > AdminUtils
> > > > > >> >>>> > >> > > > > > >> > >> > > > > which
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I
> think
> > we
> > > > > >> probably
> > > > > >> >>>> need
> > > > > >> >>>> > >> > > another
> > > > > >> >>>> > >> > > > > > client
> > > > > >> >>>> > >> > > > > > >> > >> under
> > > > > >> >>>> > >> > > > > > >> > >> > > > > clients/
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > that exposes
> administrative
> > > > > >> >>>> functionality. We
> > > > > >> >>>> > >> > will
> > > > > >> >>>> > >> > > > need
> > > > > >> >>>> > >> > > > > > >> this
> > > > > >> >>>> > >> > > > > > >> > >> just
> > > > > >> >>>> > >> > > > > > >> > >> > to
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > properly test the new
> apis,
> > I
> > > > > >> suspect. We
> > > > > >> >>>> > >> should
> > > > > >> >>>> > >> > > > figure
> > > > > >> >>>> > >> > > > > > out
> > > > > >> >>>> > >> > > > > > >> > that
> > > > > >> >>>> > >> > > > > > >> > >> > API.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > 11. The other information
> > that
> > > > > >> would be
> > > > > >> >>>> really
> > > > > >> >>>> > >> > > useful
> > > > > >> >>>> > >> > > > > to
> > > > > >> >>>> > >> > > > > > >> get
> > > > > >> >>>> > >> > > > > > >> > >> would
> > > > > >> >>>> > >> > > > > > >> > >> > be
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > information about
> > > > partitions--how
> > > > > >> much
> > > > > >> >>>> data is
> > > > > >> >>>> > >> in
> > > > > >> >>>> > >> > > the
> > > > > >> >>>> > >> > > > > > >> > partition,
> > > > > >> >>>> > >> > > > > > >> > >> > what
> > > > > >> >>>> > >> > > > > > >> > >> > > > are
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > the segment offsets, what
> is
> > > the
> > > > > >> log-end
> > > > > >> >>>> offset
> > > > > >> >>>> > >> > > (i.e.
> > > > > >> >>>> > >> > > > > > last
> > > > > >> >>>> > >> > > > > > >> > >> offset),
> > > > > >> >>>> > >> > > > > > >> > >> > > > what
> > > > > >> >>>> > >> > > > > > >> > >> > > > > is
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > the compaction point,
> etc. I
> > > > think
> > > > > >> that
> > > > > >> >>>> done
> > > > > >> >>>> > >> > right
> > > > > >> >>>> > >> > > > this
> > > > > >> >>>> > >> > > > > > >> would
> > > > > >> >>>> > >> > > > > > >> > be
> > > > > >> >>>> > >> > > > > > >> > >> > the
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > successor to the very
> > awkward
> > > > > >> >>>> OffsetRequest we
> > > > > >> >>>> > >> > have
> > > > > >> >>>> > >> > > > > > today.
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > -Jay
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at
> > 10:27
> > > > PM,
> > > > > >> Joe
> > > > > >> >>>> Stein <
> > > > > >> >>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
> > > > > >> >>>> > >> > > > > > >> > >> > > > > wrote:
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > >> >>>> > >> > > > > > >> > >>
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >>
> > > > > >> >>>> > >> > > > > >
> > > > > >> >>>> > >> > > > >
> > > > > >> >>>> > >> > > >
> > > > > >> >>>> > >> > >
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >>
> > > > > >> >>>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > > JIRA
> > > > > >> >>>> > >> > > > >
> > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > >> >>>> /*******************************************
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal
> > > Consultant
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source
> > > Security
> > > > > LLC
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Twitter:
> > @allthingshadoop <
> > > > > >> >>>> > >> > > > > > >> > >> >
> > > http://www.twitter.com/allthingshadoop
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > >> >>>> ********************************************/
> > > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > > >> >>>> > >> > > > > > >> > >> > >
> > > > > >> >>>> > >> > > > > > >> > >> >
> > > > > >> >>>> > >> > > > > > >> > >>
> > > > > >> >>>> > >> > > > > > >> > >>
> > > > > >> >>>> > >> > > > > > >> > >>
> > > > > >> >>>> > >> > > > > > >> > >> --
> > > > > >> >>>> > >> > > > > > >> > >> -- Guozhang
> > > > > >> >>>> > >> > > > > > >> > >>
> > > > > >> >>>> > >> > > > > > >> > >
> > > > > >> >>>> > >> > > > > > >> > >
> > > > > >> >>>> > >> > > > > > >> >
> > > > > >> >>>> > >> > > > > > >>
> > > > > >> >>>> > >> > > > > > >
> > > > > >> >>>> > >> > > > > > >
> > > > > >> >>>> > >> > > > > >
> > > > > >> >>>> > >> > > > >
> > > > > >> >>>> > >> > > >
> > > > > >> >>>> > >> > >
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >> > --
> > > > > >> >>>> > >> > Jeff Holoman
> > > > > >> >>>> > >> > Systems Engineer
> > > > > >> >>>> > >> >
> > > > > >> >>>> > >>
> > > > > >> >>>>
> > > > > >> >>>>
> > > > > >> >>>
> > > > > >> >>>
> > > > > >> >>> --
> > > > > >> >>> -- Guozhang
> > > > > >> >>>
> > > > > >> >>
> > > > > >> >>
> > > > > >> >>
> > > > > >> >> --
> > > > > >> >> -- Guozhang
> > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Jun,

Answering your questions:

101. If I understand you correctly, you are saying future producer versions
(which
will be ported to TMR_V1) won't be able to automatically create topic (if
we
unconditionally remove topic creation from there). But we need to this
preserve logic.
Ok, about your proposal: I'm not a big fan too, when it comes to
differentiating
clients directly in protocol schema. And also I'm not sure I understand at
all why
auto.create.topics.enable is a server side configuration. Can we deprecate
this setting
in future versions, add this setting to producer and based on that upon
receiving
UnknownTopic create topic explicitly by a separate producer call via
adminClient?

102.1. Hm, yes. It's because we want to support batching and at the same
time we
want to give descriptive error messages for clients. Since AdminClient
holds the context
to construct such messages (e.g. AdminClient layer can know that
InvalidArgumentsCode
means two cases: either invalid number - e.g. -1; or replication-factor was
provided while
partitions argument wasn't) - I wrapped responses in Exceptions. But I'm
open to any
other ideas, this was just initial version.
102.2. Yes, I agree. I'll change that to probably some other dto.

Thanks,
Andrii Biletskyi

On Fri, Mar 13, 2015 at 7:16 PM, Jun Rao <ju...@confluent.io> wrote:

> Andrii,
>
> 101. That's what I was thinking too, but it may not be that simple. In
> TopicMetadataRequest_V1,
> we can let it not trigger auto topic creation. Then, in the producer side,
> if it gets an UnknownTopicException, it can explicitly issue a
> createTopicRequest for auto topic creation. On the consumer side, it will
> never issue createTopicRequest. This works when auto topic creation is
> enabled on the broker side. However, I am not sure how things will work
> when auto topic creation is disabled on the broker side. In this case, we
> want to have a way to manually create a topic, potentially through admin
> commands. However, then we need a way to distinguish createTopicRequest
> issued from the producer clients and the admin tools. May be we can add a
> new field in createTopicRequest and set it differently in the producer
> client and the admin client. However, I am not sure if that's the best
> approach.
>
> 2. Yes, refactoring existing requests is a non-trivial amount of work. I
> posted some comments in KAFKA-1927. We will probably have to fix KAFKA-1927
> first, before adding the new logic in KAFKA-1694. Otherwise, the changes
> will be too big.
>
> 102. About the AdminClient:
> 102.1. It's a bit weird that we return exception in the api. It seems that
> we should either return error code or throw an exception when getting the
> response state.
> 102.2. We probably shouldn't explicitly use the request object in the api.
> Not every request evolution requires an api change.
>
> Thanks,
>
> Jun
>
>
> On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Jun,
> >
> > Thanks for you comments. Answers inline:
> >
> > 100. There are a few fields such as ReplicaAssignment,
> > > ReassignPartitionRequest,
> > > and PartitionsSerialized that are represented as a string, but contain
> > > composite structures in json. Could we flatten them out directly in the
> > > protocol definition as arrays/records?
> >
> >
> > Yes, now with Admin Client this looks a bit weird. My initial motivation
> > was:
> > ReassignPartitionCommand accepts input in json, we want to remain tools'
> > interfaces unchanged, where possible.
> > If we port it to deserialized format, in CLI (/tools project) we will
> have
> > to add some
> > json library since /tools is written in java and we'll need to
> deserialize
> > json file
> > provided by a user. Can we quickly agree on what this library should be
> > (Jackson, GSON, whatever)?
> >
> > 101. Does TopicMetadataRequest v1 still trigger auto topic creation? This
> > > will be a bit weird now that we have a separate topic creation api.
> Have
> > > you thought about how the new createTopicRequest and
> TopicMetadataRequest
> > > v1 will be used in the producer/consumer client, in addition to admin
> > > tools? For example, ideally, we don't want TopicMetadataRequest from
> the
> > > consumer to trigger auto topic creation.
> >
> >
> > I agree, this strange logic should be fixed. I'm not confident in this
> > Kafka part so
> > correct me if I'm wrong, but it doesn't look like a hard thing to do, I
> > think we can
> > leverage AdminClient for that in Producer and unconditionally remove
> topic
> > creation from the TopicMetadataRequest_V1.
> >
> > 2. I think Jay meant getting rid of scala classes
> > > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We did
> > that
> > > as a stop-gap thing when adding the new requests for the consumers.
> > > However, the long term plan is to get rid of all those and just reuse
> the
> > > java request/response in the client. Since this KIP proposes to add a
> > > significant number of new requests, perhaps we should bite the bullet
> to
> > > clean up the existing scala requests first before adding new ones?
> > >
> >
> > Yes, looks like I misunderstood the point of ...RequestAndHeader. Okay, I
> > will
> > rework that. The only thing is that I don't see any example how it was
> done
> > for at
> > least one existing protocol message. Thus, as I understand, I have to
> think
> > how we
> > are going to do it.
> > Re porting all existing RQ/RP in this patch. Sounds reasonable, but if
> it's
> > an *obligatory*
> > requirement to have Admin KIP done, I'm afraid this can be a serious
> > blocker for us.
> > There are 13 protocol messages and all that would require not only unit
> > tests but quite
> > intensive manual testing, no? I'm afraid I'm not the right guy to cover
> > pretty much all
> > Kafka core internals :). Let me know your thoughts on this item. Btw
> there
> > is a ticket to
> > follow-up this issue (https://issues.apache.org/jira/browse/KAFKA-2006).
> >
> > Thanks,
> > Andrii Biletskyi
> >
> >
> > On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <ju...@confluent.io> wrote:
> >
> > > Andrii,
> > >
> > >
> > > A few more comments.
> > >
> > > 100. There are a few fields such as ReplicaAssignment,
> > > ReassignPartitionRequest,
> > > and PartitionsSerialized that are represented as a string, but contain
> > > composite structures in json. Could we flatten them out directly in the
> > > protocol definition as arrays/records?
> > >
> > > 101. Does TopicMetadataRequest v1 still trigger auto topic creation?
> This
> > > will be a bit weird now that we have a separate topic creation api.
> Have
> > > you thought about how the new createTopicRequest and
> TopicMetadataRequest
> > > v1 will be used in the producer/consumer client, in addition to admin
> > > tools? For example, ideally, we don't want TopicMetadataRequest from
> the
> > > consumer to trigger auto topic creation.
> > >
> > > 2. I think Jay meant getting rid of scala classes
> > > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We did
> > that
> > > as a stop-gap thing when adding the new requests for the consumers.
> > > However, the long term plan is to get rid of all those and just reuse
> the
> > > java request/response in the client. Since this KIP proposes to add a
> > > significant number of new requests, perhaps we should bite the bullet
> to
> > > clean up the existing scala requests first before adding new ones?
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > >
> > > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Hi,
> > > >
> > > > As said above - I list again all comments from this thread so we
> > > > can see what's left and finalize all pending issues.
> > > >
> > > > Comments from Jay:
> > > > 1. This is much needed functionality, but there are a lot of the so
> > let's
> > > > really think these protocols through. We really want to end up with a
> > set
> > > > of well thought-out, orthoganol apis. For this reason I think it is
> > > really
> > > > important to think through the end state even if that includes APIs
> we
> > > > won't implement in the first phase.
> > > >
> > > > A: Definitely behind this. Would appreciate if there are concrete
> > > comments
> > > > how this can be improved.
> > > >
> > > > 2. Let's please please please wait until we have switched the server
> > over
> > > > to the new java protocol definitions. If we add upteen more ad hoc
> > scala
> > > > objects that is just generating more work for the conversion we know
> we
> > > > have to do.
> > > >
> > > > A: Fixed in the latest patch - removed scala protocol classes.
> > > >
> > > > 3. This proposal introduces a new type of optional parameter. This is
> > > > inconsistent with everything else in the protocol where we use -1 or
> > some
> > > > other marker value. You could argue either way but let's stick with
> > that
> > > > for consistency. For clients that implemented the protocol in a
> better
> > > way
> > > > than our scala code these basic primitives are hard to change.
> > > >
> > > > A: Fixed in the latest patch - removed MaybeOf type and changed
> > protocol
> > > > accordingly.
> > > >
> > > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
> which
> > > has
> > > > brokers, topics, and partitions. I think we should rename that
> request
> > > > ClusterMetadataRequest (or just MetadataRequest) and include the id
> of
> > > the
> > > > controller. Or are there other things we could add here?
> > > >
> > > > A: I agree. Updated the KIP. Let's extends TopicMetadata to version 2
> > and
> > > > include controller.
> > > >
> > > > 5. We have a tendency to try to make a lot of requests that can only
> go
> > > to
> > > > particular nodes. This adds a lot of burden for client
> implementations
> > > (it
> > > > sounds easy but each discovery can fail in many parts so it ends up
> > > being a
> > > > full state machine to do right). I think we should consider making
> > admin
> > > > commands and ideally as many of the other apis as possible available
> on
> > > all
> > > > brokers and just redirect to the controller on the broker side.
> Perhaps
> > > > there would be a general way to encapsulate this re-routing behavior.
> > > >
> > > > A: It's a very interesting idea, but seems there are some concerns
> > about
> > > > this
> > > > feature (like performance considerations, how this will complicate
> > server
> > > > etc).
> > > > I believe this shouldn't be a blocker. If this feature is implemented
> > at
> > > > some
> > > > point it won't affect Admin changes - at least no changes to public
> API
> > > > will be required.
> > > >
> > > > 6. We should probably normalize the key value pairs used for configs
> > > rather
> > > > than embedding a new formatting. So two strings rather than one with
> an
> > > > internal equals sign.
> > > >
> > > > A: Fixed in the latest patch - normalized configs and changed
> protocol
> > > > accordingly.
> > > >
> > > > 7. Is the postcondition of these APIs that the command has begun or
> > that
> > > > the command has been completed? It is a lot more usable if the
> command
> > > has
> > > > been completed so you know that if you create a topic and then
> publish
> > to
> > > > it you won't get an exception about there being no such topic.
> > > >
> > > > A: For long running requests (like reassign partitions) - the post
> > > > condition is
> > > > command has begun - so we don't block the client. In case of your
> > > example -
> > > > topic commands, this will be refactored and topic commands will be
> > > executed
> > > > immediately, since the Controller will serve Admin requests
> > > > (follow-up ticket KAFKA-1777).
> > > >
> > > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > > metadata
> > > > request. Is there a reason to give back topics marked for deletion? I
> > > feel
> > > > like if we just make the post-condition of the delete command be that
> > the
> > > > topic is deleted that will get rid of the need for this right? And it
> > > will
> > > > be much more intuitive.
> > > >
> > > > A: Fixed in the latest patch - removed topics marked for deletion in
> > > > ListTopicsRequest.
> > > >
> > > > 9. Should we consider batching these requests? We have generally
> tried
> > to
> > > > allow multiple operations to be batched. My suspicion is that without
> > > this
> > > > we will get a lot of code that does something like
> > > >    for(topic: adminClient.listTopics())
> > > >       adminClient.describeTopic(topic)
> > > > this code will work great when you test on 5 topics but not do as
> well
> > if
> > > > you have 50k.
> > > >
> > > > A: Updated the KIP - please check "Topic Admin Schema" section.
> > > >
> > > > 10. I think we should also discuss how we want to expose a
> programmatic
> > > JVM
> > > > client api for these operations. Currently people rely on AdminUtils
> > > which
> > > > is totally sketchy. I think we probably need another client under
> > > clients/
> > > > that exposes administrative functionality. We will need this just to
> > > > properly test the new apis, I suspect. We should figure out that API.
> > > >
> > > > A: Updated the KIP - please check "Admin Client" section with an
> > initial
> > > > API proposal.
> > > >
> > > > 11. The other information that would be really useful to get would be
> > > > information about partitions--how much data is in the partition, what
> > are
> > > > the segment offsets, what is the log-end offset (i.e. last offset),
> > what
> > > is
> > > > the compaction point, etc. I think that done right this would be the
> > > > successor to the very awkward OffsetRequest we have today.
> > > >
> > > > A: I removed ConsumerGroupOffsetsRequest in the latest patch. I
> believe
> > > > this should
> > > > be resolved in a separate KIP / jira ticket.
> > > >
> > > > 12. Generally we can do good error handling without needing custom
> > > > server-side
> > > > messages. I.e. generally the client has the context to know that if
> it
> > > got
> > > > an error that the topic doesn't exist to say "Topic X doesn't exist"
> > > rather
> > > > than "error code 14" (or whatever). Maybe there are specific cases
> > where
> > > > this is hard? If we want to add server-side error messages we really
> do
> > > > need to do this in a consistent way across the protocol.
> > > >
> > > > A: Updated the KIP - please check "Protocol Errors" section. I added
> > the
> > > > comprehensive, fine-grained list of error codes.
> > > >
> > > > Comments from Guozhang:
> > > > 13. Describe topic request: it would be great to go beyond just
> > batching
> > > on
> > > > topic name regex for this request. For example, a very common use
> case
> > of
> > > > the topic command is to list all topics whose config A's value is B.
> > With
> > > > topic name regex then we have to first retrieve __all__ topics's
> > > > description info and then filter at the client end, which will be a
> > huge
> > > > burden on ZK.
> > > > AND
> > > > 14. Config K-Vs in create topic: this is related to the previous
> point;
> > > > maybe we can add another metadata K-V or just a metadata string along
> > > side
> > > > with config K-V in create topic like we did for offset commit
> request.
> > > This
> > > > field can be quite useful in storing information like "owner" of the
> > > topic
> > > > who issue the create command, etc, which is quite important for a
> > > > multi-tenant setting. Then in the describe topic request we can also
> > > batch
> > > > on regex of the metadata field.
> > > >
> > > > A: As discussed it is very interesting but can be implemented later
> > after
> > > > we have some basic functionality there.
> > > >
> > > > 15. Today all the admin operations are async in the sense that
> command
> > > will
> > > > return once it is written in ZK, and that is why we need extra
> > > verification
> > > > like testUtil.waitForTopicCreated() / verify partition reassignment
> > > > request, etc. With admin requests we could add a flag to enable /
> > disable
> > > > synchronous requests; when it is turned on, the response will not
> > return
> > > > until the request has been completed. And for async requests we can
> > add a
> > > > "token" field in the response, and then only need a general "admin
> > > > verification request" with the given token to check if the async
> > request
> > > > has been completed.
> > > >
> > > > A: I see your point. My idea was to provide specific Verify...Request
> > per
> > > > each
> > > > long running request, where needed. We can do it the way you suggest.
> > The
> > > > only
> > > > concern is that introducing a token we again will make schema
> > "dynamic".
> > > We
> > > > wanted
> > > > to do similar thing introducing single AdminRequest for all topic
> > > commands
> > > > but rejected
> > > > this idea because we wanted to have schema defined. So this is more a
> > > > choice between:
> > > > a) have fixed schema but introduce each time new Verify...Request for
> > > > long-running requests
> > > > b) use one request for verification but generalize it with token
> > > > I'm fine with whatever decision community come to. Just let me know
> > your
> > > > thoughts.
> > > >
> > > > Comment from Gwen:
> > > > 16. Specifically for ownership, I think the plan is to add ACL (it
> > sounds
> > > > like you are describing ACL) via an external system (Argus, Sentry).
> > > > I remember KIP-11 described this, but I can't find the KIP any
> longer.
> > > >
> > > > A: Okay, no problem. Not sure though how we are going to handle it.
> > Wait
> > > > which KIP
> > > > will be committed first and include changes to TopicMetadata from the
> > > later
> > > > one?
> > > > Anyway, I added this note to "Open Questions" section so we don't
> miss
> > > this
> > > > piece.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > Today I uploaded the patch that covers some of the discussed and
> > agreed
> > > > > items:
> > > > > - removed MaybeOf optional type
> > > > > - switched to java protocol definitions
> > > > > - simplified messages (normalized configs, removed topic marked for
> > > > > deletion)
> > > > >
> > > > > I also updated the KIP-4 with respective changes and wrote down my
> > > > > proposal for
> > > > > pending items:
> > > > > - Batch Admin Operations -> updated Wire Protocol schema proposal
> > > > > - Remove ClusterMetadata -> changed to extend TopicMetadataRequest
> > > > > - Admin Client -> updated my initial proposal to reflect batching
> > > > > - Error codes -> proposed fine-grained error code instead of
> > > > > AdminRequestFailed
> > > > >
> > > > > I will also send a separate email to cover all comments from this
> > > thread.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > >
> > > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <
> gshapira@cloudera.com
> > >
> > > > > wrote:
> > > > >
> > > > >> Found KIP-11 (
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > > >> )
> > > > >> It actually specifies changes to the Metadata protocol, so making
> > sure
> > > > >> both KIPs are consistent in this regard will be good.
> > > > >>
> > > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> > gshapira@cloudera.com
> > > >
> > > > >> wrote:
> > > > >> > Specifically for ownership, I think the plan is to add ACL (it
> > > sounds
> > > > >> > like you are describing ACL) via an external system (Argus,
> > Sentry).
> > > > >> > I remember KIP-11 described this, but I can't find the KIP any
> > > longer.
> > > > >> >
> > > > >> > Regardless, I think KIP-4 focuses on getting information that
> > > already
> > > > >> > exists from Kafka brokers, not on adding information that
> perhaps
> > > > >> > should exist but doesn't yet?
> > > > >> >
> > > > >> > Gwen
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> > wangguoz@gmail.com>
> > > > >> wrote:
> > > > >> >> Folks,
> > > > >> >>
> > > > >> >> Just want to elaborate a bit more on the create-topic metadata
> > and
> > > > >> batching
> > > > >> >> describe-topic based on config / metadata in my previous email
> as
> > > we
> > > > >> work
> > > > >> >> on KAFKA-1694. The main motivation is to have some sort of
> topic
> > > > >> management
> > > > >> >> mechanisms, which I think is quite important in a multi-tenant
> /
> > > > cloud
> > > > >> >> architecture: today anyone can create topics in a shared Kafka
> > > > >> cluster, but
> > > > >> >> there is no concept or "ownership" of topics that are created
> by
> > > > >> different
> > > > >> >> users. For example, at LinkedIn we basically distinguish topic
> > > owners
> > > > >> via
> > > > >> >> some casual topic name prefix, which is a bit awkward and does
> > not
> > > > fly
> > > > >> as
> > > > >> >> we scale our customers. It would be great to use
> describe-topics
> > > such
> > > > >> as:
> > > > >> >>
> > > > >> >> Describe all topics that is created by me.
> > > > >> >>
> > > > >> >> Describe all topics whose retention time is overriden to X.
> > > > >> >>
> > > > >> >> Describe all topics whose writable group include user Y (this
> is
> > > > >> related to
> > > > >> >> authorization), etc..
> > > > >> >>
> > > > >> >> One possible way to achieve this is to add a metadata file in
> the
> > > > >> >> create-topic request, whose value will also be written ZK as we
> > > > create
> > > > >> the
> > > > >> >> topic; then describe-topics can choose to batch topics based on
> > 1)
> > > > name
> > > > >> >> regex, 2) config K-V matching, 3) metadata regex, etc.
> > > > >> >>
> > > > >> >> Thoughts?
> > > > >> >>
> > > > >> >> Guozhang
> > > > >> >>
> > > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> > wangguoz@gmail.com>
> > > > >> wrote:
> > > > >> >>
> > > > >> >>> Thanks for the updated wiki. A few comments below:
> > > > >> >>>
> > > > >> >>> 1. Error description in response: I think if some errorCode
> > could
> > > > >> indicate
> > > > >> >>> several different error cases then we should really change it
> to
> > > > >> multiple
> > > > >> >>> codes. In general the errorCode itself would be precise and
> > > > >> sufficient for
> > > > >> >>> describing the server side errors.
> > > > >> >>>
> > > > >> >>> 2. Describe topic request: it would be great to go beyond just
> > > > >> batching on
> > > > >> >>> topic name regex for this request. For example, a very common
> > use
> > > > >> case of
> > > > >> >>> the topic command is to list all topics whose config A's value
> > is
> > > B.
> > > > >> With
> > > > >> >>> topic name regex then we have to first retrieve __all__
> topics's
> > > > >> >>> description info and then filter at the client end, which will
> > be
> > > a
> > > > >> huge
> > > > >> >>> burden on ZK.
> > > > >> >>>
> > > > >> >>> 3. Config K-Vs in create topic: this is related to the
> previous
> > > > point;
> > > > >> >>> maybe we can add another metadata K-V or just a metadata
> string
> > > > along
> > > > >> side
> > > > >> >>> with config K-V in create topic like we did for offset commit
> > > > >> request. This
> > > > >> >>> field can be quite useful in storing information like "owner"
> of
> > > the
> > > > >> topic
> > > > >> >>> who issue the create command, etc, which is quite important
> for
> > a
> > > > >> >>> multi-tenant setting. Then in the describe topic request we
> can
> > > also
> > > > >> batch
> > > > >> >>> on regex of the metadata field.
> > > > >> >>>
> > > > >> >>> 4. Today all the admin operations are async in the sense that
> > > > command
> > > > >> will
> > > > >> >>> return once it is written in ZK, and that is why we need extra
> > > > >> verification
> > > > >> >>> like testUtil.waitForTopicCreated() / verify partition
> > > reassignment
> > > > >> >>> request, etc. With admin requests we could add a flag to
> enable
> > /
> > > > >> disable
> > > > >> >>> synchronous requests; when it is turned on, the response will
> > not
> > > > >> return
> > > > >> >>> until the request has been completed. And for async requests
> we
> > > can
> > > > >> add a
> > > > >> >>> "token" field in the response, and then only need a general
> > "admin
> > > > >> >>> verification request" with the given token to check if the
> async
> > > > >> request
> > > > >> >>> has been completed.
> > > > >> >>>
> > > > >> >>> 5. +1 for extending Metadata request to include controller /
> > > > >> coordinator
> > > > >> >>> information, and then we can remove the ConsumerMetadata /
> > > > >> ClusterMetadata
> > > > >> >>> requests.
> > > > >> >>>
> > > > >> >>> Guozhang
> > > > >> >>>
> > > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> > jjkoshy.w@gmail.com>
> > > > >> wrote:
> > > > >> >>>
> > > > >> >>>> Thanks for sending that out Joe - I don't think I will be
> able
> > to
> > > > >> make
> > > > >> >>>> it today, so if notes can be sent out afterward that would be
> > > > great.
> > > > >> >>>>
> > > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> > > > >> >>>> > Thanks for sending this out Joe. Looking forward to
> chatting
> > > with
> > > > >> >>>> everyone :)
> > > > >> >>>> >
> > > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> > > joe.stein@stealth.ly>
> > > > >> wrote:
> > > > >> >>>> > > Hey, I just sent out a google hangout invite to all pmc,
> > > > >> committers
> > > > >> >>>> and
> > > > >> >>>> > > everyone I found working on a KIP. If I missed anyone in
> > the
> > > > >> invite
> > > > >> >>>> please
> > > > >> >>>> > > let me know and can update it, np.
> > > > >> >>>> > >
> > > > >> >>>> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe
> > we
> > > > can
> > > > >> get
> > > > >> >>>> INFRA
> > > > >> >>>> > > help to make a google account so we can manage better?
> > > > >> >>>> > >
> > > > >> >>>> > > To discuss
> > > > >> >>>> > >
> > > > >> >>>>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > > >> >>>> > > in progress and related JIRA that are interdependent and
> > > common
> > > > >> work.
> > > > >> >>>> > >
> > > > >> >>>> > > ~ Joe Stein
> > > > >> >>>> > >
> > > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
> > > > jay.kreps@gmail.com>
> > > > >> >>>> wrote:
> > > > >> >>>> > >
> > > > >> >>>> > >> Let's stay on Google hangouts that will also record and
> > make
> > > > the
> > > > >> >>>> sessions
> > > > >> >>>> > >> available on youtube.
> > > > >> >>>> > >>
> > > > >> >>>> > >> -Jay
> > > > >> >>>> > >>
> > > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> > > > >> >>>> jholoman@cloudera.com>
> > > > >> >>>> > >> wrote:
> > > > >> >>>> > >>
> > > > >> >>>> > >> > Jay / Joe
> > > > >> >>>> > >> >
> > > > >> >>>> > >> > We're happy to send out a Webex for this purpose. We
> > could
> > > > >> record
> > > > >> >>>> the
> > > > >> >>>> > >> > sessions if there is interest and publish them out.
> > > > >> >>>> > >> >
> > > > >> >>>> > >> > Thanks
> > > > >> >>>> > >> >
> > > > >> >>>> > >> > Jeff
> > > > >> >>>> > >> >
> > > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> > > > >> jay.kreps@gmail.com>
> > > > >> >>>> wrote:
> > > > >> >>>> > >> >
> > > > >> >>>> > >> > > Let's try to get the technical hang-ups sorted out,
> > > > though.
> > > > >> I
> > > > >> >>>> really
> > > > >> >>>> > >> > think
> > > > >> >>>> > >> > > there is some benefit to live discussion vs
> writing. I
> > > am
> > > > >> >>>> hopeful that
> > > > >> >>>> > >> if
> > > > >> >>>> > >> > > we post instructions and give ourselves a few
> attempts
> > > we
> > > > >> can
> > > > >> >>>> get it
> > > > >> >>>> > >> > > working.
> > > > >> >>>> > >> > >
> > > > >> >>>> > >> > > Tuesday at that time would work for me...any
> > objections?
> > > > >> >>>> > >> > >
> > > > >> >>>> > >> > > -Jay
> > > > >> >>>> > >> > >
> > > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> > > > >> joe.stein@stealth.ly
> > > > >> >>>> >
> > > > >> >>>> > >> wrote:
> > > > >> >>>> > >> > >
> > > > >> >>>> > >> > > > Weekly would be great maybe like every Tuesday ~
> 1pm
> > > ET
> > > > /
> > > > >> 10am
> > > > >> >>>> PT
> > > > >> >>>> > >> ????
> > > > >> >>>> > >> > > >
> > > > >> >>>> > >> > > > I don't mind google hangout but there is always
> some
> > > > >> issue or
> > > > >> >>>> > >> whatever
> > > > >> >>>> > >> > so
> > > > >> >>>> > >> > > > we know the apache irc channel works. We can start
> > > there
> > > > >> and
> > > > >> >>>> see how
> > > > >> >>>> > >> it
> > > > >> >>>> > >> > > > goes? We can pull transcripts too and associate to
> > > > >> tickets if
> > > > >> >>>> need be
> > > > >> >>>> > >> > > makes
> > > > >> >>>> > >> > > > it helpful for things.
> > > > >> >>>> > >> > > >
> > > > >> >>>> > >> > > > ~ Joestein
> > > > >> >>>> > >> > > >
> > > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> > > > >> >>>> jay.kreps@gmail.com>
> > > > >> >>>> > >> > wrote:
> > > > >> >>>> > >> > > >
> > > > >> >>>> > >> > > > > We'd talked about doing a Google Hangout to chat
> > > about
> > > > >> this.
> > > > >> >>>> What
> > > > >> >>>> > >> > about
> > > > >> >>>> > >> > > > > generalizing that a little further...I actually
> > > think
> > > > it
> > > > >> >>>> would be
> > > > >> >>>> > >> > good
> > > > >> >>>> > >> > > > for
> > > > >> >>>> > >> > > > > everyone spending a reasonable chunk of their
> week
> > > on
> > > > >> Kafka
> > > > >> >>>> stuff
> > > > >> >>>> > >> to
> > > > >> >>>> > >> > > > maybe
> > > > >> >>>> > >> > > > > sync up once a week. I think we could use time
> to
> > > talk
> > > > >> >>>> through
> > > > >> >>>> > >> design
> > > > >> >>>> > >> > > > > stuff, make sure we are on top of code reviews,
> > talk
> > > > >> through
> > > > >> >>>> any
> > > > >> >>>> > >> > tricky
> > > > >> >>>> > >> > > > > issues, etc.
> > > > >> >>>> > >> > > > >
> > > > >> >>>> > >> > > > > We can make it publicly available so that any
> one
> > > can
> > > > >> follow
> > > > >> >>>> along
> > > > >> >>>> > >> > who
> > > > >> >>>> > >> > > > > likes.
> > > > >> >>>> > >> > > > >
> > > > >> >>>> > >> > > > > Any interest in doing this? If so I'll try to
> set
> > it
> > > > up
> > > > >> >>>> starting
> > > > >> >>>> > >> next
> > > > >> >>>> > >> > > > week.
> > > > >> >>>> > >> > > > >
> > > > >> >>>> > >> > > > > -Jay
> > > > >> >>>> > >> > > > >
> > > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii
> Biletskyi
> > <
> > > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> >>>> > >> > > > >
> > > > >> >>>> > >> > > > > > Hi all,
> > > > >> >>>> > >> > > > > >
> > > > >> >>>> > >> > > > > > I've updated KIP page, fixed / aligned
> document
> > > > >> structure.
> > > > >> >>>> Also I
> > > > >> >>>> > >> > > added
> > > > >> >>>> > >> > > > > > some
> > > > >> >>>> > >> > > > > > very initial proposal for AdminClient so we
> have
> > > > >> something
> > > > >> >>>> to
> > > > >> >>>> > >> start
> > > > >> >>>> > >> > > > from
> > > > >> >>>> > >> > > > > > while
> > > > >> >>>> > >> > > > > > discussing the KIP.
> > > > >> >>>> > >> > > > > >
> > > > >> >>>> > >> > > > > >
> > > > >> >>>> > >> > > > > >
> > > > >> >>>> > >> > > > >
> > > > >> >>>> > >> > > >
> > > > >> >>>> > >> > >
> > > > >> >>>> > >> >
> > > > >> >>>> > >>
> > > > >> >>>>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > >> >>>> > >> > > > > >
> > > > >> >>>> > >> > > > > > Thanks,
> > > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > > >> >>>> > >> > > > > >
> > > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii
> > Biletskyi
> > > <
> > > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> >>>> > >> > > > > >
> > > > >> >>>> > >> > > > > > > Jay,
> > > > >> >>>> > >> > > > > > >
> > > > >> >>>> > >> > > > > > > Re error messages: you are right, in most
> > cases
> > > > >> client
> > > > >> >>>> will
> > > > >> >>>> > >> have
> > > > >> >>>> > >> > > > enough
> > > > >> >>>> > >> > > > > > > context to show descriptive error message.
> My
> > > > >> concern is
> > > > >> >>>> that
> > > > >> >>>> > >> we
> > > > >> >>>> > >> > > will
> > > > >> >>>> > >> > > > > > have
> > > > >> >>>> > >> > > > > > > to
> > > > >> >>>> > >> > > > > > > add lots of new error codes for each
> possible
> > > > >> error. Of
> > > > >> >>>> course,
> > > > >> >>>> > >> > we
> > > > >> >>>> > >> > > > > could
> > > > >> >>>> > >> > > > > > > reuse
> > > > >> >>>> > >> > > > > > > some of existing like
> > > UknownTopicOrPartitionCode,
> > > > >> but we
> > > > >> >>>> will
> > > > >> >>>> > >> > also
> > > > >> >>>> > >> > > > need
> > > > >> >>>> > >> > > > > > to
> > > > >> >>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> > > > >> >>>> TopicConfigInvalid (both
> > > > >> >>>> > >> > for
> > > > >> >>>> > >> > > > > topic
> > > > >> >>>> > >> > > > > > > name and config, and probably user would
> like
> > to
> > > > >> know
> > > > >> >>>> what
> > > > >> >>>> > >> > exactly
> > > > >> >>>> > >> > > > > > > is wrong in his config),
> > > InvalidReplicaAssignment,
> > > > >> >>>> > >> InternalError
> > > > >> >>>> > >> > > > (e.g.
> > > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > > >> >>>> > >> > > > > > > And this is only for TopicCommand, we will
> > also
> > > > >> need to
> > > > >> >>>> add
> > > > >> >>>> > >> > similar
> > > > >> >>>> > >> > > > > stuff
> > > > >> >>>> > >> > > > > > > for
> > > > >> >>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So
> we'll
> > > end
> > > > >> up
> > > > >> >>>> with a
> > > > >> >>>> > >> > large
> > > > >> >>>> > >> > > > list
> > > > >> >>>> > >> > > > > > of
> > > > >> >>>> > >> > > > > > > error codes, used only in Admin protocol.
> > > > >> >>>> > >> > > > > > > Having said that, I agree my proposal is not
> > > > >> consistent
> > > > >> >>>> with
> > > > >> >>>> > >> > other
> > > > >> >>>> > >> > > > > cases.
> > > > >> >>>> > >> > > > > > > Maybe we can find better solution or
> something
> > > > >> >>>> in-between.
> > > > >> >>>> > >> > > > > > >
> > > > >> >>>> > >> > > > > > > Re Hangout chat: I think it is a great idea.
> > > This
> > > > >> way we
> > > > >> >>>> can
> > > > >> >>>> > >> move
> > > > >> >>>> > >> > > on
> > > > >> >>>> > >> > > > > > > faster.
> > > > >> >>>> > >> > > > > > > Let's agree somehow on date/time so people
> can
> > > > join.
> > > > >> >>>> Will work
> > > > >> >>>> > >> > for
> > > > >> >>>> > >> > > me
> > > > >> >>>> > >> > > > > > this
> > > > >> >>>> > >> > > > > > > and
> > > > >> >>>> > >> > > > > > > next week almost anytime if agreed in
> advance.
> > > > >> >>>> > >> > > > > > >
> > > > >> >>>> > >> > > > > > > Thanks,
> > > > >> >>>> > >> > > > > > > Andrii
> > > > >> >>>> > >> > > > > > >
> > > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> > > > >> >>>> > >> jay.kreps@gmail.com>
> > > > >> >>>> > >> > > > > wrote:
> > > > >> >>>> > >> > > > > > >
> > > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > > >> >>>> > >> > > > > > >>
> > > > >> >>>> > >> > > > > > >> Generally we can do good error handling
> > without
> > > > >> needing
> > > > >> >>>> custom
> > > > >> >>>> > >> > > > > > server-side
> > > > >> >>>> > >> > > > > > >> messages. I.e. generally the client has the
> > > > >> context to
> > > > >> >>>> know
> > > > >> >>>> > >> that
> > > > >> >>>> > >> > > if
> > > > >> >>>> > >> > > > it
> > > > >> >>>> > >> > > > > > got
> > > > >> >>>> > >> > > > > > >> an error that the topic doesn't exist to
> say
> > > > >> "Topic X
> > > > >> >>>> doesn't
> > > > >> >>>> > >> > > exist"
> > > > >> >>>> > >> > > > > > >> rather
> > > > >> >>>> > >> > > > > > >> than "error code 14" (or whatever). Maybe
> > there
> > > > are
> > > > >> >>>> specific
> > > > >> >>>> > >> > cases
> > > > >> >>>> > >> > > > > where
> > > > >> >>>> > >> > > > > > >> this is hard? If we want to add server-side
> > > error
> > > > >> >>>> messages we
> > > > >> >>>> > >> > > really
> > > > >> >>>> > >> > > > > do
> > > > >> >>>> > >> > > > > > >> need to do this in a consistent way across
> > the
> > > > >> protocol.
> > > > >> >>>> > >> > > > > > >>
> > > > >> >>>> > >> > > > > > >> I still have a bunch of open questions here
> > > from
> > > > my
> > > > >> >>>> previous
> > > > >> >>>> > >> > > list. I
> > > > >> >>>> > >> > > > > > will
> > > > >> >>>> > >> > > > > > >> be out for the next few days for Strata
> > though.
> > > > >> Maybe
> > > > >> >>>> we could
> > > > >> >>>> > >> > do
> > > > >> >>>> > >> > > a
> > > > >> >>>> > >> > > > > > Google
> > > > >> >>>> > >> > > > > > >> Hangout chat on any open issues some time
> > > towards
> > > > >> the
> > > > >> >>>> end of
> > > > >> >>>> > >> > next
> > > > >> >>>> > >> > > > week
> > > > >> >>>> > >> > > > > > for
> > > > >> >>>> > >> > > > > > >> anyone interested in this ticket? I have a
> > > > feeling
> > > > >> that
> > > > >> >>>> might
> > > > >> >>>> > >> > > > progress
> > > > >> >>>> > >> > > > > > >> things a little faster than email--I think
> we
> > > > >> could talk
> > > > >> >>>> > >> through
> > > > >> >>>> > >> > > > those
> > > > >> >>>> > >> > > > > > >> issues I brought up fairly quickly...
> > > > >> >>>> > >> > > > > > >>
> > > > >> >>>> > >> > > > > > >> -Jay
> > > > >> >>>> > >> > > > > > >>
> > > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii
> > > > Biletskyi <
> > > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > > > >> >>>> > >> > > > > > >>
> > > > >> >>>> > >> > > > > > >> > Hi all,
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >> > I'm trying to address some of the issues
> > > which
> > > > >> were
> > > > >> >>>> > >> mentioned
> > > > >> >>>> > >> > > > > earlier
> > > > >> >>>> > >> > > > > > >> about
> > > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of those was
> about
> > > > >> batching
> > > > >> >>>> > >> > operations.
> > > > >> >>>> > >> > > > What
> > > > >> >>>> > >> > > > > > if
> > > > >> >>>> > >> > > > > > >> we
> > > > >> >>>> > >> > > > > > >> > follow TopicCommand approach and let
> people
> > > > >> specify
> > > > >> >>>> > >> topic-name
> > > > >> >>>> > >> > > by
> > > > >> >>>> > >> > > > > > >> regexp -
> > > > >> >>>> > >> > > > > > >> > would that cover most of the use cases?
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >> > Secondly, is what information should we
> > > > generally
> > > > >> >>>> provide in
> > > > >> >>>> > >> > > Admin
> > > > >> >>>> > >> > > > > > >> > responses.
> > > > >> >>>> > >> > > > > > >> > I realize that Admin commands don't imply
> > > they
> > > > >> will
> > > > >> >>>> be used
> > > > >> >>>> > >> > only
> > > > >> >>>> > >> > > > in
> > > > >> >>>> > >> > > > > > CLI
> > > > >> >>>> > >> > > > > > >> > but,
> > > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a very important
> > > client
> > > > >> of this
> > > > >> >>>> > >> > feature.
> > > > >> >>>> > >> > > In
> > > > >> >>>> > >> > > > > > this
> > > > >> >>>> > >> > > > > > >> > case,
> > > > >> >>>> > >> > > > > > >> > seems logical, we would like to provide
> > users
> > > > >> with
> > > > >> >>>> rich
> > > > >> >>>> > >> > > experience
> > > > >> >>>> > >> > > > > in
> > > > >> >>>> > >> > > > > > >> terms
> > > > >> >>>> > >> > > > > > >> > of
> > > > >> >>>> > >> > > > > > >> > getting results / errors of the executed
> > > > >> commands.
> > > > >> >>>> Usually
> > > > >> >>>> > >> we
> > > > >> >>>> > >> > > > supply
> > > > >> >>>> > >> > > > > > >> with
> > > > >> >>>> > >> > > > > > >> > responses only errorCode, which looks
> very
> > > > >> limiting,
> > > > >> >>>> in case
> > > > >> >>>> > >> > of
> > > > >> >>>> > >> > > > CLI
> > > > >> >>>> > >> > > > > we
> > > > >> >>>> > >> > > > > > >> may
> > > > >> >>>> > >> > > > > > >> > want to print human readable error
> > > description.
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >> > So, taking into account previous item
> about
> > > > >> batching,
> > > > >> >>>> what
> > > > >> >>>> > >> do
> > > > >> >>>> > >> > > you
> > > > >> >>>> > >> > > > > > think
> > > > >> >>>> > >> > > > > > >> > about
> > > > >> >>>> > >> > > > > > >> > having smth like:
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >> > ('create' doesn't support regexp)
> > > > >> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName
> Partitions
> > > > >> Replicas
> > > > >> >>>> > >> > > > > ReplicaAssignment
> > > > >> >>>> > >> > > > > > >> > [Config]
> > > > >> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode
> > > > ErrorDescription
> > > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > > >> >>>> > >> > > > > > >> >   ErrorDescription => string (empty if
> > > > >> successful)
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp
> > > Partitions
> > > > >> >>>> > >> > > ReplicaAssignment
> > > > >> >>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> > > > >> >>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName
> ErrorCode
> > > > >> >>>> ErrorDescription]
> > > > >> >>>> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
> > > > >> >>>> > >> > > > > > >> >   CommandErrorCode => int16
> > > > >> >>>> > >> > > > > > >> >   CommandErrorDescription => string
> > (nonempty
> > > > in
> > > > >> case
> > > > >> >>>> of
> > > > >> >>>> > >> fatal
> > > > >> >>>> > >> > > > > error,
> > > > >> >>>> > >> > > > > > >> e.g.
> > > > >> >>>> > >> > > > > > >> > we couldn't get topics by regexp)
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> > > > >> >>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> > > > >> TopicDescription
> > > > >> >>>> > >> ErrorCode
> > > > >> >>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> > > > >> >>>> CommandErrorDescription
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >> > Also, any thoughts about our discussion
> > > > regarding
> > > > >> >>>> re-routing
> > > > >> >>>> > >> > > > > facility?
> > > > >> >>>> > >> > > > > > >> In
> > > > >> >>>> > >> > > > > > >> > my
> > > > >> >>>> > >> > > > > > >> > understanding, it is like between
> > augmenting
> > > > >> >>>> > >> > > TopicMetadataRequest
> > > > >> >>>> > >> > > > > > >> > (to include at least controllerId) and
> > > > >> implementing
> > > > >> >>>> new
> > > > >> >>>> > >> > generic
> > > > >> >>>> > >> > > > > > >> re-routing
> > > > >> >>>> > >> > > > > > >> > facility so sending messages to
> controller
> > > will
> > > > >> be
> > > > >> >>>> handled
> > > > >> >>>> > >> by
> > > > >> >>>> > >> > > it.
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >> > Thanks,
> > > > >> >>>> > >> > > > > > >> > Andrii Biletskyi
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii
> > > > >> Biletskyi <
> > > > >> >>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >> > > @Guozhang:
> > > > >> >>>> > >> > > > > > >> > > Thanks for your comments, I've answered
> > > some
> > > > of
> > > > >> >>>> those. The
> > > > >> >>>> > >> > > main
> > > > >> >>>> > >> > > > > > thing
> > > > >> >>>> > >> > > > > > >> is
> > > > >> >>>> > >> > > > > > >> > > having merged request for
> > > > >> >>>> create-alter-delete-describe - I
> > > > >> >>>> > >> > > have
> > > > >> >>>> > >> > > > > some
> > > > >> >>>> > >> > > > > > >> > > concerns about this approach.
> > > > >> >>>> > >> > > > > > >> > >
> > > > >> >>>> > >> > > > > > >> > > @*Jay*:
> > > > >> >>>> > >> > > > > > >> > > I see that introduced
> > ClusterMetadaRequest
> > > is
> > > > >> also
> > > > >> >>>> one of
> > > > >> >>>> > >> > the
> > > > >> >>>> > >> > > > > > >> concerns.
> > > > >> >>>> > >> > > > > > >> > We
> > > > >> >>>> > >> > > > > > >> > > can solve it if we implement re-routing
> > > > >> facility.
> > > > >> >>>> But I
> > > > >> >>>> > >> > agree
> > > > >> >>>> > >> > > > with
> > > > >> >>>> > >> > > > > > >> > > Guozhang - it will make clients'
> > internals
> > > a
> > > > >> little
> > > > >> >>>> bit
> > > > >> >>>> > >> > easier
> > > > >> >>>> > >> > > > but
> > > > >> >>>> > >> > > > > > >> this
> > > > >> >>>> > >> > > > > > >> > > seems to be a complex logic to
> implement
> > > and
> > > > >> >>>> support then.
> > > > >> >>>> > >> > > > > > Especially
> > > > >> >>>> > >> > > > > > >> for
> > > > >> >>>> > >> > > > > > >> > > Fetch and Produce (even if we add
> > > re-routing
> > > > >> later
> > > > >> >>>> for
> > > > >> >>>> > >> these
> > > > >> >>>> > >> > > > > > >> requests).
> > > > >> >>>> > >> > > > > > >> > > Also people will tend to avoid this
> > > > re-routing
> > > > >> >>>> facility
> > > > >> >>>> > >> and
> > > > >> >>>> > >> > > hold
> > > > >> >>>> > >> > > > > > local
> > > > >> >>>> > >> > > > > > >> > > cluster cache to ensure their
> > high-priority
> > > > >> requests
> > > > >> >>>> > >> (which
> > > > >> >>>> > >> > > some
> > > > >> >>>> > >> > > > > of
> > > > >> >>>> > >> > > > > > >> the
> > > > >> >>>> > >> > > > > > >> > > admin requests are) not sent to some
> busy
> > > > >> broker
> > > > >> >>>> where
> > > > >> >>>> > >> they
> > > > >> >>>> > >> > > wait
> > > > >> >>>> > >> > > > > to
> > > > >> >>>> > >> > > > > > be
> > > > >> >>>> > >> > > > > > >> > > routed to the correct one.
> > > > >> >>>> > >> > > > > > >> > > As pointed out by Jun here (
> > > > >> >>>> > >> > > > > > >> > >
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >>
> > > > >> >>>> > >> > > > > >
> > > > >> >>>> > >> > > > >
> > > > >> >>>> > >> > > >
> > > > >> >>>> > >> > >
> > > > >> >>>> > >> >
> > > > >> >>>> > >>
> > > > >> >>>>
> > > > >>
> > > >
> > >
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > > >> >>>> > >> > > > > > >> > )
> > > > >> >>>> > >> > > > > > >> > > to solve the issue we might introduce a
> > > > message
> > > > >> >>>> type to
> > > > >> >>>> > >> get
> > > > >> >>>> > >> > > > > cluster
> > > > >> >>>> > >> > > > > > >> > state.
> > > > >> >>>> > >> > > > > > >> > > But I agree we can just update
> > > > >> >>>> TopicMetadataResponse to
> > > > >> >>>> > >> > > include
> > > > >> >>>> > >> > > > > > >> > > controllerId (and probably smth else).
> > > > >> >>>> > >> > > > > > >> > > What are you thougths?
> > > > >> >>>> > >> > > > > > >> > >
> > > > >> >>>> > >> > > > > > >> > > Thanks,
> > > > >> >>>> > >> > > > > > >> > > Andrii
> > > > >> >>>> > >> > > > > > >> > >
> > > > >> >>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM,
> Guozhang
> > > > Wang
> > > > >> <
> > > > >> >>>> > >> > > > > wangguoz@gmail.com>
> > > > >> >>>> > >> > > > > > >> > wrote:
> > > > >> >>>> > >> > > > > > >> > >
> > > > >> >>>> > >> > > > > > >> > >> I think for the topics commands we can
> > > > >> actually
> > > > >> >>>> merge
> > > > >> >>>> > >> > > > > > >> > >> create/alter/delete/describe as one
> > > request
> > > > >> type
> > > > >> >>>> since
> > > > >> >>>> > >> > their
> > > > >> >>>> > >> > > > > > formats
> > > > >> >>>> > >> > > > > > >> are
> > > > >> >>>> > >> > > > > > >> > >> very much similar, and keep
> list-topics
> > > and
> > > > >> others
> > > > >> >>>> like
> > > > >> >>>> > >> > > > > > >> > >> partition-reassignment /
> > > > >> preferred-leader-election
> > > > >> >>>> as
> > > > >> >>>> > >> > > separate
> > > > >> >>>> > >> > > > > > >> request
> > > > >> >>>> > >> > > > > > >> > >> types, I also left some other comments
> > on
> > > > the
> > > > >> RB (
> > > > >> >>>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> > > > >> >>>> > >> > > > > > >> > >>
> > > > >> >>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay
> > > Kreps <
> > > > >> >>>> > >> > > > jay.kreps@gmail.com>
> > > > >> >>>> > >> > > > > > >> wrote:
> > > > >> >>>> > >> > > > > > >> > >>
> > > > >> >>>> > >> > > > > > >> > >> > Yeah I totally agree that we don't
> > want
> > > to
> > > > >> just
> > > > >> >>>> have
> > > > >> >>>> > >> one
> > > > >> >>>> > >> > > "do
> > > > >> >>>> > >> > > > > > admin
> > > > >> >>>> > >> > > > > > >> > >> stuff"
> > > > >> >>>> > >> > > > > > >> > >> > command that has the union of all
> > > > >> parameters.
> > > > >> >>>> > >> > > > > > >> > >> >
> > > > >> >>>> > >> > > > > > >> > >> > What I am saying is that command
> line
> > > > tools
> > > > >> are
> > > > >> >>>> one
> > > > >> >>>> > >> > client
> > > > >> >>>> > >> > > of
> > > > >> >>>> > >> > > > > the
> > > > >> >>>> > >> > > > > > >> > >> > administrative apis, but these will
> be
> > > > used
> > > > >> in a
> > > > >> >>>> number
> > > > >> >>>> > >> > of
> > > > >> >>>> > >> > > > > > >> scenarios
> > > > >> >>>> > >> > > > > > >> > so
> > > > >> >>>> > >> > > > > > >> > >> > they should make logical sense even
> in
> > > the
> > > > >> >>>> absence of
> > > > >> >>>> > >> the
> > > > >> >>>> > >> > > > > command
> > > > >> >>>> > >> > > > > > >> line
> > > > >> >>>> > >> > > > > > >> > >> > tool. Hence comments like trying to
> > > > clarify
> > > > >> the
> > > > >> >>>> > >> > > relationship
> > > > >> >>>> > >> > > > > > >> between
> > > > >> >>>> > >> > > > > > >> > >> > ClusterMetadata and
> > > TopicMetadata...these
> > > > >> kinds
> > > > >> >>>> of
> > > > >> >>>> > >> things
> > > > >> >>>> > >> > > > > really
> > > > >> >>>> > >> > > > > > >> need
> > > > >> >>>> > >> > > > > > >> > >> to be
> > > > >> >>>> > >> > > > > > >> > >> > thought through.
> > > > >> >>>> > >> > > > > > >> > >> >
> > > > >> >>>> > >> > > > > > >> > >> > Hope that makes sense.
> > > > >> >>>> > >> > > > > > >> > >> >
> > > > >> >>>> > >> > > > > > >> > >> > -Jay
> > > > >> >>>> > >> > > > > > >> > >> >
> > > > >> >>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM,
> > Andrii
> > > > >> >>>> Biletskyi <
> > > > >> >>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > >> >>>> > >> > > > > > >> > >> >
> > > > >> >>>> > >> > > > > > >> > >> > > Jay,
> > > > >> >>>> > >> > > > > > >> > >> > >
> > > > >> >>>> > >> > > > > > >> > >> > > Thanks for answering. You
> understood
> > > > >> >>>> correctly, most
> > > > >> >>>> > >> of
> > > > >> >>>> > >> > > my
> > > > >> >>>> > >> > > > > > >> comments
> > > > >> >>>> > >> > > > > > >> > >> were
> > > > >> >>>> > >> > > > > > >> > >> > > related to your point 1) - about
> > "well
> > > > >> >>>> thought-out"
> > > > >> >>>> > >> > apis.
> > > > >> >>>> > >> > > > > Also,
> > > > >> >>>> > >> > > > > > >> yes,
> > > > >> >>>> > >> > > > > > >> > >> as I
> > > > >> >>>> > >> > > > > > >> > >> > > understood we would like to
> > introduce
> > > a
> > > > >> single
> > > > >> >>>> > >> unified
> > > > >> >>>> > >> > > CLI
> > > > >> >>>> > >> > > > > tool
> > > > >> >>>> > >> > > > > > >> with
> > > > >> >>>> > >> > > > > > >> > >> > > centralized server-side request
> > > handling
> > > > >> for
> > > > >> >>>> lots of
> > > > >> >>>> > >> > > > existing
> > > > >> >>>> > >> > > > > > >> ones
> > > > >> >>>> > >> > > > > > >> > >> (incl.
> > > > >> >>>> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> > > > >> >>>> > >> ReassignPartitions,
> > > > >> >>>> > >> > > smth
> > > > >> >>>> > >> > > > > > else
> > > > >> >>>> > >> > > > > > >> if
> > > > >> >>>> > >> > > > > > >> > >> added
> > > > >> >>>> > >> > > > > > >> > >> > > in future). In our previous
> > > discussion (
> > > > >> >>>> > >> > > > > > >> > >> > >
> > > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694)
> > > > >> >>>> > >> > people
> > > > >> >>>> > >> > > > > said
> > > > >> >>>> > >> > > > > > >> > they'd
> > > > >> >>>> > >> > > > > > >> > >> > > rather
> > > > >> >>>> > >> > > > > > >> > >> > > have a separate message for each
> > > > command,
> > > > >> so,
> > > > >> >>>> yes,
> > > > >> >>>> > >> this
> > > > >> >>>> > >> > > > way I
> > > > >> >>>> > >> > > > > > >> came
> > > > >> >>>> > >> > > > > > >> > to
> > > > >> >>>> > >> > > > > > >> > >> 1-1
> > > > >> >>>> > >> > > > > > >> > >> > > mapping between commands in the
> tool
> > > and
> > > > >> >>>> protocol
> > > > >> >>>> > >> > > > additions.
> > > > >> >>>> > >> > > > > > But
> > > > >> >>>> > >> > > > > > >> I
> > > > >> >>>> > >> > > > > > >> > >> might
> > > > >> >>>> > >> > > > > > >> > >> > be
> > > > >> >>>> > >> > > > > > >> > >> > > wrong.
> > > > >> >>>> > >> > > > > > >> > >> > > At the end I just try to start
> > > > discussion
> > > > >> how
> > > > >> >>>> at
> > > > >> >>>> > >> least
> > > > >> >>>> > >> > > > > > generally
> > > > >> >>>> > >> > > > > > >> > this
> > > > >> >>>> > >> > > > > > >> > >> > > protocol should look like.
> > > > >> >>>> > >> > > > > > >> > >> > >
> > > > >> >>>> > >> > > > > > >> > >> > > Thanks,
> > > > >> >>>> > >> > > > > > >> > >> > > Andrii
> > > > >> >>>> > >> > > > > > >> > >> > >
> > > > >> >>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM,
> > Jay
> > > > >> Kreps <
> > > > >> >>>> > >> > > > > > jay.kreps@gmail.com
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >> > >> wrote:
> > > > >> >>>> > >> > > > > > >> > >> > >
> > > > >> >>>> > >> > > > > > >> > >> > > > Hey Andrii,
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > To answer your earlier question
> we
> > > > just
> > > > >> >>>> really
> > > > >> >>>> > >> can't
> > > > >> >>>> > >> > be
> > > > >> >>>> > >> > > > > > adding
> > > > >> >>>> > >> > > > > > >> any
> > > > >> >>>> > >> > > > > > >> > >> more
> > > > >> >>>> > >> > > > > > >> > >> > > > scala protocol objects. These
> > things
> > > > are
> > > > >> >>>> super hard
> > > > >> >>>> > >> > to
> > > > >> >>>> > >> > > > > > maintain
> > > > >> >>>> > >> > > > > > >> > >> because
> > > > >> >>>> > >> > > > > > >> > >> > > > they hand code the byte parsing
> > and
> > > > >> don't
> > > > >> >>>> have good
> > > > >> >>>> > >> > > > > > versioning
> > > > >> >>>> > >> > > > > > >> > >> support.
> > > > >> >>>> > >> > > > > > >> > >> > > > Since we are already planning on
> > > > >> converting
> > > > >> >>>> we
> > > > >> >>>> > >> > > definitely
> > > > >> >>>> > >> > > > > > don't
> > > > >> >>>> > >> > > > > > >> > >> want to
> > > > >> >>>> > >> > > > > > >> > >> > > add
> > > > >> >>>> > >> > > > > > >> > >> > > > a ton more of these--they are
> > total
> > > > tech
> > > > >> >>>> debt.
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > What does it mean that the
> changes
> > > are
> > > > >> >>>> isolated
> > > > >> >>>> > >> from
> > > > >> >>>> > >> > > the
> > > > >> >>>> > >> > > > > > >> current
> > > > >> >>>> > >> > > > > > >> > >> code
> > > > >> >>>> > >> > > > > > >> > >> > > base?
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > I actually didn't understand the
> > > > >> remaining
> > > > >> >>>> > >> comments,
> > > > >> >>>> > >> > > > which
> > > > >> >>>> > >> > > > > of
> > > > >> >>>> > >> > > > > > >> the
> > > > >> >>>> > >> > > > > > >> > >> > points
> > > > >> >>>> > >> > > > > > >> > >> > > > are you responding to?
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > Maybe one sticking point here is
> > > that
> > > > it
> > > > >> >>>> seems like
> > > > >> >>>> > >> > you
> > > > >> >>>> > >> > > > > want
> > > > >> >>>> > >> > > > > > to
> > > > >> >>>> > >> > > > > > >> > make
> > > > >> >>>> > >> > > > > > >> > >> > some
> > > > >> >>>> > >> > > > > > >> > >> > > > kind of tool, and you have made
> a
> > > 1-1
> > > > >> mapping
> > > > >> >>>> > >> between
> > > > >> >>>> > >> > > > > > commands
> > > > >> >>>> > >> > > > > > >> you
> > > > >> >>>> > >> > > > > > >> > >> > > imagine
> > > > >> >>>> > >> > > > > > >> > >> > > > in the tool and protocol
> > additions.
> > > I
> > > > >> want
> > > > >> >>>> to make
> > > > >> >>>> > >> > sure
> > > > >> >>>> > >> > > > we
> > > > >> >>>> > >> > > > > > >> don't
> > > > >> >>>> > >> > > > > > >> > do
> > > > >> >>>> > >> > > > > > >> > >> > that.
> > > > >> >>>> > >> > > > > > >> > >> > > > The protocol needs to be really
> > > really
> > > > >> well
> > > > >> >>>> thought
> > > > >> >>>> > >> > out
> > > > >> >>>> > >> > > > > > against
> > > > >> >>>> > >> > > > > > >> > many
> > > > >> >>>> > >> > > > > > >> > >> > use
> > > > >> >>>> > >> > > > > > >> > >> > > > cases so it should make perfect
> > > > logical
> > > > >> >>>> sense in
> > > > >> >>>> > >> the
> > > > >> >>>> > >> > > > > absence
> > > > >> >>>> > >> > > > > > of
> > > > >> >>>> > >> > > > > > >> > >> knowing
> > > > >> >>>> > >> > > > > > >> > >> > > the
> > > > >> >>>> > >> > > > > > >> > >> > > > command line tool, right?
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > -Jay
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57
> AM,
> > > > Andrii
> > > > >> >>>> Biletskyi
> > > > >> >>>> > >> <
> > > > >> >>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly>
> > wrote:
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > Hey Jay,
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > I would like to continue this
> > > > >> discussion
> > > > >> >>>> as it
> > > > >> >>>> > >> seem
> > > > >> >>>> > >> > > > there
> > > > >> >>>> > >> > > > > > is
> > > > >> >>>> > >> > > > > > >> no
> > > > >> >>>> > >> > > > > > >> > >> > > progress
> > > > >> >>>> > >> > > > > > >> > >> > > > > here.
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > First of all, could you please
> > > > explain
> > > > >> >>>> what did
> > > > >> >>>> > >> you
> > > > >> >>>> > >> > > > mean
> > > > >> >>>> > >> > > > > in
> > > > >> >>>> > >> > > > > > >> 2?
> > > > >> >>>> > >> > > > > > >> > How
> > > > >> >>>> > >> > > > > > >> > >> > > > exactly
> > > > >> >>>> > >> > > > > > >> > >> > > > > are we going to migrate to the
> > new
> > > > >> java
> > > > >> >>>> protocol
> > > > >> >>>> > >> > > > > > definitions.
> > > > >> >>>> > >> > > > > > >> > And
> > > > >> >>>> > >> > > > > > >> > >> why
> > > > >> >>>> > >> > > > > > >> > >> > > > it's
> > > > >> >>>> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > I agree with you, this feature
> > > > >> includes
> > > > >> >>>> lots of
> > > > >> >>>> > >> > > stuff,
> > > > >> >>>> > >> > > > > but
> > > > >> >>>> > >> > > > > > >> > >> thankfully
> > > > >> >>>> > >> > > > > > >> > >> > > > > almost all changes are
> isolated
> > > from
> > > > >> the
> > > > >> >>>> current
> > > > >> >>>> > >> > code
> > > > >> >>>> > >> > > > > base,
> > > > >> >>>> > >> > > > > > >> > >> > > > > so the main thing, I think, we
> > > need
> > > > to
> > > > >> >>>> agree is
> > > > >> >>>> > >> > RQ/RP
> > > > >> >>>> > >> > > > > > format.
> > > > >> >>>> > >> > > > > > >> > >> > > > > So how can we start discussion
> > > about
> > > > >> the
> > > > >> >>>> concrete
> > > > >> >>>> > >> > > > > messages
> > > > >> >>>> > >> > > > > > >> > format?
> > > > >> >>>> > >> > > > > > >> > >> > > > > Can we take (
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> > >> > > > > > >> > >> > >
> > > > >> >>>> > >> > > > > > >> > >> >
> > > > >> >>>> > >> > > > > > >> > >>
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >>
> > > > >> >>>> > >> > > > > >
> > > > >> >>>> > >> > > > >
> > > > >> >>>> > >> > > >
> > > > >> >>>> > >> > >
> > > > >> >>>> > >> >
> > > > >> >>>> > >>
> > > > >> >>>>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > > >> >>>> > >> > > > > > >> > >> > > > > as starting point?
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > We had some doubts earlier
> > whether
> > > > it
> > > > >> worth
> > > > >> >>>> > >> > > introducing
> > > > >> >>>> > >> > > > > one
> > > > >> >>>> > >> > > > > > >> > >> generic
> > > > >> >>>> > >> > > > > > >> > >> > > Admin
> > > > >> >>>> > >> > > > > > >> > >> > > > > Request for all commands (
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694
> > > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > > >> >>>> > >> > > > > > >> > >> > > > > but then everybody agreed it
> > would
> > > > be
> > > > >> >>>> better to
> > > > >> >>>> > >> > have
> > > > >> >>>> > >> > > > > > separate
> > > > >> >>>> > >> > > > > > >> > >> message
> > > > >> >>>> > >> > > > > > >> > >> > > for
> > > > >> >>>> > >> > > > > > >> > >> > > > > each admin command. The
> Request
> > > part
> > > > >> is
> > > > >> >>>> really
> > > > >> >>>> > >> > > dictated
> > > > >> >>>> > >> > > > > > from
> > > > >> >>>> > >> > > > > > >> the
> > > > >> >>>> > >> > > > > > >> > >> > > command
> > > > >> >>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments
> > > > itself,
> > > > >> so
> > > > >> >>>> the
> > > > >> >>>> > >> > proposed
> > > > >> >>>> > >> > > > > > version
> > > > >> >>>> > >> > > > > > >> > >> should
> > > > >> >>>> > >> > > > > > >> > >> > be
> > > > >> >>>> > >> > > > > > >> > >> > > > > fine (let's put aside for now
> > > > remarks
> > > > >> about
> > > > >> >>>> > >> > Optional
> > > > >> >>>> > >> > > > > type,
> > > > >> >>>> > >> > > > > > >> > >> batching,
> > > > >> >>>> > >> > > > > > >> > >> > > > > configs normalization - I
> agree
> > > with
> > > > >> all of
> > > > >> >>>> > >> them).
> > > > >> >>>> > >> > > > > > >> > >> > > > > So the second part is
> Response.
> > I
> > > > see
> > > > >> >>>> there are
> > > > >> >>>> > >> two
> > > > >> >>>> > >> > > > cases
> > > > >> >>>> > >> > > > > > >> here.
> > > > >> >>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> > > > >> Create/Alter/... ;
> > > > >> >>>> b)
> > > > >> >>>> > >> "Get"
> > > > >> >>>> > >> > > > > > requests -
> > > > >> >>>> > >> > > > > > >> > >> > > > > List/Describe...
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > a) should only hold request
> > result
> > > > >> >>>> (regardless
> > > > >> >>>> > >> what
> > > > >> >>>> > >> > > we
> > > > >> >>>> > >> > > > > > decide
> > > > >> >>>> > >> > > > > > >> > >> about
> > > > >> >>>> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
> > > > >> execution).
> > > > >> >>>> > >> > > > > > >> > >> > > > > Usually we provide error code
> in
> > > > >> response
> > > > >> >>>> but
> > > > >> >>>> > >> since
> > > > >> >>>> > >> > > we
> > > > >> >>>> > >> > > > > will
> > > > >> >>>> > >> > > > > > >> use
> > > > >> >>>> > >> > > > > > >> > >> this
> > > > >> >>>> > >> > > > > > >> > >> > in
> > > > >> >>>> > >> > > > > > >> > >> > > > > interactive shell we need some
> > > human
> > > > >> >>>> readable
> > > > >> >>>> > >> error
> > > > >> >>>> > >> > > > > > >> description
> > > > >> >>>> > >> > > > > > >> > -
> > > > >> >>>> > >> > > > > > >> > >> so
> > > > >> >>>> > >> > > > > > >> > >> > I
> > > > >> >>>> > >> > > > > > >> > >> > > > > added errorDesription field
> > where
> > > > you
> > > > >> can
> > > > >> >>>> at
> > > > >> >>>> > >> least
> > > > >> >>>> > >> > > > leave
> > > > >> >>>> > >> > > > > > >> > >> > > > > exception.getMessage.
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > b) in addition to previous
> item
> > > > >> message
> > > > >> >>>> should
> > > > >> >>>> > >> hold
> > > > >> >>>> > >> > > > > command
> > > > >> >>>> > >> > > > > > >> > >> specific
> > > > >> >>>> > >> > > > > > >> > >> > > > > response data. We can discuss
> in
> > > > >> detail
> > > > >> >>>> each of
> > > > >> >>>> > >> > them
> > > > >> >>>> > >> > > > but
> > > > >> >>>> > >> > > > > > >> let's
> > > > >> >>>> > >> > > > > > >> > for
> > > > >> >>>> > >> > > > > > >> > >> > now
> > > > >> >>>> > >> > > > > > >> > >> > > > > agree about the overall
> pattern.
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > Thanks,
> > > > >> >>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59
> AM,
> > > Jay
> > > > >> Kreps
> > > > >> >>>> <
> > > > >> >>>> > >> > > > > > >> jay.kreps@gmail.com
> > > > >> >>>> > >> > > > > > >> > >
> > > > >> >>>> > >> > > > > > >> > >> > > wrote:
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > Hey Joe,
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > This is great. A few
> comments
> > on
> > > > >> KIP-4
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > 1. This is much needed
> > > > >> functionality,
> > > > >> >>>> but there
> > > > >> >>>> > >> > > are a
> > > > >> >>>> > >> > > > > lot
> > > > >> >>>> > >> > > > > > >> of
> > > > >> >>>> > >> > > > > > >> > >> the so
> > > > >> >>>> > >> > > > > > >> > >> > > > let's
> > > > >> >>>> > >> > > > > > >> > >> > > > > > really think these protocols
> > > > >> through. We
> > > > >> >>>> really
> > > > >> >>>> > >> > > want
> > > > >> >>>> > >> > > > to
> > > > >> >>>> > >> > > > > > >> end up
> > > > >> >>>> > >> > > > > > >> > >> > with a
> > > > >> >>>> > >> > > > > > >> > >> > > > set
> > > > >> >>>> > >> > > > > > >> > >> > > > > > of well thought-out,
> > orthoganol
> > > > >> apis.
> > > > >> >>>> For this
> > > > >> >>>> > >> > > > reason I
> > > > >> >>>> > >> > > > > > >> think
> > > > >> >>>> > >> > > > > > >> > >> it is
> > > > >> >>>> > >> > > > > > >> > >> > > > > really
> > > > >> >>>> > >> > > > > > >> > >> > > > > > important to think through
> the
> > > end
> > > > >> state
> > > > >> >>>> even
> > > > >> >>>> > >> if
> > > > >> >>>> > >> > > that
> > > > >> >>>> > >> > > > > > >> includes
> > > > >> >>>> > >> > > > > > >> > >> APIs
> > > > >> >>>> > >> > > > > > >> > >> > > we
> > > > >> >>>> > >> > > > > > >> > >> > > > > > won't implement in the first
> > > > phase.
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > 2. Let's please please
> please
> > > wait
> > > > >> until
> > > > >> >>>> we
> > > > >> >>>> > >> have
> > > > >> >>>> > >> > > > > switched
> > > > >> >>>> > >> > > > > > >> the
> > > > >> >>>> > >> > > > > > >> > >> > server
> > > > >> >>>> > >> > > > > > >> > >> > > > over
> > > > >> >>>> > >> > > > > > >> > >> > > > > > to the new java protocol
> > > > >> definitions. If
> > > > >> >>>> we add
> > > > >> >>>> > >> > > > upteen
> > > > >> >>>> > >> > > > > > >> more ad
> > > > >> >>>> > >> > > > > > >> > >> hoc
> > > > >> >>>> > >> > > > > > >> > >> > > > scala
> > > > >> >>>> > >> > > > > > >> > >> > > > > > objects that is just
> > generating
> > > > more
> > > > >> >>>> work for
> > > > >> >>>> > >> the
> > > > >> >>>> > >> > > > > > >> conversion
> > > > >> >>>> > >> > > > > > >> > we
> > > > >> >>>> > >> > > > > > >> > >> > know
> > > > >> >>>> > >> > > > > > >> > >> > > we
> > > > >> >>>> > >> > > > > > >> > >> > > > > > have to do.
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > 3. This proposal introduces
> a
> > > new
> > > > >> type of
> > > > >> >>>> > >> > optional
> > > > >> >>>> > >> > > > > > >> parameter.
> > > > >> >>>> > >> > > > > > >> > >> This
> > > > >> >>>> > >> > > > > > >> > >> > is
> > > > >> >>>> > >> > > > > > >> > >> > > > > > inconsistent with everything
> > > else
> > > > >> in the
> > > > >> >>>> > >> protocol
> > > > >> >>>> > >> > > > where
> > > > >> >>>> > >> > > > > > we
> > > > >> >>>> > >> > > > > > >> use
> > > > >> >>>> > >> > > > > > >> > >> -1
> > > > >> >>>> > >> > > > > > >> > >> > or
> > > > >> >>>> > >> > > > > > >> > >> > > > some
> > > > >> >>>> > >> > > > > > >> > >> > > > > > other marker value. You
> could
> > > > argue
> > > > >> >>>> either way
> > > > >> >>>> > >> > but
> > > > >> >>>> > >> > > > > let's
> > > > >> >>>> > >> > > > > > >> stick
> > > > >> >>>> > >> > > > > > >> > >> with
> > > > >> >>>> > >> > > > > > >> > >> > > > that
> > > > >> >>>> > >> > > > > > >> > >> > > > > > for consistency. For clients
> > > that
> > > > >> >>>> implemented
> > > > >> >>>> > >> the
> > > > >> >>>> > >> > > > > > protocol
> > > > >> >>>> > >> > > > > > >> in
> > > > >> >>>> > >> > > > > > >> > a
> > > > >> >>>> > >> > > > > > >> > >> > > better
> > > > >> >>>> > >> > > > > > >> > >> > > > > way
> > > > >> >>>> > >> > > > > > >> > >> > > > > > than our scala code these
> > basic
> > > > >> >>>> primitives are
> > > > >> >>>> > >> > hard
> > > > >> >>>> > >> > > > to
> > > > >> >>>> > >> > > > > > >> change.
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This
> seems
> > > to
> > > > >> >>>> duplicate
> > > > >> >>>> > >> > > > > > >> > TopicMetadataRequest
> > > > >> >>>> > >> > > > > > >> > >> > > which
> > > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > > >> >>>> > >> > > > > > >> > >> > > > > > brokers, topics, and
> > > partitions. I
> > > > >> think
> > > > >> >>>> we
> > > > >> >>>> > >> > should
> > > > >> >>>> > >> > > > > rename
> > > > >> >>>> > >> > > > > > >> that
> > > > >> >>>> > >> > > > > > >> > >> > > request
> > > > >> >>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or
> > just
> > > > >> >>>> > >> MetadataRequest)
> > > > >> >>>> > >> > > and
> > > > >> >>>> > >> > > > > > >> include
> > > > >> >>>> > >> > > > > > >> > >> the id
> > > > >> >>>> > >> > > > > > >> > >> > > of
> > > > >> >>>> > >> > > > > > >> > >> > > > > the
> > > > >> >>>> > >> > > > > > >> > >> > > > > > controller. Or are there
> other
> > > > >> things we
> > > > >> >>>> could
> > > > >> >>>> > >> > add
> > > > >> >>>> > >> > > > > here?
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try
> > to
> > > > >> make a
> > > > >> >>>> lot of
> > > > >> >>>> > >> > > > requests
> > > > >> >>>> > >> > > > > > that
> > > > >> >>>> > >> > > > > > >> > can
> > > > >> >>>> > >> > > > > > >> > >> > only
> > > > >> >>>> > >> > > > > > >> > >> > > go
> > > > >> >>>> > >> > > > > > >> > >> > > > > to
> > > > >> >>>> > >> > > > > > >> > >> > > > > > particular nodes. This adds
> a
> > > lot
> > > > of
> > > > >> >>>> burden for
> > > > >> >>>> > >> > > > client
> > > > >> >>>> > >> > > > > > >> > >> > > implementations
> > > > >> >>>> > >> > > > > > >> > >> > > > > (it
> > > > >> >>>> > >> > > > > > >> > >> > > > > > sounds easy but each
> discovery
> > > can
> > > > >> fail
> > > > >> >>>> in many
> > > > >> >>>> > >> > > parts
> > > > >> >>>> > >> > > > > so
> > > > >> >>>> > >> > > > > > it
> > > > >> >>>> > >> > > > > > >> > >> ends up
> > > > >> >>>> > >> > > > > > >> > >> > > > > being a
> > > > >> >>>> > >> > > > > > >> > >> > > > > > full state machine to do
> > > right). I
> > > > >> think
> > > > >> >>>> we
> > > > >> >>>> > >> > should
> > > > >> >>>> > >> > > > > > consider
> > > > >> >>>> > >> > > > > > >> > >> making
> > > > >> >>>> > >> > > > > > >> > >> > > > admin
> > > > >> >>>> > >> > > > > > >> > >> > > > > > commands and ideally as many
> > of
> > > > the
> > > > >> >>>> other apis
> > > > >> >>>> > >> as
> > > > >> >>>> > >> > > > > > possible
> > > > >> >>>> > >> > > > > > >> > >> > available
> > > > >> >>>> > >> > > > > > >> > >> > > on
> > > > >> >>>> > >> > > > > > >> > >> > > > > all
> > > > >> >>>> > >> > > > > > >> > >> > > > > > brokers and just redirect to
> > the
> > > > >> >>>> controller on
> > > > >> >>>> > >> > the
> > > > >> >>>> > >> > > > > broker
> > > > >> >>>> > >> > > > > > >> > side.
> > > > >> >>>> > >> > > > > > >> > >> > > Perhaps
> > > > >> >>>> > >> > > > > > >> > >> > > > > > there would be a general way
> > to
> > > > >> >>>> encapsulate
> > > > >> >>>> > >> this
> > > > >> >>>> > >> > > > > > re-routing
> > > > >> >>>> > >> > > > > > >> > >> > behavior.
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > 6. We should probably
> > normalize
> > > > the
> > > > >> key
> > > > >> >>>> value
> > > > >> >>>> > >> > pairs
> > > > >> >>>> > >> > > > > used
> > > > >> >>>> > >> > > > > > >> for
> > > > >> >>>> > >> > > > > > >> > >> > configs
> > > > >> >>>> > >> > > > > > >> > >> > > > > rather
> > > > >> >>>> > >> > > > > > >> > >> > > > > > than embedding a new
> > formatting.
> > > > So
> > > > >> two
> > > > >> >>>> strings
> > > > >> >>>> > >> > > > rather
> > > > >> >>>> > >> > > > > > than
> > > > >> >>>> > >> > > > > > >> > one
> > > > >> >>>> > >> > > > > > >> > >> > with
> > > > >> >>>> > >> > > > > > >> > >> > > an
> > > > >> >>>> > >> > > > > > >> > >> > > > > > internal equals sign.
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of
> > these
> > > > >> APIs
> > > > >> >>>> that the
> > > > >> >>>> > >> > > > command
> > > > >> >>>> > >> > > > > > has
> > > > >> >>>> > >> > > > > > >> > >> begun or
> > > > >> >>>> > >> > > > > > >> > >> > > > that
> > > > >> >>>> > >> > > > > > >> > >> > > > > > the command has been
> > completed?
> > > It
> > > > >> is a
> > > > >> >>>> lot
> > > > >> >>>> > >> more
> > > > >> >>>> > >> > > > usable
> > > > >> >>>> > >> > > > > > if
> > > > >> >>>> > >> > > > > > >> the
> > > > >> >>>> > >> > > > > > >> > >> > > command
> > > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > > >> >>>> > >> > > > > > >> > >> > > > > > been completed so you know
> > that
> > > if
> > > > >> you
> > > > >> >>>> create a
> > > > >> >>>> > >> > > topic
> > > > >> >>>> > >> > > > > and
> > > > >> >>>> > >> > > > > > >> then
> > > > >> >>>> > >> > > > > > >> > >> > > publish
> > > > >> >>>> > >> > > > > > >> > >> > > > to
> > > > >> >>>> > >> > > > > > >> > >> > > > > > it you won't get an
> exception
> > > > about
> > > > >> >>>> there being
> > > > >> >>>> > >> > no
> > > > >> >>>> > >> > > > such
> > > > >> >>>> > >> > > > > > >> topic.
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list
> > > topics
> > > > >> >>>> duplicate a
> > > > >> >>>> > >> lot
> > > > >> >>>> > >> > > of
> > > > >> >>>> > >> > > > > > stuff
> > > > >> >>>> > >> > > > > > >> in
> > > > >> >>>> > >> > > > > > >> > >> the
> > > > >> >>>> > >> > > > > > >> > >> > > > > metadata
> > > > >> >>>> > >> > > > > > >> > >> > > > > > request. Is there a reason
> to
> > > give
> > > > >> back
> > > > >> >>>> topics
> > > > >> >>>> > >> > > marked
> > > > >> >>>> > >> > > > > for
> > > > >> >>>> > >> > > > > > >> > >> > deletion? I
> > > > >> >>>> > >> > > > > > >> > >> > > > > feel
> > > > >> >>>> > >> > > > > > >> > >> > > > > > like if we just make the
> > > > >> post-condition
> > > > >> >>>> of the
> > > > >> >>>> > >> > > delete
> > > > >> >>>> > >> > > > > > >> command
> > > > >> >>>> > >> > > > > > >> > be
> > > > >> >>>> > >> > > > > > >> > >> > that
> > > > >> >>>> > >> > > > > > >> > >> > > > the
> > > > >> >>>> > >> > > > > > >> > >> > > > > > topic is deleted that will
> get
> > > rid
> > > > >> of
> > > > >> >>>> the need
> > > > >> >>>> > >> > for
> > > > >> >>>> > >> > > > this
> > > > >> >>>> > >> > > > > > >> right?
> > > > >> >>>> > >> > > > > > >> > >> And
> > > > >> >>>> > >> > > > > > >> > >> > it
> > > > >> >>>> > >> > > > > > >> > >> > > > > will
> > > > >> >>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > 9. Should we consider
> batching
> > > > these
> > > > >> >>>> requests?
> > > > >> >>>> > >> We
> > > > >> >>>> > >> > > > have
> > > > >> >>>> > >> > > > > > >> > generally
> > > > >> >>>> > >> > > > > > >> > >> > > tried
> > > > >> >>>> > >> > > > > > >> > >> > > > to
> > > > >> >>>> > >> > > > > > >> > >> > > > > > allow multiple operations to
> > be
> > > > >> batched.
> > > > >> >>>> My
> > > > >> >>>> > >> > > suspicion
> > > > >> >>>> > >> > > > > is
> > > > >> >>>> > >> > > > > > >> that
> > > > >> >>>> > >> > > > > > >> > >> > without
> > > > >> >>>> > >> > > > > > >> > >> > > > > this
> > > > >> >>>> > >> > > > > > >> > >> > > > > > we will get a lot of code
> that
> > > > does
> > > > >> >>>> something
> > > > >> >>>> > >> > like
> > > > >> >>>> > >> > > > > > >> > >> > > > > >    for(topic:
> > > > >> adminClient.listTopics())
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >>  adminClient.describeTopic(topic)
> > > > >> >>>> > >> > > > > > >> > >> > > > > > this code will work great
> when
> > > you
> > > > >> test
> > > > >> >>>> on 5
> > > > >> >>>> > >> > topics
> > > > >> >>>> > >> > > > but
> > > > >> >>>> > >> > > > > > >> not do
> > > > >> >>>> > >> > > > > > >> > >> as
> > > > >> >>>> > >> > > > > > >> > >> > > well
> > > > >> >>>> > >> > > > > > >> > >> > > > if
> > > > >> >>>> > >> > > > > > >> > >> > > > > > you have 50k.
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > 10. I think we should also
> > > discuss
> > > > >> how
> > > > >> >>>> we want
> > > > >> >>>> > >> to
> > > > >> >>>> > >> > > > > expose
> > > > >> >>>> > >> > > > > > a
> > > > >> >>>> > >> > > > > > >> > >> > > programmatic
> > > > >> >>>> > >> > > > > > >> > >> > > > > JVM
> > > > >> >>>> > >> > > > > > >> > >> > > > > > client api for these
> > operations.
> > > > >> >>>> Currently
> > > > >> >>>> > >> people
> > > > >> >>>> > >> > > > rely
> > > > >> >>>> > >> > > > > on
> > > > >> >>>> > >> > > > > > >> > >> > AdminUtils
> > > > >> >>>> > >> > > > > > >> > >> > > > > which
> > > > >> >>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I think
> we
> > > > >> probably
> > > > >> >>>> need
> > > > >> >>>> > >> > > another
> > > > >> >>>> > >> > > > > > client
> > > > >> >>>> > >> > > > > > >> > >> under
> > > > >> >>>> > >> > > > > > >> > >> > > > > clients/
> > > > >> >>>> > >> > > > > > >> > >> > > > > > that exposes administrative
> > > > >> >>>> functionality. We
> > > > >> >>>> > >> > will
> > > > >> >>>> > >> > > > need
> > > > >> >>>> > >> > > > > > >> this
> > > > >> >>>> > >> > > > > > >> > >> just
> > > > >> >>>> > >> > > > > > >> > >> > to
> > > > >> >>>> > >> > > > > > >> > >> > > > > > properly test the new apis,
> I
> > > > >> suspect. We
> > > > >> >>>> > >> should
> > > > >> >>>> > >> > > > figure
> > > > >> >>>> > >> > > > > > out
> > > > >> >>>> > >> > > > > > >> > that
> > > > >> >>>> > >> > > > > > >> > >> > API.
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > 11. The other information
> that
> > > > >> would be
> > > > >> >>>> really
> > > > >> >>>> > >> > > useful
> > > > >> >>>> > >> > > > > to
> > > > >> >>>> > >> > > > > > >> get
> > > > >> >>>> > >> > > > > > >> > >> would
> > > > >> >>>> > >> > > > > > >> > >> > be
> > > > >> >>>> > >> > > > > > >> > >> > > > > > information about
> > > partitions--how
> > > > >> much
> > > > >> >>>> data is
> > > > >> >>>> > >> in
> > > > >> >>>> > >> > > the
> > > > >> >>>> > >> > > > > > >> > partition,
> > > > >> >>>> > >> > > > > > >> > >> > what
> > > > >> >>>> > >> > > > > > >> > >> > > > are
> > > > >> >>>> > >> > > > > > >> > >> > > > > > the segment offsets, what is
> > the
> > > > >> log-end
> > > > >> >>>> offset
> > > > >> >>>> > >> > > (i.e.
> > > > >> >>>> > >> > > > > > last
> > > > >> >>>> > >> > > > > > >> > >> offset),
> > > > >> >>>> > >> > > > > > >> > >> > > > what
> > > > >> >>>> > >> > > > > > >> > >> > > > > is
> > > > >> >>>> > >> > > > > > >> > >> > > > > > the compaction point, etc. I
> > > think
> > > > >> that
> > > > >> >>>> done
> > > > >> >>>> > >> > right
> > > > >> >>>> > >> > > > this
> > > > >> >>>> > >> > > > > > >> would
> > > > >> >>>> > >> > > > > > >> > be
> > > > >> >>>> > >> > > > > > >> > >> > the
> > > > >> >>>> > >> > > > > > >> > >> > > > > > successor to the very
> awkward
> > > > >> >>>> OffsetRequest we
> > > > >> >>>> > >> > have
> > > > >> >>>> > >> > > > > > today.
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > -Jay
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at
> 10:27
> > > PM,
> > > > >> Joe
> > > > >> >>>> Stein <
> > > > >> >>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
> > > > >> >>>> > >> > > > > > >> > >> > > > > wrote:
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> > >> > > > > > >> > >> > >
> > > > >> >>>> > >> > > > > > >> > >> >
> > > > >> >>>> > >> > > > > > >> > >>
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >>
> > > > >> >>>> > >> > > > > >
> > > > >> >>>> > >> > > > >
> > > > >> >>>> > >> > > >
> > > > >> >>>> > >> > >
> > > > >> >>>> > >> >
> > > > >> >>>> > >>
> > > > >> >>>>
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > > JIRA
> > > > >> >>>> > >> > > > >
> https://issues.apache.org/jira/browse/KAFKA-1694
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > >> >>>> /*******************************************
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal
> > Consultant
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source
> > Security
> > > > LLC
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >  Twitter:
> @allthingshadoop <
> > > > >> >>>> > >> > > > > > >> > >> >
> > http://www.twitter.com/allthingshadoop
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > >> >>>> ********************************************/
> > > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > > >
> > > > >> >>>> > >> > > > > > >> > >> > > >
> > > > >> >>>> > >> > > > > > >> > >> > >
> > > > >> >>>> > >> > > > > > >> > >> >
> > > > >> >>>> > >> > > > > > >> > >>
> > > > >> >>>> > >> > > > > > >> > >>
> > > > >> >>>> > >> > > > > > >> > >>
> > > > >> >>>> > >> > > > > > >> > >> --
> > > > >> >>>> > >> > > > > > >> > >> -- Guozhang
> > > > >> >>>> > >> > > > > > >> > >>
> > > > >> >>>> > >> > > > > > >> > >
> > > > >> >>>> > >> > > > > > >> > >
> > > > >> >>>> > >> > > > > > >> >
> > > > >> >>>> > >> > > > > > >>
> > > > >> >>>> > >> > > > > > >
> > > > >> >>>> > >> > > > > > >
> > > > >> >>>> > >> > > > > >
> > > > >> >>>> > >> > > > >
> > > > >> >>>> > >> > > >
> > > > >> >>>> > >> > >
> > > > >> >>>> > >> >
> > > > >> >>>> > >> >
> > > > >> >>>> > >> >
> > > > >> >>>> > >> > --
> > > > >> >>>> > >> > Jeff Holoman
> > > > >> >>>> > >> > Systems Engineer
> > > > >> >>>> > >> >
> > > > >> >>>> > >>
> > > > >> >>>>
> > > > >> >>>>
> > > > >> >>>
> > > > >> >>>
> > > > >> >>> --
> > > > >> >>> -- Guozhang
> > > > >> >>>
> > > > >> >>
> > > > >> >>
> > > > >> >>
> > > > >> >> --
> > > > >> >> -- Guozhang
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
Andrii,

101. That's what I was thinking too, but it may not be that simple. In
TopicMetadataRequest_V1,
we can let it not trigger auto topic creation. Then, in the producer side,
if it gets an UnknownTopicException, it can explicitly issue a
createTopicRequest for auto topic creation. On the consumer side, it will
never issue createTopicRequest. This works when auto topic creation is
enabled on the broker side. However, I am not sure how things will work
when auto topic creation is disabled on the broker side. In this case, we
want to have a way to manually create a topic, potentially through admin
commands. However, then we need a way to distinguish createTopicRequest
issued from the producer clients and the admin tools. May be we can add a
new field in createTopicRequest and set it differently in the producer
client and the admin client. However, I am not sure if that's the best
approach.

2. Yes, refactoring existing requests is a non-trivial amount of work. I
posted some comments in KAFKA-1927. We will probably have to fix KAFKA-1927
first, before adding the new logic in KAFKA-1694. Otherwise, the changes
will be too big.

102. About the AdminClient:
102.1. It's a bit weird that we return exception in the api. It seems that
we should either return error code or throw an exception when getting the
response state.
102.2. We probably shouldn't explicitly use the request object in the api.
Not every request evolution requires an api change.

Thanks,

Jun


On Fri, Mar 13, 2015 at 4:08 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Jun,
>
> Thanks for you comments. Answers inline:
>
> 100. There are a few fields such as ReplicaAssignment,
> > ReassignPartitionRequest,
> > and PartitionsSerialized that are represented as a string, but contain
> > composite structures in json. Could we flatten them out directly in the
> > protocol definition as arrays/records?
>
>
> Yes, now with Admin Client this looks a bit weird. My initial motivation
> was:
> ReassignPartitionCommand accepts input in json, we want to remain tools'
> interfaces unchanged, where possible.
> If we port it to deserialized format, in CLI (/tools project) we will have
> to add some
> json library since /tools is written in java and we'll need to deserialize
> json file
> provided by a user. Can we quickly agree on what this library should be
> (Jackson, GSON, whatever)?
>
> 101. Does TopicMetadataRequest v1 still trigger auto topic creation? This
> > will be a bit weird now that we have a separate topic creation api. Have
> > you thought about how the new createTopicRequest and TopicMetadataRequest
> > v1 will be used in the producer/consumer client, in addition to admin
> > tools? For example, ideally, we don't want TopicMetadataRequest from the
> > consumer to trigger auto topic creation.
>
>
> I agree, this strange logic should be fixed. I'm not confident in this
> Kafka part so
> correct me if I'm wrong, but it doesn't look like a hard thing to do, I
> think we can
> leverage AdminClient for that in Producer and unconditionally remove topic
> creation from the TopicMetadataRequest_V1.
>
> 2. I think Jay meant getting rid of scala classes
> > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We did
> that
> > as a stop-gap thing when adding the new requests for the consumers.
> > However, the long term plan is to get rid of all those and just reuse the
> > java request/response in the client. Since this KIP proposes to add a
> > significant number of new requests, perhaps we should bite the bullet to
> > clean up the existing scala requests first before adding new ones?
> >
>
> Yes, looks like I misunderstood the point of ...RequestAndHeader. Okay, I
> will
> rework that. The only thing is that I don't see any example how it was done
> for at
> least one existing protocol message. Thus, as I understand, I have to think
> how we
> are going to do it.
> Re porting all existing RQ/RP in this patch. Sounds reasonable, but if it's
> an *obligatory*
> requirement to have Admin KIP done, I'm afraid this can be a serious
> blocker for us.
> There are 13 protocol messages and all that would require not only unit
> tests but quite
> intensive manual testing, no? I'm afraid I'm not the right guy to cover
> pretty much all
> Kafka core internals :). Let me know your thoughts on this item. Btw there
> is a ticket to
> follow-up this issue (https://issues.apache.org/jira/browse/KAFKA-2006).
>
> Thanks,
> Andrii Biletskyi
>
>
> On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <ju...@confluent.io> wrote:
>
> > Andrii,
> >
> >
> > A few more comments.
> >
> > 100. There are a few fields such as ReplicaAssignment,
> > ReassignPartitionRequest,
> > and PartitionsSerialized that are represented as a string, but contain
> > composite structures in json. Could we flatten them out directly in the
> > protocol definition as arrays/records?
> >
> > 101. Does TopicMetadataRequest v1 still trigger auto topic creation? This
> > will be a bit weird now that we have a separate topic creation api. Have
> > you thought about how the new createTopicRequest and TopicMetadataRequest
> > v1 will be used in the producer/consumer client, in addition to admin
> > tools? For example, ideally, we don't want TopicMetadataRequest from the
> > consumer to trigger auto topic creation.
> >
> > 2. I think Jay meant getting rid of scala classes
> > like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We did
> that
> > as a stop-gap thing when adding the new requests for the consumers.
> > However, the long term plan is to get rid of all those and just reuse the
> > java request/response in the client. Since this KIP proposes to add a
> > significant number of new requests, perhaps we should bite the bullet to
> > clean up the existing scala requests first before adding new ones?
> >
> > Thanks,
> >
> > Jun
> >
> >
> >
> > On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Hi,
> > >
> > > As said above - I list again all comments from this thread so we
> > > can see what's left and finalize all pending issues.
> > >
> > > Comments from Jay:
> > > 1. This is much needed functionality, but there are a lot of the so
> let's
> > > really think these protocols through. We really want to end up with a
> set
> > > of well thought-out, orthoganol apis. For this reason I think it is
> > really
> > > important to think through the end state even if that includes APIs we
> > > won't implement in the first phase.
> > >
> > > A: Definitely behind this. Would appreciate if there are concrete
> > comments
> > > how this can be improved.
> > >
> > > 2. Let's please please please wait until we have switched the server
> over
> > > to the new java protocol definitions. If we add upteen more ad hoc
> scala
> > > objects that is just generating more work for the conversion we know we
> > > have to do.
> > >
> > > A: Fixed in the latest patch - removed scala protocol classes.
> > >
> > > 3. This proposal introduces a new type of optional parameter. This is
> > > inconsistent with everything else in the protocol where we use -1 or
> some
> > > other marker value. You could argue either way but let's stick with
> that
> > > for consistency. For clients that implemented the protocol in a better
> > way
> > > than our scala code these basic primitives are hard to change.
> > >
> > > A: Fixed in the latest patch - removed MaybeOf type and changed
> protocol
> > > accordingly.
> > >
> > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest which
> > has
> > > brokers, topics, and partitions. I think we should rename that request
> > > ClusterMetadataRequest (or just MetadataRequest) and include the id of
> > the
> > > controller. Or are there other things we could add here?
> > >
> > > A: I agree. Updated the KIP. Let's extends TopicMetadata to version 2
> and
> > > include controller.
> > >
> > > 5. We have a tendency to try to make a lot of requests that can only go
> > to
> > > particular nodes. This adds a lot of burden for client implementations
> > (it
> > > sounds easy but each discovery can fail in many parts so it ends up
> > being a
> > > full state machine to do right). I think we should consider making
> admin
> > > commands and ideally as many of the other apis as possible available on
> > all
> > > brokers and just redirect to the controller on the broker side. Perhaps
> > > there would be a general way to encapsulate this re-routing behavior.
> > >
> > > A: It's a very interesting idea, but seems there are some concerns
> about
> > > this
> > > feature (like performance considerations, how this will complicate
> server
> > > etc).
> > > I believe this shouldn't be a blocker. If this feature is implemented
> at
> > > some
> > > point it won't affect Admin changes - at least no changes to public API
> > > will be required.
> > >
> > > 6. We should probably normalize the key value pairs used for configs
> > rather
> > > than embedding a new formatting. So two strings rather than one with an
> > > internal equals sign.
> > >
> > > A: Fixed in the latest patch - normalized configs and changed protocol
> > > accordingly.
> > >
> > > 7. Is the postcondition of these APIs that the command has begun or
> that
> > > the command has been completed? It is a lot more usable if the command
> > has
> > > been completed so you know that if you create a topic and then publish
> to
> > > it you won't get an exception about there being no such topic.
> > >
> > > A: For long running requests (like reassign partitions) - the post
> > > condition is
> > > command has begun - so we don't block the client. In case of your
> > example -
> > > topic commands, this will be refactored and topic commands will be
> > executed
> > > immediately, since the Controller will serve Admin requests
> > > (follow-up ticket KAFKA-1777).
> > >
> > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > metadata
> > > request. Is there a reason to give back topics marked for deletion? I
> > feel
> > > like if we just make the post-condition of the delete command be that
> the
> > > topic is deleted that will get rid of the need for this right? And it
> > will
> > > be much more intuitive.
> > >
> > > A: Fixed in the latest patch - removed topics marked for deletion in
> > > ListTopicsRequest.
> > >
> > > 9. Should we consider batching these requests? We have generally tried
> to
> > > allow multiple operations to be batched. My suspicion is that without
> > this
> > > we will get a lot of code that does something like
> > >    for(topic: adminClient.listTopics())
> > >       adminClient.describeTopic(topic)
> > > this code will work great when you test on 5 topics but not do as well
> if
> > > you have 50k.
> > >
> > > A: Updated the KIP - please check "Topic Admin Schema" section.
> > >
> > > 10. I think we should also discuss how we want to expose a programmatic
> > JVM
> > > client api for these operations. Currently people rely on AdminUtils
> > which
> > > is totally sketchy. I think we probably need another client under
> > clients/
> > > that exposes administrative functionality. We will need this just to
> > > properly test the new apis, I suspect. We should figure out that API.
> > >
> > > A: Updated the KIP - please check "Admin Client" section with an
> initial
> > > API proposal.
> > >
> > > 11. The other information that would be really useful to get would be
> > > information about partitions--how much data is in the partition, what
> are
> > > the segment offsets, what is the log-end offset (i.e. last offset),
> what
> > is
> > > the compaction point, etc. I think that done right this would be the
> > > successor to the very awkward OffsetRequest we have today.
> > >
> > > A: I removed ConsumerGroupOffsetsRequest in the latest patch. I believe
> > > this should
> > > be resolved in a separate KIP / jira ticket.
> > >
> > > 12. Generally we can do good error handling without needing custom
> > > server-side
> > > messages. I.e. generally the client has the context to know that if it
> > got
> > > an error that the topic doesn't exist to say "Topic X doesn't exist"
> > rather
> > > than "error code 14" (or whatever). Maybe there are specific cases
> where
> > > this is hard? If we want to add server-side error messages we really do
> > > need to do this in a consistent way across the protocol.
> > >
> > > A: Updated the KIP - please check "Protocol Errors" section. I added
> the
> > > comprehensive, fine-grained list of error codes.
> > >
> > > Comments from Guozhang:
> > > 13. Describe topic request: it would be great to go beyond just
> batching
> > on
> > > topic name regex for this request. For example, a very common use case
> of
> > > the topic command is to list all topics whose config A's value is B.
> With
> > > topic name regex then we have to first retrieve __all__ topics's
> > > description info and then filter at the client end, which will be a
> huge
> > > burden on ZK.
> > > AND
> > > 14. Config K-Vs in create topic: this is related to the previous point;
> > > maybe we can add another metadata K-V or just a metadata string along
> > side
> > > with config K-V in create topic like we did for offset commit request.
> > This
> > > field can be quite useful in storing information like "owner" of the
> > topic
> > > who issue the create command, etc, which is quite important for a
> > > multi-tenant setting. Then in the describe topic request we can also
> > batch
> > > on regex of the metadata field.
> > >
> > > A: As discussed it is very interesting but can be implemented later
> after
> > > we have some basic functionality there.
> > >
> > > 15. Today all the admin operations are async in the sense that command
> > will
> > > return once it is written in ZK, and that is why we need extra
> > verification
> > > like testUtil.waitForTopicCreated() / verify partition reassignment
> > > request, etc. With admin requests we could add a flag to enable /
> disable
> > > synchronous requests; when it is turned on, the response will not
> return
> > > until the request has been completed. And for async requests we can
> add a
> > > "token" field in the response, and then only need a general "admin
> > > verification request" with the given token to check if the async
> request
> > > has been completed.
> > >
> > > A: I see your point. My idea was to provide specific Verify...Request
> per
> > > each
> > > long running request, where needed. We can do it the way you suggest.
> The
> > > only
> > > concern is that introducing a token we again will make schema
> "dynamic".
> > We
> > > wanted
> > > to do similar thing introducing single AdminRequest for all topic
> > commands
> > > but rejected
> > > this idea because we wanted to have schema defined. So this is more a
> > > choice between:
> > > a) have fixed schema but introduce each time new Verify...Request for
> > > long-running requests
> > > b) use one request for verification but generalize it with token
> > > I'm fine with whatever decision community come to. Just let me know
> your
> > > thoughts.
> > >
> > > Comment from Gwen:
> > > 16. Specifically for ownership, I think the plan is to add ACL (it
> sounds
> > > like you are describing ACL) via an external system (Argus, Sentry).
> > > I remember KIP-11 described this, but I can't find the KIP any longer.
> > >
> > > A: Okay, no problem. Not sure though how we are going to handle it.
> Wait
> > > which KIP
> > > will be committed first and include changes to TopicMetadata from the
> > later
> > > one?
> > > Anyway, I added this note to "Open Questions" section so we don't miss
> > this
> > > piece.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Hi all,
> > > >
> > > > Today I uploaded the patch that covers some of the discussed and
> agreed
> > > > items:
> > > > - removed MaybeOf optional type
> > > > - switched to java protocol definitions
> > > > - simplified messages (normalized configs, removed topic marked for
> > > > deletion)
> > > >
> > > > I also updated the KIP-4 with respective changes and wrote down my
> > > > proposal for
> > > > pending items:
> > > > - Batch Admin Operations -> updated Wire Protocol schema proposal
> > > > - Remove ClusterMetadata -> changed to extend TopicMetadataRequest
> > > > - Admin Client -> updated my initial proposal to reflect batching
> > > > - Error codes -> proposed fine-grained error code instead of
> > > > AdminRequestFailed
> > > >
> > > > I will also send a separate email to cover all comments from this
> > thread.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > >
> > > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <gshapira@cloudera.com
> >
> > > > wrote:
> > > >
> > > >> Found KIP-11 (
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > > >> )
> > > >> It actually specifies changes to the Metadata protocol, so making
> sure
> > > >> both KIPs are consistent in this regard will be good.
> > > >>
> > > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <
> gshapira@cloudera.com
> > >
> > > >> wrote:
> > > >> > Specifically for ownership, I think the plan is to add ACL (it
> > sounds
> > > >> > like you are describing ACL) via an external system (Argus,
> Sentry).
> > > >> > I remember KIP-11 described this, but I can't find the KIP any
> > longer.
> > > >> >
> > > >> > Regardless, I think KIP-4 focuses on getting information that
> > already
> > > >> > exists from Kafka brokers, not on adding information that perhaps
> > > >> > should exist but doesn't yet?
> > > >> >
> > > >> > Gwen
> > > >> >
> > > >> >
> > > >> >
> > > >> >
> > > >> >
> > > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <
> wangguoz@gmail.com>
> > > >> wrote:
> > > >> >> Folks,
> > > >> >>
> > > >> >> Just want to elaborate a bit more on the create-topic metadata
> and
> > > >> batching
> > > >> >> describe-topic based on config / metadata in my previous email as
> > we
> > > >> work
> > > >> >> on KAFKA-1694. The main motivation is to have some sort of topic
> > > >> management
> > > >> >> mechanisms, which I think is quite important in a multi-tenant /
> > > cloud
> > > >> >> architecture: today anyone can create topics in a shared Kafka
> > > >> cluster, but
> > > >> >> there is no concept or "ownership" of topics that are created by
> > > >> different
> > > >> >> users. For example, at LinkedIn we basically distinguish topic
> > owners
> > > >> via
> > > >> >> some casual topic name prefix, which is a bit awkward and does
> not
> > > fly
> > > >> as
> > > >> >> we scale our customers. It would be great to use describe-topics
> > such
> > > >> as:
> > > >> >>
> > > >> >> Describe all topics that is created by me.
> > > >> >>
> > > >> >> Describe all topics whose retention time is overriden to X.
> > > >> >>
> > > >> >> Describe all topics whose writable group include user Y (this is
> > > >> related to
> > > >> >> authorization), etc..
> > > >> >>
> > > >> >> One possible way to achieve this is to add a metadata file in the
> > > >> >> create-topic request, whose value will also be written ZK as we
> > > create
> > > >> the
> > > >> >> topic; then describe-topics can choose to batch topics based on
> 1)
> > > name
> > > >> >> regex, 2) config K-V matching, 3) metadata regex, etc.
> > > >> >>
> > > >> >> Thoughts?
> > > >> >>
> > > >> >> Guozhang
> > > >> >>
> > > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <
> wangguoz@gmail.com>
> > > >> wrote:
> > > >> >>
> > > >> >>> Thanks for the updated wiki. A few comments below:
> > > >> >>>
> > > >> >>> 1. Error description in response: I think if some errorCode
> could
> > > >> indicate
> > > >> >>> several different error cases then we should really change it to
> > > >> multiple
> > > >> >>> codes. In general the errorCode itself would be precise and
> > > >> sufficient for
> > > >> >>> describing the server side errors.
> > > >> >>>
> > > >> >>> 2. Describe topic request: it would be great to go beyond just
> > > >> batching on
> > > >> >>> topic name regex for this request. For example, a very common
> use
> > > >> case of
> > > >> >>> the topic command is to list all topics whose config A's value
> is
> > B.
> > > >> With
> > > >> >>> topic name regex then we have to first retrieve __all__ topics's
> > > >> >>> description info and then filter at the client end, which will
> be
> > a
> > > >> huge
> > > >> >>> burden on ZK.
> > > >> >>>
> > > >> >>> 3. Config K-Vs in create topic: this is related to the previous
> > > point;
> > > >> >>> maybe we can add another metadata K-V or just a metadata string
> > > along
> > > >> side
> > > >> >>> with config K-V in create topic like we did for offset commit
> > > >> request. This
> > > >> >>> field can be quite useful in storing information like "owner" of
> > the
> > > >> topic
> > > >> >>> who issue the create command, etc, which is quite important for
> a
> > > >> >>> multi-tenant setting. Then in the describe topic request we can
> > also
> > > >> batch
> > > >> >>> on regex of the metadata field.
> > > >> >>>
> > > >> >>> 4. Today all the admin operations are async in the sense that
> > > command
> > > >> will
> > > >> >>> return once it is written in ZK, and that is why we need extra
> > > >> verification
> > > >> >>> like testUtil.waitForTopicCreated() / verify partition
> > reassignment
> > > >> >>> request, etc. With admin requests we could add a flag to enable
> /
> > > >> disable
> > > >> >>> synchronous requests; when it is turned on, the response will
> not
> > > >> return
> > > >> >>> until the request has been completed. And for async requests we
> > can
> > > >> add a
> > > >> >>> "token" field in the response, and then only need a general
> "admin
> > > >> >>> verification request" with the given token to check if the async
> > > >> request
> > > >> >>> has been completed.
> > > >> >>>
> > > >> >>> 5. +1 for extending Metadata request to include controller /
> > > >> coordinator
> > > >> >>> information, and then we can remove the ConsumerMetadata /
> > > >> ClusterMetadata
> > > >> >>> requests.
> > > >> >>>
> > > >> >>> Guozhang
> > > >> >>>
> > > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <
> jjkoshy.w@gmail.com>
> > > >> wrote:
> > > >> >>>
> > > >> >>>> Thanks for sending that out Joe - I don't think I will be able
> to
> > > >> make
> > > >> >>>> it today, so if notes can be sent out afterward that would be
> > > great.
> > > >> >>>>
> > > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> > > >> >>>> > Thanks for sending this out Joe. Looking forward to chatting
> > with
> > > >> >>>> everyone :)
> > > >> >>>> >
> > > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> > joe.stein@stealth.ly>
> > > >> wrote:
> > > >> >>>> > > Hey, I just sent out a google hangout invite to all pmc,
> > > >> committers
> > > >> >>>> and
> > > >> >>>> > > everyone I found working on a KIP. If I missed anyone in
> the
> > > >> invite
> > > >> >>>> please
> > > >> >>>> > > let me know and can update it, np.
> > > >> >>>> > >
> > > >> >>>> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe
> we
> > > can
> > > >> get
> > > >> >>>> INFRA
> > > >> >>>> > > help to make a google account so we can manage better?
> > > >> >>>> > >
> > > >> >>>> > > To discuss
> > > >> >>>> > >
> > > >> >>>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > >> >>>> > > in progress and related JIRA that are interdependent and
> > common
> > > >> work.
> > > >> >>>> > >
> > > >> >>>> > > ~ Joe Stein
> > > >> >>>> > >
> > > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
> > > jay.kreps@gmail.com>
> > > >> >>>> wrote:
> > > >> >>>> > >
> > > >> >>>> > >> Let's stay on Google hangouts that will also record and
> make
> > > the
> > > >> >>>> sessions
> > > >> >>>> > >> available on youtube.
> > > >> >>>> > >>
> > > >> >>>> > >> -Jay
> > > >> >>>> > >>
> > > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> > > >> >>>> jholoman@cloudera.com>
> > > >> >>>> > >> wrote:
> > > >> >>>> > >>
> > > >> >>>> > >> > Jay / Joe
> > > >> >>>> > >> >
> > > >> >>>> > >> > We're happy to send out a Webex for this purpose. We
> could
> > > >> record
> > > >> >>>> the
> > > >> >>>> > >> > sessions if there is interest and publish them out.
> > > >> >>>> > >> >
> > > >> >>>> > >> > Thanks
> > > >> >>>> > >> >
> > > >> >>>> > >> > Jeff
> > > >> >>>> > >> >
> > > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> > > >> jay.kreps@gmail.com>
> > > >> >>>> wrote:
> > > >> >>>> > >> >
> > > >> >>>> > >> > > Let's try to get the technical hang-ups sorted out,
> > > though.
> > > >> I
> > > >> >>>> really
> > > >> >>>> > >> > think
> > > >> >>>> > >> > > there is some benefit to live discussion vs writing. I
> > am
> > > >> >>>> hopeful that
> > > >> >>>> > >> if
> > > >> >>>> > >> > > we post instructions and give ourselves a few attempts
> > we
> > > >> can
> > > >> >>>> get it
> > > >> >>>> > >> > > working.
> > > >> >>>> > >> > >
> > > >> >>>> > >> > > Tuesday at that time would work for me...any
> objections?
> > > >> >>>> > >> > >
> > > >> >>>> > >> > > -Jay
> > > >> >>>> > >> > >
> > > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> > > >> joe.stein@stealth.ly
> > > >> >>>> >
> > > >> >>>> > >> wrote:
> > > >> >>>> > >> > >
> > > >> >>>> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm
> > ET
> > > /
> > > >> 10am
> > > >> >>>> PT
> > > >> >>>> > >> ????
> > > >> >>>> > >> > > >
> > > >> >>>> > >> > > > I don't mind google hangout but there is always some
> > > >> issue or
> > > >> >>>> > >> whatever
> > > >> >>>> > >> > so
> > > >> >>>> > >> > > > we know the apache irc channel works. We can start
> > there
> > > >> and
> > > >> >>>> see how
> > > >> >>>> > >> it
> > > >> >>>> > >> > > > goes? We can pull transcripts too and associate to
> > > >> tickets if
> > > >> >>>> need be
> > > >> >>>> > >> > > makes
> > > >> >>>> > >> > > > it helpful for things.
> > > >> >>>> > >> > > >
> > > >> >>>> > >> > > > ~ Joestein
> > > >> >>>> > >> > > >
> > > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> > > >> >>>> jay.kreps@gmail.com>
> > > >> >>>> > >> > wrote:
> > > >> >>>> > >> > > >
> > > >> >>>> > >> > > > > We'd talked about doing a Google Hangout to chat
> > about
> > > >> this.
> > > >> >>>> What
> > > >> >>>> > >> > about
> > > >> >>>> > >> > > > > generalizing that a little further...I actually
> > think
> > > it
> > > >> >>>> would be
> > > >> >>>> > >> > good
> > > >> >>>> > >> > > > for
> > > >> >>>> > >> > > > > everyone spending a reasonable chunk of their week
> > on
> > > >> Kafka
> > > >> >>>> stuff
> > > >> >>>> > >> to
> > > >> >>>> > >> > > > maybe
> > > >> >>>> > >> > > > > sync up once a week. I think we could use time to
> > talk
> > > >> >>>> through
> > > >> >>>> > >> design
> > > >> >>>> > >> > > > > stuff, make sure we are on top of code reviews,
> talk
> > > >> through
> > > >> >>>> any
> > > >> >>>> > >> > tricky
> > > >> >>>> > >> > > > > issues, etc.
> > > >> >>>> > >> > > > >
> > > >> >>>> > >> > > > > We can make it publicly available so that any one
> > can
> > > >> follow
> > > >> >>>> along
> > > >> >>>> > >> > who
> > > >> >>>> > >> > > > > likes.
> > > >> >>>> > >> > > > >
> > > >> >>>> > >> > > > > Any interest in doing this? If so I'll try to set
> it
> > > up
> > > >> >>>> starting
> > > >> >>>> > >> next
> > > >> >>>> > >> > > > week.
> > > >> >>>> > >> > > > >
> > > >> >>>> > >> > > > > -Jay
> > > >> >>>> > >> > > > >
> > > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi
> <
> > > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> >>>> > >> > > > >
> > > >> >>>> > >> > > > > > Hi all,
> > > >> >>>> > >> > > > > >
> > > >> >>>> > >> > > > > > I've updated KIP page, fixed / aligned document
> > > >> structure.
> > > >> >>>> Also I
> > > >> >>>> > >> > > added
> > > >> >>>> > >> > > > > > some
> > > >> >>>> > >> > > > > > very initial proposal for AdminClient so we have
> > > >> something
> > > >> >>>> to
> > > >> >>>> > >> start
> > > >> >>>> > >> > > > from
> > > >> >>>> > >> > > > > > while
> > > >> >>>> > >> > > > > > discussing the KIP.
> > > >> >>>> > >> > > > > >
> > > >> >>>> > >> > > > > >
> > > >> >>>> > >> > > > > >
> > > >> >>>> > >> > > > >
> > > >> >>>> > >> > > >
> > > >> >>>> > >> > >
> > > >> >>>> > >> >
> > > >> >>>> > >>
> > > >> >>>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > >> >>>> > >> > > > > >
> > > >> >>>> > >> > > > > > Thanks,
> > > >> >>>> > >> > > > > > Andrii Biletskyi
> > > >> >>>> > >> > > > > >
> > > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii
> Biletskyi
> > <
> > > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> >>>> > >> > > > > >
> > > >> >>>> > >> > > > > > > Jay,
> > > >> >>>> > >> > > > > > >
> > > >> >>>> > >> > > > > > > Re error messages: you are right, in most
> cases
> > > >> client
> > > >> >>>> will
> > > >> >>>> > >> have
> > > >> >>>> > >> > > > enough
> > > >> >>>> > >> > > > > > > context to show descriptive error message. My
> > > >> concern is
> > > >> >>>> that
> > > >> >>>> > >> we
> > > >> >>>> > >> > > will
> > > >> >>>> > >> > > > > > have
> > > >> >>>> > >> > > > > > > to
> > > >> >>>> > >> > > > > > > add lots of new error codes for each possible
> > > >> error. Of
> > > >> >>>> course,
> > > >> >>>> > >> > we
> > > >> >>>> > >> > > > > could
> > > >> >>>> > >> > > > > > > reuse
> > > >> >>>> > >> > > > > > > some of existing like
> > UknownTopicOrPartitionCode,
> > > >> but we
> > > >> >>>> will
> > > >> >>>> > >> > also
> > > >> >>>> > >> > > > need
> > > >> >>>> > >> > > > > > to
> > > >> >>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> > > >> >>>> TopicConfigInvalid (both
> > > >> >>>> > >> > for
> > > >> >>>> > >> > > > > topic
> > > >> >>>> > >> > > > > > > name and config, and probably user would like
> to
> > > >> know
> > > >> >>>> what
> > > >> >>>> > >> > exactly
> > > >> >>>> > >> > > > > > > is wrong in his config),
> > InvalidReplicaAssignment,
> > > >> >>>> > >> InternalError
> > > >> >>>> > >> > > > (e.g.
> > > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > > >> >>>> > >> > > > > > > And this is only for TopicCommand, we will
> also
> > > >> need to
> > > >> >>>> add
> > > >> >>>> > >> > similar
> > > >> >>>> > >> > > > > stuff
> > > >> >>>> > >> > > > > > > for
> > > >> >>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll
> > end
> > > >> up
> > > >> >>>> with a
> > > >> >>>> > >> > large
> > > >> >>>> > >> > > > list
> > > >> >>>> > >> > > > > > of
> > > >> >>>> > >> > > > > > > error codes, used only in Admin protocol.
> > > >> >>>> > >> > > > > > > Having said that, I agree my proposal is not
> > > >> consistent
> > > >> >>>> with
> > > >> >>>> > >> > other
> > > >> >>>> > >> > > > > cases.
> > > >> >>>> > >> > > > > > > Maybe we can find better solution or something
> > > >> >>>> in-between.
> > > >> >>>> > >> > > > > > >
> > > >> >>>> > >> > > > > > > Re Hangout chat: I think it is a great idea.
> > This
> > > >> way we
> > > >> >>>> can
> > > >> >>>> > >> move
> > > >> >>>> > >> > > on
> > > >> >>>> > >> > > > > > > faster.
> > > >> >>>> > >> > > > > > > Let's agree somehow on date/time so people can
> > > join.
> > > >> >>>> Will work
> > > >> >>>> > >> > for
> > > >> >>>> > >> > > me
> > > >> >>>> > >> > > > > > this
> > > >> >>>> > >> > > > > > > and
> > > >> >>>> > >> > > > > > > next week almost anytime if agreed in advance.
> > > >> >>>> > >> > > > > > >
> > > >> >>>> > >> > > > > > > Thanks,
> > > >> >>>> > >> > > > > > > Andrii
> > > >> >>>> > >> > > > > > >
> > > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> > > >> >>>> > >> jay.kreps@gmail.com>
> > > >> >>>> > >> > > > > wrote:
> > > >> >>>> > >> > > > > > >
> > > >> >>>> > >> > > > > > >> Hey Andrii,
> > > >> >>>> > >> > > > > > >>
> > > >> >>>> > >> > > > > > >> Generally we can do good error handling
> without
> > > >> needing
> > > >> >>>> custom
> > > >> >>>> > >> > > > > > server-side
> > > >> >>>> > >> > > > > > >> messages. I.e. generally the client has the
> > > >> context to
> > > >> >>>> know
> > > >> >>>> > >> that
> > > >> >>>> > >> > > if
> > > >> >>>> > >> > > > it
> > > >> >>>> > >> > > > > > got
> > > >> >>>> > >> > > > > > >> an error that the topic doesn't exist to say
> > > >> "Topic X
> > > >> >>>> doesn't
> > > >> >>>> > >> > > exist"
> > > >> >>>> > >> > > > > > >> rather
> > > >> >>>> > >> > > > > > >> than "error code 14" (or whatever). Maybe
> there
> > > are
> > > >> >>>> specific
> > > >> >>>> > >> > cases
> > > >> >>>> > >> > > > > where
> > > >> >>>> > >> > > > > > >> this is hard? If we want to add server-side
> > error
> > > >> >>>> messages we
> > > >> >>>> > >> > > really
> > > >> >>>> > >> > > > > do
> > > >> >>>> > >> > > > > > >> need to do this in a consistent way across
> the
> > > >> protocol.
> > > >> >>>> > >> > > > > > >>
> > > >> >>>> > >> > > > > > >> I still have a bunch of open questions here
> > from
> > > my
> > > >> >>>> previous
> > > >> >>>> > >> > > list. I
> > > >> >>>> > >> > > > > > will
> > > >> >>>> > >> > > > > > >> be out for the next few days for Strata
> though.
> > > >> Maybe
> > > >> >>>> we could
> > > >> >>>> > >> > do
> > > >> >>>> > >> > > a
> > > >> >>>> > >> > > > > > Google
> > > >> >>>> > >> > > > > > >> Hangout chat on any open issues some time
> > towards
> > > >> the
> > > >> >>>> end of
> > > >> >>>> > >> > next
> > > >> >>>> > >> > > > week
> > > >> >>>> > >> > > > > > for
> > > >> >>>> > >> > > > > > >> anyone interested in this ticket? I have a
> > > feeling
> > > >> that
> > > >> >>>> might
> > > >> >>>> > >> > > > progress
> > > >> >>>> > >> > > > > > >> things a little faster than email--I think we
> > > >> could talk
> > > >> >>>> > >> through
> > > >> >>>> > >> > > > those
> > > >> >>>> > >> > > > > > >> issues I brought up fairly quickly...
> > > >> >>>> > >> > > > > > >>
> > > >> >>>> > >> > > > > > >> -Jay
> > > >> >>>> > >> > > > > > >>
> > > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii
> > > Biletskyi <
> > > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > > >> >>>> > >> > > > > > >>
> > > >> >>>> > >> > > > > > >> > Hi all,
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >> > I'm trying to address some of the issues
> > which
> > > >> were
> > > >> >>>> > >> mentioned
> > > >> >>>> > >> > > > > earlier
> > > >> >>>> > >> > > > > > >> about
> > > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of those was about
> > > >> batching
> > > >> >>>> > >> > operations.
> > > >> >>>> > >> > > > What
> > > >> >>>> > >> > > > > > if
> > > >> >>>> > >> > > > > > >> we
> > > >> >>>> > >> > > > > > >> > follow TopicCommand approach and let people
> > > >> specify
> > > >> >>>> > >> topic-name
> > > >> >>>> > >> > > by
> > > >> >>>> > >> > > > > > >> regexp -
> > > >> >>>> > >> > > > > > >> > would that cover most of the use cases?
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >> > Secondly, is what information should we
> > > generally
> > > >> >>>> provide in
> > > >> >>>> > >> > > Admin
> > > >> >>>> > >> > > > > > >> > responses.
> > > >> >>>> > >> > > > > > >> > I realize that Admin commands don't imply
> > they
> > > >> will
> > > >> >>>> be used
> > > >> >>>> > >> > only
> > > >> >>>> > >> > > > in
> > > >> >>>> > >> > > > > > CLI
> > > >> >>>> > >> > > > > > >> > but,
> > > >> >>>> > >> > > > > > >> > it seems to me, CLI is a very important
> > client
> > > >> of this
> > > >> >>>> > >> > feature.
> > > >> >>>> > >> > > In
> > > >> >>>> > >> > > > > > this
> > > >> >>>> > >> > > > > > >> > case,
> > > >> >>>> > >> > > > > > >> > seems logical, we would like to provide
> users
> > > >> with
> > > >> >>>> rich
> > > >> >>>> > >> > > experience
> > > >> >>>> > >> > > > > in
> > > >> >>>> > >> > > > > > >> terms
> > > >> >>>> > >> > > > > > >> > of
> > > >> >>>> > >> > > > > > >> > getting results / errors of the executed
> > > >> commands.
> > > >> >>>> Usually
> > > >> >>>> > >> we
> > > >> >>>> > >> > > > supply
> > > >> >>>> > >> > > > > > >> with
> > > >> >>>> > >> > > > > > >> > responses only errorCode, which looks very
> > > >> limiting,
> > > >> >>>> in case
> > > >> >>>> > >> > of
> > > >> >>>> > >> > > > CLI
> > > >> >>>> > >> > > > > we
> > > >> >>>> > >> > > > > > >> may
> > > >> >>>> > >> > > > > > >> > want to print human readable error
> > description.
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >> > So, taking into account previous item about
> > > >> batching,
> > > >> >>>> what
> > > >> >>>> > >> do
> > > >> >>>> > >> > > you
> > > >> >>>> > >> > > > > > think
> > > >> >>>> > >> > > > > > >> > about
> > > >> >>>> > >> > > > > > >> > having smth like:
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >> > ('create' doesn't support regexp)
> > > >> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions
> > > >> Replicas
> > > >> >>>> > >> > > > > ReplicaAssignment
> > > >> >>>> > >> > > > > > >> > [Config]
> > > >> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode
> > > ErrorDescription
> > > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > > >> >>>> > >> > > > > > >> >   ErrorDescription => string (empty if
> > > >> successful)
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp
> > Partitions
> > > >> >>>> > >> > > ReplicaAssignment
> > > >> >>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> > > >> >>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
> > > >> >>>> ErrorDescription]
> > > >> >>>> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
> > > >> >>>> > >> > > > > > >> >   CommandErrorCode => int16
> > > >> >>>> > >> > > > > > >> >   CommandErrorDescription => string
> (nonempty
> > > in
> > > >> case
> > > >> >>>> of
> > > >> >>>> > >> fatal
> > > >> >>>> > >> > > > > error,
> > > >> >>>> > >> > > > > > >> e.g.
> > > >> >>>> > >> > > > > > >> > we couldn't get topics by regexp)
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> > > >> >>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> > > >> TopicDescription
> > > >> >>>> > >> ErrorCode
> > > >> >>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> > > >> >>>> CommandErrorDescription
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >> > Also, any thoughts about our discussion
> > > regarding
> > > >> >>>> re-routing
> > > >> >>>> > >> > > > > facility?
> > > >> >>>> > >> > > > > > >> In
> > > >> >>>> > >> > > > > > >> > my
> > > >> >>>> > >> > > > > > >> > understanding, it is like between
> augmenting
> > > >> >>>> > >> > > TopicMetadataRequest
> > > >> >>>> > >> > > > > > >> > (to include at least controllerId) and
> > > >> implementing
> > > >> >>>> new
> > > >> >>>> > >> > generic
> > > >> >>>> > >> > > > > > >> re-routing
> > > >> >>>> > >> > > > > > >> > facility so sending messages to controller
> > will
> > > >> be
> > > >> >>>> handled
> > > >> >>>> > >> by
> > > >> >>>> > >> > > it.
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >> > Thanks,
> > > >> >>>> > >> > > > > > >> > Andrii Biletskyi
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii
> > > >> Biletskyi <
> > > >> >>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >> > > @Guozhang:
> > > >> >>>> > >> > > > > > >> > > Thanks for your comments, I've answered
> > some
> > > of
> > > >> >>>> those. The
> > > >> >>>> > >> > > main
> > > >> >>>> > >> > > > > > thing
> > > >> >>>> > >> > > > > > >> is
> > > >> >>>> > >> > > > > > >> > > having merged request for
> > > >> >>>> create-alter-delete-describe - I
> > > >> >>>> > >> > > have
> > > >> >>>> > >> > > > > some
> > > >> >>>> > >> > > > > > >> > > concerns about this approach.
> > > >> >>>> > >> > > > > > >> > >
> > > >> >>>> > >> > > > > > >> > > @*Jay*:
> > > >> >>>> > >> > > > > > >> > > I see that introduced
> ClusterMetadaRequest
> > is
> > > >> also
> > > >> >>>> one of
> > > >> >>>> > >> > the
> > > >> >>>> > >> > > > > > >> concerns.
> > > >> >>>> > >> > > > > > >> > We
> > > >> >>>> > >> > > > > > >> > > can solve it if we implement re-routing
> > > >> facility.
> > > >> >>>> But I
> > > >> >>>> > >> > agree
> > > >> >>>> > >> > > > with
> > > >> >>>> > >> > > > > > >> > > Guozhang - it will make clients'
> internals
> > a
> > > >> little
> > > >> >>>> bit
> > > >> >>>> > >> > easier
> > > >> >>>> > >> > > > but
> > > >> >>>> > >> > > > > > >> this
> > > >> >>>> > >> > > > > > >> > > seems to be a complex logic to implement
> > and
> > > >> >>>> support then.
> > > >> >>>> > >> > > > > > Especially
> > > >> >>>> > >> > > > > > >> for
> > > >> >>>> > >> > > > > > >> > > Fetch and Produce (even if we add
> > re-routing
> > > >> later
> > > >> >>>> for
> > > >> >>>> > >> these
> > > >> >>>> > >> > > > > > >> requests).
> > > >> >>>> > >> > > > > > >> > > Also people will tend to avoid this
> > > re-routing
> > > >> >>>> facility
> > > >> >>>> > >> and
> > > >> >>>> > >> > > hold
> > > >> >>>> > >> > > > > > local
> > > >> >>>> > >> > > > > > >> > > cluster cache to ensure their
> high-priority
> > > >> requests
> > > >> >>>> > >> (which
> > > >> >>>> > >> > > some
> > > >> >>>> > >> > > > > of
> > > >> >>>> > >> > > > > > >> the
> > > >> >>>> > >> > > > > > >> > > admin requests are) not sent to some busy
> > > >> broker
> > > >> >>>> where
> > > >> >>>> > >> they
> > > >> >>>> > >> > > wait
> > > >> >>>> > >> > > > > to
> > > >> >>>> > >> > > > > > be
> > > >> >>>> > >> > > > > > >> > > routed to the correct one.
> > > >> >>>> > >> > > > > > >> > > As pointed out by Jun here (
> > > >> >>>> > >> > > > > > >> > >
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >>
> > > >> >>>> > >> > > > > >
> > > >> >>>> > >> > > > >
> > > >> >>>> > >> > > >
> > > >> >>>> > >> > >
> > > >> >>>> > >> >
> > > >> >>>> > >>
> > > >> >>>>
> > > >>
> > >
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > >> >>>> > >> > > > > > >> > )
> > > >> >>>> > >> > > > > > >> > > to solve the issue we might introduce a
> > > message
> > > >> >>>> type to
> > > >> >>>> > >> get
> > > >> >>>> > >> > > > > cluster
> > > >> >>>> > >> > > > > > >> > state.
> > > >> >>>> > >> > > > > > >> > > But I agree we can just update
> > > >> >>>> TopicMetadataResponse to
> > > >> >>>> > >> > > include
> > > >> >>>> > >> > > > > > >> > > controllerId (and probably smth else).
> > > >> >>>> > >> > > > > > >> > > What are you thougths?
> > > >> >>>> > >> > > > > > >> > >
> > > >> >>>> > >> > > > > > >> > > Thanks,
> > > >> >>>> > >> > > > > > >> > > Andrii
> > > >> >>>> > >> > > > > > >> > >
> > > >> >>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang
> > > Wang
> > > >> <
> > > >> >>>> > >> > > > > wangguoz@gmail.com>
> > > >> >>>> > >> > > > > > >> > wrote:
> > > >> >>>> > >> > > > > > >> > >
> > > >> >>>> > >> > > > > > >> > >> I think for the topics commands we can
> > > >> actually
> > > >> >>>> merge
> > > >> >>>> > >> > > > > > >> > >> create/alter/delete/describe as one
> > request
> > > >> type
> > > >> >>>> since
> > > >> >>>> > >> > their
> > > >> >>>> > >> > > > > > formats
> > > >> >>>> > >> > > > > > >> are
> > > >> >>>> > >> > > > > > >> > >> very much similar, and keep list-topics
> > and
> > > >> others
> > > >> >>>> like
> > > >> >>>> > >> > > > > > >> > >> partition-reassignment /
> > > >> preferred-leader-election
> > > >> >>>> as
> > > >> >>>> > >> > > separate
> > > >> >>>> > >> > > > > > >> request
> > > >> >>>> > >> > > > > > >> > >> types, I also left some other comments
> on
> > > the
> > > >> RB (
> > > >> >>>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> > > >> >>>> > >> > > > > > >> > >>
> > > >> >>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay
> > Kreps <
> > > >> >>>> > >> > > > jay.kreps@gmail.com>
> > > >> >>>> > >> > > > > > >> wrote:
> > > >> >>>> > >> > > > > > >> > >>
> > > >> >>>> > >> > > > > > >> > >> > Yeah I totally agree that we don't
> want
> > to
> > > >> just
> > > >> >>>> have
> > > >> >>>> > >> one
> > > >> >>>> > >> > > "do
> > > >> >>>> > >> > > > > > admin
> > > >> >>>> > >> > > > > > >> > >> stuff"
> > > >> >>>> > >> > > > > > >> > >> > command that has the union of all
> > > >> parameters.
> > > >> >>>> > >> > > > > > >> > >> >
> > > >> >>>> > >> > > > > > >> > >> > What I am saying is that command line
> > > tools
> > > >> are
> > > >> >>>> one
> > > >> >>>> > >> > client
> > > >> >>>> > >> > > of
> > > >> >>>> > >> > > > > the
> > > >> >>>> > >> > > > > > >> > >> > administrative apis, but these will be
> > > used
> > > >> in a
> > > >> >>>> number
> > > >> >>>> > >> > of
> > > >> >>>> > >> > > > > > >> scenarios
> > > >> >>>> > >> > > > > > >> > so
> > > >> >>>> > >> > > > > > >> > >> > they should make logical sense even in
> > the
> > > >> >>>> absence of
> > > >> >>>> > >> the
> > > >> >>>> > >> > > > > command
> > > >> >>>> > >> > > > > > >> line
> > > >> >>>> > >> > > > > > >> > >> > tool. Hence comments like trying to
> > > clarify
> > > >> the
> > > >> >>>> > >> > > relationship
> > > >> >>>> > >> > > > > > >> between
> > > >> >>>> > >> > > > > > >> > >> > ClusterMetadata and
> > TopicMetadata...these
> > > >> kinds
> > > >> >>>> of
> > > >> >>>> > >> things
> > > >> >>>> > >> > > > > really
> > > >> >>>> > >> > > > > > >> need
> > > >> >>>> > >> > > > > > >> > >> to be
> > > >> >>>> > >> > > > > > >> > >> > thought through.
> > > >> >>>> > >> > > > > > >> > >> >
> > > >> >>>> > >> > > > > > >> > >> > Hope that makes sense.
> > > >> >>>> > >> > > > > > >> > >> >
> > > >> >>>> > >> > > > > > >> > >> > -Jay
> > > >> >>>> > >> > > > > > >> > >> >
> > > >> >>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM,
> Andrii
> > > >> >>>> Biletskyi <
> > > >> >>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > > >> >>>> > >> > > > > > >> > >> >
> > > >> >>>> > >> > > > > > >> > >> > > Jay,
> > > >> >>>> > >> > > > > > >> > >> > >
> > > >> >>>> > >> > > > > > >> > >> > > Thanks for answering. You understood
> > > >> >>>> correctly, most
> > > >> >>>> > >> of
> > > >> >>>> > >> > > my
> > > >> >>>> > >> > > > > > >> comments
> > > >> >>>> > >> > > > > > >> > >> were
> > > >> >>>> > >> > > > > > >> > >> > > related to your point 1) - about
> "well
> > > >> >>>> thought-out"
> > > >> >>>> > >> > apis.
> > > >> >>>> > >> > > > > Also,
> > > >> >>>> > >> > > > > > >> yes,
> > > >> >>>> > >> > > > > > >> > >> as I
> > > >> >>>> > >> > > > > > >> > >> > > understood we would like to
> introduce
> > a
> > > >> single
> > > >> >>>> > >> unified
> > > >> >>>> > >> > > CLI
> > > >> >>>> > >> > > > > tool
> > > >> >>>> > >> > > > > > >> with
> > > >> >>>> > >> > > > > > >> > >> > > centralized server-side request
> > handling
> > > >> for
> > > >> >>>> lots of
> > > >> >>>> > >> > > > existing
> > > >> >>>> > >> > > > > > >> ones
> > > >> >>>> > >> > > > > > >> > >> (incl.
> > > >> >>>> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> > > >> >>>> > >> ReassignPartitions,
> > > >> >>>> > >> > > smth
> > > >> >>>> > >> > > > > > else
> > > >> >>>> > >> > > > > > >> if
> > > >> >>>> > >> > > > > > >> > >> added
> > > >> >>>> > >> > > > > > >> > >> > > in future). In our previous
> > discussion (
> > > >> >>>> > >> > > > > > >> > >> > >
> > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694)
> > > >> >>>> > >> > people
> > > >> >>>> > >> > > > > said
> > > >> >>>> > >> > > > > > >> > they'd
> > > >> >>>> > >> > > > > > >> > >> > > rather
> > > >> >>>> > >> > > > > > >> > >> > > have a separate message for each
> > > command,
> > > >> so,
> > > >> >>>> yes,
> > > >> >>>> > >> this
> > > >> >>>> > >> > > > way I
> > > >> >>>> > >> > > > > > >> came
> > > >> >>>> > >> > > > > > >> > to
> > > >> >>>> > >> > > > > > >> > >> 1-1
> > > >> >>>> > >> > > > > > >> > >> > > mapping between commands in the tool
> > and
> > > >> >>>> protocol
> > > >> >>>> > >> > > > additions.
> > > >> >>>> > >> > > > > > But
> > > >> >>>> > >> > > > > > >> I
> > > >> >>>> > >> > > > > > >> > >> might
> > > >> >>>> > >> > > > > > >> > >> > be
> > > >> >>>> > >> > > > > > >> > >> > > wrong.
> > > >> >>>> > >> > > > > > >> > >> > > At the end I just try to start
> > > discussion
> > > >> how
> > > >> >>>> at
> > > >> >>>> > >> least
> > > >> >>>> > >> > > > > > generally
> > > >> >>>> > >> > > > > > >> > this
> > > >> >>>> > >> > > > > > >> > >> > > protocol should look like.
> > > >> >>>> > >> > > > > > >> > >> > >
> > > >> >>>> > >> > > > > > >> > >> > > Thanks,
> > > >> >>>> > >> > > > > > >> > >> > > Andrii
> > > >> >>>> > >> > > > > > >> > >> > >
> > > >> >>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM,
> Jay
> > > >> Kreps <
> > > >> >>>> > >> > > > > > jay.kreps@gmail.com
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >> > >> wrote:
> > > >> >>>> > >> > > > > > >> > >> > >
> > > >> >>>> > >> > > > > > >> > >> > > > Hey Andrii,
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> > >> > > > > > >> > >> > > > To answer your earlier question we
> > > just
> > > >> >>>> really
> > > >> >>>> > >> can't
> > > >> >>>> > >> > be
> > > >> >>>> > >> > > > > > adding
> > > >> >>>> > >> > > > > > >> any
> > > >> >>>> > >> > > > > > >> > >> more
> > > >> >>>> > >> > > > > > >> > >> > > > scala protocol objects. These
> things
> > > are
> > > >> >>>> super hard
> > > >> >>>> > >> > to
> > > >> >>>> > >> > > > > > maintain
> > > >> >>>> > >> > > > > > >> > >> because
> > > >> >>>> > >> > > > > > >> > >> > > > they hand code the byte parsing
> and
> > > >> don't
> > > >> >>>> have good
> > > >> >>>> > >> > > > > > versioning
> > > >> >>>> > >> > > > > > >> > >> support.
> > > >> >>>> > >> > > > > > >> > >> > > > Since we are already planning on
> > > >> converting
> > > >> >>>> we
> > > >> >>>> > >> > > definitely
> > > >> >>>> > >> > > > > > don't
> > > >> >>>> > >> > > > > > >> > >> want to
> > > >> >>>> > >> > > > > > >> > >> > > add
> > > >> >>>> > >> > > > > > >> > >> > > > a ton more of these--they are
> total
> > > tech
> > > >> >>>> debt.
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> > >> > > > > > >> > >> > > > What does it mean that the changes
> > are
> > > >> >>>> isolated
> > > >> >>>> > >> from
> > > >> >>>> > >> > > the
> > > >> >>>> > >> > > > > > >> current
> > > >> >>>> > >> > > > > > >> > >> code
> > > >> >>>> > >> > > > > > >> > >> > > base?
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> > >> > > > > > >> > >> > > > I actually didn't understand the
> > > >> remaining
> > > >> >>>> > >> comments,
> > > >> >>>> > >> > > > which
> > > >> >>>> > >> > > > > of
> > > >> >>>> > >> > > > > > >> the
> > > >> >>>> > >> > > > > > >> > >> > points
> > > >> >>>> > >> > > > > > >> > >> > > > are you responding to?
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> > >> > > > > > >> > >> > > > Maybe one sticking point here is
> > that
> > > it
> > > >> >>>> seems like
> > > >> >>>> > >> > you
> > > >> >>>> > >> > > > > want
> > > >> >>>> > >> > > > > > to
> > > >> >>>> > >> > > > > > >> > make
> > > >> >>>> > >> > > > > > >> > >> > some
> > > >> >>>> > >> > > > > > >> > >> > > > kind of tool, and you have made a
> > 1-1
> > > >> mapping
> > > >> >>>> > >> between
> > > >> >>>> > >> > > > > > commands
> > > >> >>>> > >> > > > > > >> you
> > > >> >>>> > >> > > > > > >> > >> > > imagine
> > > >> >>>> > >> > > > > > >> > >> > > > in the tool and protocol
> additions.
> > I
> > > >> want
> > > >> >>>> to make
> > > >> >>>> > >> > sure
> > > >> >>>> > >> > > > we
> > > >> >>>> > >> > > > > > >> don't
> > > >> >>>> > >> > > > > > >> > do
> > > >> >>>> > >> > > > > > >> > >> > that.
> > > >> >>>> > >> > > > > > >> > >> > > > The protocol needs to be really
> > really
> > > >> well
> > > >> >>>> thought
> > > >> >>>> > >> > out
> > > >> >>>> > >> > > > > > against
> > > >> >>>> > >> > > > > > >> > many
> > > >> >>>> > >> > > > > > >> > >> > use
> > > >> >>>> > >> > > > > > >> > >> > > > cases so it should make perfect
> > > logical
> > > >> >>>> sense in
> > > >> >>>> > >> the
> > > >> >>>> > >> > > > > absence
> > > >> >>>> > >> > > > > > of
> > > >> >>>> > >> > > > > > >> > >> knowing
> > > >> >>>> > >> > > > > > >> > >> > > the
> > > >> >>>> > >> > > > > > >> > >> > > > command line tool, right?
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> > >> > > > > > >> > >> > > > -Jay
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM,
> > > Andrii
> > > >> >>>> Biletskyi
> > > >> >>>> > >> <
> > > >> >>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly>
> wrote:
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > Hey Jay,
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > I would like to continue this
> > > >> discussion
> > > >> >>>> as it
> > > >> >>>> > >> seem
> > > >> >>>> > >> > > > there
> > > >> >>>> > >> > > > > > is
> > > >> >>>> > >> > > > > > >> no
> > > >> >>>> > >> > > > > > >> > >> > > progress
> > > >> >>>> > >> > > > > > >> > >> > > > > here.
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > First of all, could you please
> > > explain
> > > >> >>>> what did
> > > >> >>>> > >> you
> > > >> >>>> > >> > > > mean
> > > >> >>>> > >> > > > > in
> > > >> >>>> > >> > > > > > >> 2?
> > > >> >>>> > >> > > > > > >> > How
> > > >> >>>> > >> > > > > > >> > >> > > > exactly
> > > >> >>>> > >> > > > > > >> > >> > > > > are we going to migrate to the
> new
> > > >> java
> > > >> >>>> protocol
> > > >> >>>> > >> > > > > > definitions.
> > > >> >>>> > >> > > > > > >> > And
> > > >> >>>> > >> > > > > > >> > >> why
> > > >> >>>> > >> > > > > > >> > >> > > > it's
> > > >> >>>> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > I agree with you, this feature
> > > >> includes
> > > >> >>>> lots of
> > > >> >>>> > >> > > stuff,
> > > >> >>>> > >> > > > > but
> > > >> >>>> > >> > > > > > >> > >> thankfully
> > > >> >>>> > >> > > > > > >> > >> > > > > almost all changes are isolated
> > from
> > > >> the
> > > >> >>>> current
> > > >> >>>> > >> > code
> > > >> >>>> > >> > > > > base,
> > > >> >>>> > >> > > > > > >> > >> > > > > so the main thing, I think, we
> > need
> > > to
> > > >> >>>> agree is
> > > >> >>>> > >> > RQ/RP
> > > >> >>>> > >> > > > > > format.
> > > >> >>>> > >> > > > > > >> > >> > > > > So how can we start discussion
> > about
> > > >> the
> > > >> >>>> concrete
> > > >> >>>> > >> > > > > messages
> > > >> >>>> > >> > > > > > >> > format?
> > > >> >>>> > >> > > > > > >> > >> > > > > Can we take (
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> > >> > > > > > >> > >> > >
> > > >> >>>> > >> > > > > > >> > >> >
> > > >> >>>> > >> > > > > > >> > >>
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >>
> > > >> >>>> > >> > > > > >
> > > >> >>>> > >> > > > >
> > > >> >>>> > >> > > >
> > > >> >>>> > >> > >
> > > >> >>>> > >> >
> > > >> >>>> > >>
> > > >> >>>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > >> >>>> > >> > > > > > >> > >> > > > > as starting point?
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > We had some doubts earlier
> whether
> > > it
> > > >> worth
> > > >> >>>> > >> > > introducing
> > > >> >>>> > >> > > > > one
> > > >> >>>> > >> > > > > > >> > >> generic
> > > >> >>>> > >> > > > > > >> > >> > > Admin
> > > >> >>>> > >> > > > > > >> > >> > > > > Request for all commands (
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694
> > > >> >>>> > >> > > > > > >> > >> > > > > )
> > > >> >>>> > >> > > > > > >> > >> > > > > but then everybody agreed it
> would
> > > be
> > > >> >>>> better to
> > > >> >>>> > >> > have
> > > >> >>>> > >> > > > > > separate
> > > >> >>>> > >> > > > > > >> > >> message
> > > >> >>>> > >> > > > > > >> > >> > > for
> > > >> >>>> > >> > > > > > >> > >> > > > > each admin command. The Request
> > part
> > > >> is
> > > >> >>>> really
> > > >> >>>> > >> > > dictated
> > > >> >>>> > >> > > > > > from
> > > >> >>>> > >> > > > > > >> the
> > > >> >>>> > >> > > > > > >> > >> > > command
> > > >> >>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments
> > > itself,
> > > >> so
> > > >> >>>> the
> > > >> >>>> > >> > proposed
> > > >> >>>> > >> > > > > > version
> > > >> >>>> > >> > > > > > >> > >> should
> > > >> >>>> > >> > > > > > >> > >> > be
> > > >> >>>> > >> > > > > > >> > >> > > > > fine (let's put aside for now
> > > remarks
> > > >> about
> > > >> >>>> > >> > Optional
> > > >> >>>> > >> > > > > type,
> > > >> >>>> > >> > > > > > >> > >> batching,
> > > >> >>>> > >> > > > > > >> > >> > > > > configs normalization - I agree
> > with
> > > >> all of
> > > >> >>>> > >> them).
> > > >> >>>> > >> > > > > > >> > >> > > > > So the second part is Response.
> I
> > > see
> > > >> >>>> there are
> > > >> >>>> > >> two
> > > >> >>>> > >> > > > cases
> > > >> >>>> > >> > > > > > >> here.
> > > >> >>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> > > >> Create/Alter/... ;
> > > >> >>>> b)
> > > >> >>>> > >> "Get"
> > > >> >>>> > >> > > > > > requests -
> > > >> >>>> > >> > > > > > >> > >> > > > > List/Describe...
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > a) should only hold request
> result
> > > >> >>>> (regardless
> > > >> >>>> > >> what
> > > >> >>>> > >> > > we
> > > >> >>>> > >> > > > > > decide
> > > >> >>>> > >> > > > > > >> > >> about
> > > >> >>>> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
> > > >> execution).
> > > >> >>>> > >> > > > > > >> > >> > > > > Usually we provide error code in
> > > >> response
> > > >> >>>> but
> > > >> >>>> > >> since
> > > >> >>>> > >> > > we
> > > >> >>>> > >> > > > > will
> > > >> >>>> > >> > > > > > >> use
> > > >> >>>> > >> > > > > > >> > >> this
> > > >> >>>> > >> > > > > > >> > >> > in
> > > >> >>>> > >> > > > > > >> > >> > > > > interactive shell we need some
> > human
> > > >> >>>> readable
> > > >> >>>> > >> error
> > > >> >>>> > >> > > > > > >> description
> > > >> >>>> > >> > > > > > >> > -
> > > >> >>>> > >> > > > > > >> > >> so
> > > >> >>>> > >> > > > > > >> > >> > I
> > > >> >>>> > >> > > > > > >> > >> > > > > added errorDesription field
> where
> > > you
> > > >> can
> > > >> >>>> at
> > > >> >>>> > >> least
> > > >> >>>> > >> > > > leave
> > > >> >>>> > >> > > > > > >> > >> > > > > exception.getMessage.
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > b) in addition to previous item
> > > >> message
> > > >> >>>> should
> > > >> >>>> > >> hold
> > > >> >>>> > >> > > > > command
> > > >> >>>> > >> > > > > > >> > >> specific
> > > >> >>>> > >> > > > > > >> > >> > > > > response data. We can discuss in
> > > >> detail
> > > >> >>>> each of
> > > >> >>>> > >> > them
> > > >> >>>> > >> > > > but
> > > >> >>>> > >> > > > > > >> let's
> > > >> >>>> > >> > > > > > >> > for
> > > >> >>>> > >> > > > > > >> > >> > now
> > > >> >>>> > >> > > > > > >> > >> > > > > agree about the overall pattern.
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > Thanks,
> > > >> >>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM,
> > Jay
> > > >> Kreps
> > > >> >>>> <
> > > >> >>>> > >> > > > > > >> jay.kreps@gmail.com
> > > >> >>>> > >> > > > > > >> > >
> > > >> >>>> > >> > > > > > >> > >> > > wrote:
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > Hey Joe,
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > This is great. A few comments
> on
> > > >> KIP-4
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > 1. This is much needed
> > > >> functionality,
> > > >> >>>> but there
> > > >> >>>> > >> > > are a
> > > >> >>>> > >> > > > > lot
> > > >> >>>> > >> > > > > > >> of
> > > >> >>>> > >> > > > > > >> > >> the so
> > > >> >>>> > >> > > > > > >> > >> > > > let's
> > > >> >>>> > >> > > > > > >> > >> > > > > > really think these protocols
> > > >> through. We
> > > >> >>>> really
> > > >> >>>> > >> > > want
> > > >> >>>> > >> > > > to
> > > >> >>>> > >> > > > > > >> end up
> > > >> >>>> > >> > > > > > >> > >> > with a
> > > >> >>>> > >> > > > > > >> > >> > > > set
> > > >> >>>> > >> > > > > > >> > >> > > > > > of well thought-out,
> orthoganol
> > > >> apis.
> > > >> >>>> For this
> > > >> >>>> > >> > > > reason I
> > > >> >>>> > >> > > > > > >> think
> > > >> >>>> > >> > > > > > >> > >> it is
> > > >> >>>> > >> > > > > > >> > >> > > > > really
> > > >> >>>> > >> > > > > > >> > >> > > > > > important to think through the
> > end
> > > >> state
> > > >> >>>> even
> > > >> >>>> > >> if
> > > >> >>>> > >> > > that
> > > >> >>>> > >> > > > > > >> includes
> > > >> >>>> > >> > > > > > >> > >> APIs
> > > >> >>>> > >> > > > > > >> > >> > > we
> > > >> >>>> > >> > > > > > >> > >> > > > > > won't implement in the first
> > > phase.
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > 2. Let's please please please
> > wait
> > > >> until
> > > >> >>>> we
> > > >> >>>> > >> have
> > > >> >>>> > >> > > > > switched
> > > >> >>>> > >> > > > > > >> the
> > > >> >>>> > >> > > > > > >> > >> > server
> > > >> >>>> > >> > > > > > >> > >> > > > over
> > > >> >>>> > >> > > > > > >> > >> > > > > > to the new java protocol
> > > >> definitions. If
> > > >> >>>> we add
> > > >> >>>> > >> > > > upteen
> > > >> >>>> > >> > > > > > >> more ad
> > > >> >>>> > >> > > > > > >> > >> hoc
> > > >> >>>> > >> > > > > > >> > >> > > > scala
> > > >> >>>> > >> > > > > > >> > >> > > > > > objects that is just
> generating
> > > more
> > > >> >>>> work for
> > > >> >>>> > >> the
> > > >> >>>> > >> > > > > > >> conversion
> > > >> >>>> > >> > > > > > >> > we
> > > >> >>>> > >> > > > > > >> > >> > know
> > > >> >>>> > >> > > > > > >> > >> > > we
> > > >> >>>> > >> > > > > > >> > >> > > > > > have to do.
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a
> > new
> > > >> type of
> > > >> >>>> > >> > optional
> > > >> >>>> > >> > > > > > >> parameter.
> > > >> >>>> > >> > > > > > >> > >> This
> > > >> >>>> > >> > > > > > >> > >> > is
> > > >> >>>> > >> > > > > > >> > >> > > > > > inconsistent with everything
> > else
> > > >> in the
> > > >> >>>> > >> protocol
> > > >> >>>> > >> > > > where
> > > >> >>>> > >> > > > > > we
> > > >> >>>> > >> > > > > > >> use
> > > >> >>>> > >> > > > > > >> > >> -1
> > > >> >>>> > >> > > > > > >> > >> > or
> > > >> >>>> > >> > > > > > >> > >> > > > some
> > > >> >>>> > >> > > > > > >> > >> > > > > > other marker value. You could
> > > argue
> > > >> >>>> either way
> > > >> >>>> > >> > but
> > > >> >>>> > >> > > > > let's
> > > >> >>>> > >> > > > > > >> stick
> > > >> >>>> > >> > > > > > >> > >> with
> > > >> >>>> > >> > > > > > >> > >> > > > that
> > > >> >>>> > >> > > > > > >> > >> > > > > > for consistency. For clients
> > that
> > > >> >>>> implemented
> > > >> >>>> > >> the
> > > >> >>>> > >> > > > > > protocol
> > > >> >>>> > >> > > > > > >> in
> > > >> >>>> > >> > > > > > >> > a
> > > >> >>>> > >> > > > > > >> > >> > > better
> > > >> >>>> > >> > > > > > >> > >> > > > > way
> > > >> >>>> > >> > > > > > >> > >> > > > > > than our scala code these
> basic
> > > >> >>>> primitives are
> > > >> >>>> > >> > hard
> > > >> >>>> > >> > > > to
> > > >> >>>> > >> > > > > > >> change.
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems
> > to
> > > >> >>>> duplicate
> > > >> >>>> > >> > > > > > >> > TopicMetadataRequest
> > > >> >>>> > >> > > > > > >> > >> > > which
> > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > >> >>>> > >> > > > > > >> > >> > > > > > brokers, topics, and
> > partitions. I
> > > >> think
> > > >> >>>> we
> > > >> >>>> > >> > should
> > > >> >>>> > >> > > > > rename
> > > >> >>>> > >> > > > > > >> that
> > > >> >>>> > >> > > > > > >> > >> > > request
> > > >> >>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or
> just
> > > >> >>>> > >> MetadataRequest)
> > > >> >>>> > >> > > and
> > > >> >>>> > >> > > > > > >> include
> > > >> >>>> > >> > > > > > >> > >> the id
> > > >> >>>> > >> > > > > > >> > >> > > of
> > > >> >>>> > >> > > > > > >> > >> > > > > the
> > > >> >>>> > >> > > > > > >> > >> > > > > > controller. Or are there other
> > > >> things we
> > > >> >>>> could
> > > >> >>>> > >> > add
> > > >> >>>> > >> > > > > here?
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try
> to
> > > >> make a
> > > >> >>>> lot of
> > > >> >>>> > >> > > > requests
> > > >> >>>> > >> > > > > > that
> > > >> >>>> > >> > > > > > >> > can
> > > >> >>>> > >> > > > > > >> > >> > only
> > > >> >>>> > >> > > > > > >> > >> > > go
> > > >> >>>> > >> > > > > > >> > >> > > > > to
> > > >> >>>> > >> > > > > > >> > >> > > > > > particular nodes. This adds a
> > lot
> > > of
> > > >> >>>> burden for
> > > >> >>>> > >> > > > client
> > > >> >>>> > >> > > > > > >> > >> > > implementations
> > > >> >>>> > >> > > > > > >> > >> > > > > (it
> > > >> >>>> > >> > > > > > >> > >> > > > > > sounds easy but each discovery
> > can
> > > >> fail
> > > >> >>>> in many
> > > >> >>>> > >> > > parts
> > > >> >>>> > >> > > > > so
> > > >> >>>> > >> > > > > > it
> > > >> >>>> > >> > > > > > >> > >> ends up
> > > >> >>>> > >> > > > > > >> > >> > > > > being a
> > > >> >>>> > >> > > > > > >> > >> > > > > > full state machine to do
> > right). I
> > > >> think
> > > >> >>>> we
> > > >> >>>> > >> > should
> > > >> >>>> > >> > > > > > consider
> > > >> >>>> > >> > > > > > >> > >> making
> > > >> >>>> > >> > > > > > >> > >> > > > admin
> > > >> >>>> > >> > > > > > >> > >> > > > > > commands and ideally as many
> of
> > > the
> > > >> >>>> other apis
> > > >> >>>> > >> as
> > > >> >>>> > >> > > > > > possible
> > > >> >>>> > >> > > > > > >> > >> > available
> > > >> >>>> > >> > > > > > >> > >> > > on
> > > >> >>>> > >> > > > > > >> > >> > > > > all
> > > >> >>>> > >> > > > > > >> > >> > > > > > brokers and just redirect to
> the
> > > >> >>>> controller on
> > > >> >>>> > >> > the
> > > >> >>>> > >> > > > > broker
> > > >> >>>> > >> > > > > > >> > side.
> > > >> >>>> > >> > > > > > >> > >> > > Perhaps
> > > >> >>>> > >> > > > > > >> > >> > > > > > there would be a general way
> to
> > > >> >>>> encapsulate
> > > >> >>>> > >> this
> > > >> >>>> > >> > > > > > re-routing
> > > >> >>>> > >> > > > > > >> > >> > behavior.
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > 6. We should probably
> normalize
> > > the
> > > >> key
> > > >> >>>> value
> > > >> >>>> > >> > pairs
> > > >> >>>> > >> > > > > used
> > > >> >>>> > >> > > > > > >> for
> > > >> >>>> > >> > > > > > >> > >> > configs
> > > >> >>>> > >> > > > > > >> > >> > > > > rather
> > > >> >>>> > >> > > > > > >> > >> > > > > > than embedding a new
> formatting.
> > > So
> > > >> two
> > > >> >>>> strings
> > > >> >>>> > >> > > > rather
> > > >> >>>> > >> > > > > > than
> > > >> >>>> > >> > > > > > >> > one
> > > >> >>>> > >> > > > > > >> > >> > with
> > > >> >>>> > >> > > > > > >> > >> > > an
> > > >> >>>> > >> > > > > > >> > >> > > > > > internal equals sign.
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of
> these
> > > >> APIs
> > > >> >>>> that the
> > > >> >>>> > >> > > > command
> > > >> >>>> > >> > > > > > has
> > > >> >>>> > >> > > > > > >> > >> begun or
> > > >> >>>> > >> > > > > > >> > >> > > > that
> > > >> >>>> > >> > > > > > >> > >> > > > > > the command has been
> completed?
> > It
> > > >> is a
> > > >> >>>> lot
> > > >> >>>> > >> more
> > > >> >>>> > >> > > > usable
> > > >> >>>> > >> > > > > > if
> > > >> >>>> > >> > > > > > >> the
> > > >> >>>> > >> > > > > > >> > >> > > command
> > > >> >>>> > >> > > > > > >> > >> > > > > has
> > > >> >>>> > >> > > > > > >> > >> > > > > > been completed so you know
> that
> > if
> > > >> you
> > > >> >>>> create a
> > > >> >>>> > >> > > topic
> > > >> >>>> > >> > > > > and
> > > >> >>>> > >> > > > > > >> then
> > > >> >>>> > >> > > > > > >> > >> > > publish
> > > >> >>>> > >> > > > > > >> > >> > > > to
> > > >> >>>> > >> > > > > > >> > >> > > > > > it you won't get an exception
> > > about
> > > >> >>>> there being
> > > >> >>>> > >> > no
> > > >> >>>> > >> > > > such
> > > >> >>>> > >> > > > > > >> topic.
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list
> > topics
> > > >> >>>> duplicate a
> > > >> >>>> > >> lot
> > > >> >>>> > >> > > of
> > > >> >>>> > >> > > > > > stuff
> > > >> >>>> > >> > > > > > >> in
> > > >> >>>> > >> > > > > > >> > >> the
> > > >> >>>> > >> > > > > > >> > >> > > > > metadata
> > > >> >>>> > >> > > > > > >> > >> > > > > > request. Is there a reason to
> > give
> > > >> back
> > > >> >>>> topics
> > > >> >>>> > >> > > marked
> > > >> >>>> > >> > > > > for
> > > >> >>>> > >> > > > > > >> > >> > deletion? I
> > > >> >>>> > >> > > > > > >> > >> > > > > feel
> > > >> >>>> > >> > > > > > >> > >> > > > > > like if we just make the
> > > >> post-condition
> > > >> >>>> of the
> > > >> >>>> > >> > > delete
> > > >> >>>> > >> > > > > > >> command
> > > >> >>>> > >> > > > > > >> > be
> > > >> >>>> > >> > > > > > >> > >> > that
> > > >> >>>> > >> > > > > > >> > >> > > > the
> > > >> >>>> > >> > > > > > >> > >> > > > > > topic is deleted that will get
> > rid
> > > >> of
> > > >> >>>> the need
> > > >> >>>> > >> > for
> > > >> >>>> > >> > > > this
> > > >> >>>> > >> > > > > > >> right?
> > > >> >>>> > >> > > > > > >> > >> And
> > > >> >>>> > >> > > > > > >> > >> > it
> > > >> >>>> > >> > > > > > >> > >> > > > > will
> > > >> >>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > 9. Should we consider batching
> > > these
> > > >> >>>> requests?
> > > >> >>>> > >> We
> > > >> >>>> > >> > > > have
> > > >> >>>> > >> > > > > > >> > generally
> > > >> >>>> > >> > > > > > >> > >> > > tried
> > > >> >>>> > >> > > > > > >> > >> > > > to
> > > >> >>>> > >> > > > > > >> > >> > > > > > allow multiple operations to
> be
> > > >> batched.
> > > >> >>>> My
> > > >> >>>> > >> > > suspicion
> > > >> >>>> > >> > > > > is
> > > >> >>>> > >> > > > > > >> that
> > > >> >>>> > >> > > > > > >> > >> > without
> > > >> >>>> > >> > > > > > >> > >> > > > > this
> > > >> >>>> > >> > > > > > >> > >> > > > > > we will get a lot of code that
> > > does
> > > >> >>>> something
> > > >> >>>> > >> > like
> > > >> >>>> > >> > > > > > >> > >> > > > > >    for(topic:
> > > >> adminClient.listTopics())
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >>  adminClient.describeTopic(topic)
> > > >> >>>> > >> > > > > > >> > >> > > > > > this code will work great when
> > you
> > > >> test
> > > >> >>>> on 5
> > > >> >>>> > >> > topics
> > > >> >>>> > >> > > > but
> > > >> >>>> > >> > > > > > >> not do
> > > >> >>>> > >> > > > > > >> > >> as
> > > >> >>>> > >> > > > > > >> > >> > > well
> > > >> >>>> > >> > > > > > >> > >> > > > if
> > > >> >>>> > >> > > > > > >> > >> > > > > > you have 50k.
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > 10. I think we should also
> > discuss
> > > >> how
> > > >> >>>> we want
> > > >> >>>> > >> to
> > > >> >>>> > >> > > > > expose
> > > >> >>>> > >> > > > > > a
> > > >> >>>> > >> > > > > > >> > >> > > programmatic
> > > >> >>>> > >> > > > > > >> > >> > > > > JVM
> > > >> >>>> > >> > > > > > >> > >> > > > > > client api for these
> operations.
> > > >> >>>> Currently
> > > >> >>>> > >> people
> > > >> >>>> > >> > > > rely
> > > >> >>>> > >> > > > > on
> > > >> >>>> > >> > > > > > >> > >> > AdminUtils
> > > >> >>>> > >> > > > > > >> > >> > > > > which
> > > >> >>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we
> > > >> probably
> > > >> >>>> need
> > > >> >>>> > >> > > another
> > > >> >>>> > >> > > > > > client
> > > >> >>>> > >> > > > > > >> > >> under
> > > >> >>>> > >> > > > > > >> > >> > > > > clients/
> > > >> >>>> > >> > > > > > >> > >> > > > > > that exposes administrative
> > > >> >>>> functionality. We
> > > >> >>>> > >> > will
> > > >> >>>> > >> > > > need
> > > >> >>>> > >> > > > > > >> this
> > > >> >>>> > >> > > > > > >> > >> just
> > > >> >>>> > >> > > > > > >> > >> > to
> > > >> >>>> > >> > > > > > >> > >> > > > > > properly test the new apis, I
> > > >> suspect. We
> > > >> >>>> > >> should
> > > >> >>>> > >> > > > figure
> > > >> >>>> > >> > > > > > out
> > > >> >>>> > >> > > > > > >> > that
> > > >> >>>> > >> > > > > > >> > >> > API.
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > 11. The other information that
> > > >> would be
> > > >> >>>> really
> > > >> >>>> > >> > > useful
> > > >> >>>> > >> > > > > to
> > > >> >>>> > >> > > > > > >> get
> > > >> >>>> > >> > > > > > >> > >> would
> > > >> >>>> > >> > > > > > >> > >> > be
> > > >> >>>> > >> > > > > > >> > >> > > > > > information about
> > partitions--how
> > > >> much
> > > >> >>>> data is
> > > >> >>>> > >> in
> > > >> >>>> > >> > > the
> > > >> >>>> > >> > > > > > >> > partition,
> > > >> >>>> > >> > > > > > >> > >> > what
> > > >> >>>> > >> > > > > > >> > >> > > > are
> > > >> >>>> > >> > > > > > >> > >> > > > > > the segment offsets, what is
> the
> > > >> log-end
> > > >> >>>> offset
> > > >> >>>> > >> > > (i.e.
> > > >> >>>> > >> > > > > > last
> > > >> >>>> > >> > > > > > >> > >> offset),
> > > >> >>>> > >> > > > > > >> > >> > > > what
> > > >> >>>> > >> > > > > > >> > >> > > > > is
> > > >> >>>> > >> > > > > > >> > >> > > > > > the compaction point, etc. I
> > think
> > > >> that
> > > >> >>>> done
> > > >> >>>> > >> > right
> > > >> >>>> > >> > > > this
> > > >> >>>> > >> > > > > > >> would
> > > >> >>>> > >> > > > > > >> > be
> > > >> >>>> > >> > > > > > >> > >> > the
> > > >> >>>> > >> > > > > > >> > >> > > > > > successor to the very awkward
> > > >> >>>> OffsetRequest we
> > > >> >>>> > >> > have
> > > >> >>>> > >> > > > > > today.
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > -Jay
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27
> > PM,
> > > >> Joe
> > > >> >>>> Stein <
> > > >> >>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
> > > >> >>>> > >> > > > > > >> > >> > > > > wrote:
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> > >> > > > > > >> > >> > >
> > > >> >>>> > >> > > > > > >> > >> >
> > > >> >>>> > >> > > > > > >> > >>
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >>
> > > >> >>>> > >> > > > > >
> > > >> >>>> > >> > > > >
> > > >> >>>> > >> > > >
> > > >> >>>> > >> > >
> > > >> >>>> > >> >
> > > >> >>>> > >>
> > > >> >>>>
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > > JIRA
> > > >> >>>> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > >> >>>> /*******************************************
> > > >> >>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
> > > >> >>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal
> Consultant
> > > >> >>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source
> Security
> > > LLC
> > > >> >>>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> > > >> >>>> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> > > >> >>>> > >> > > > > > >> > >> >
> http://www.twitter.com/allthingshadoop
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > >> >>>> ********************************************/
> > > >> >>>> > >> > > > > > >> > >> > > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > > >
> > > >> >>>> > >> > > > > > >> > >> > > > >
> > > >> >>>> > >> > > > > > >> > >> > > >
> > > >> >>>> > >> > > > > > >> > >> > >
> > > >> >>>> > >> > > > > > >> > >> >
> > > >> >>>> > >> > > > > > >> > >>
> > > >> >>>> > >> > > > > > >> > >>
> > > >> >>>> > >> > > > > > >> > >>
> > > >> >>>> > >> > > > > > >> > >> --
> > > >> >>>> > >> > > > > > >> > >> -- Guozhang
> > > >> >>>> > >> > > > > > >> > >>
> > > >> >>>> > >> > > > > > >> > >
> > > >> >>>> > >> > > > > > >> > >
> > > >> >>>> > >> > > > > > >> >
> > > >> >>>> > >> > > > > > >>
> > > >> >>>> > >> > > > > > >
> > > >> >>>> > >> > > > > > >
> > > >> >>>> > >> > > > > >
> > > >> >>>> > >> > > > >
> > > >> >>>> > >> > > >
> > > >> >>>> > >> > >
> > > >> >>>> > >> >
> > > >> >>>> > >> >
> > > >> >>>> > >> >
> > > >> >>>> > >> > --
> > > >> >>>> > >> > Jeff Holoman
> > > >> >>>> > >> > Systems Engineer
> > > >> >>>> > >> >
> > > >> >>>> > >>
> > > >> >>>>
> > > >> >>>>
> > > >> >>>
> > > >> >>>
> > > >> >>> --
> > > >> >>> -- Guozhang
> > > >> >>>
> > > >> >>
> > > >> >>
> > > >> >>
> > > >> >> --
> > > >> >> -- Guozhang
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Jun,

Thanks for you comments. Answers inline:

100. There are a few fields such as ReplicaAssignment,
> ReassignPartitionRequest,
> and PartitionsSerialized that are represented as a string, but contain
> composite structures in json. Could we flatten them out directly in the
> protocol definition as arrays/records?


Yes, now with Admin Client this looks a bit weird. My initial motivation
was:
ReassignPartitionCommand accepts input in json, we want to remain tools'
interfaces unchanged, where possible.
If we port it to deserialized format, in CLI (/tools project) we will have
to add some
json library since /tools is written in java and we'll need to deserialize
json file
provided by a user. Can we quickly agree on what this library should be
(Jackson, GSON, whatever)?

101. Does TopicMetadataRequest v1 still trigger auto topic creation? This
> will be a bit weird now that we have a separate topic creation api. Have
> you thought about how the new createTopicRequest and TopicMetadataRequest
> v1 will be used in the producer/consumer client, in addition to admin
> tools? For example, ideally, we don't want TopicMetadataRequest from the
> consumer to trigger auto topic creation.


I agree, this strange logic should be fixed. I'm not confident in this
Kafka part so
correct me if I'm wrong, but it doesn't look like a hard thing to do, I
think we can
leverage AdminClient for that in Producer and unconditionally remove topic
creation from the TopicMetadataRequest_V1.

2. I think Jay meant getting rid of scala classes
> like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We did that
> as a stop-gap thing when adding the new requests for the consumers.
> However, the long term plan is to get rid of all those and just reuse the
> java request/response in the client. Since this KIP proposes to add a
> significant number of new requests, perhaps we should bite the bullet to
> clean up the existing scala requests first before adding new ones?
>

Yes, looks like I misunderstood the point of ...RequestAndHeader. Okay, I
will
rework that. The only thing is that I don't see any example how it was done
for at
least one existing protocol message. Thus, as I understand, I have to think
how we
are going to do it.
Re porting all existing RQ/RP in this patch. Sounds reasonable, but if it's
an *obligatory*
requirement to have Admin KIP done, I'm afraid this can be a serious
blocker for us.
There are 13 protocol messages and all that would require not only unit
tests but quite
intensive manual testing, no? I'm afraid I'm not the right guy to cover
pretty much all
Kafka core internals :). Let me know your thoughts on this item. Btw there
is a ticket to
follow-up this issue (https://issues.apache.org/jira/browse/KAFKA-2006).

Thanks,
Andrii Biletskyi


On Fri, Mar 13, 2015 at 6:40 AM, Jun Rao <ju...@confluent.io> wrote:

> Andrii,
>
>
> A few more comments.
>
> 100. There are a few fields such as ReplicaAssignment,
> ReassignPartitionRequest,
> and PartitionsSerialized that are represented as a string, but contain
> composite structures in json. Could we flatten them out directly in the
> protocol definition as arrays/records?
>
> 101. Does TopicMetadataRequest v1 still trigger auto topic creation? This
> will be a bit weird now that we have a separate topic creation api. Have
> you thought about how the new createTopicRequest and TopicMetadataRequest
> v1 will be used in the producer/consumer client, in addition to admin
> tools? For example, ideally, we don't want TopicMetadataRequest from the
> consumer to trigger auto topic creation.
>
> 2. I think Jay meant getting rid of scala classes
> like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We did that
> as a stop-gap thing when adding the new requests for the consumers.
> However, the long term plan is to get rid of all those and just reuse the
> java request/response in the client. Since this KIP proposes to add a
> significant number of new requests, perhaps we should bite the bullet to
> clean up the existing scala requests first before adding new ones?
>
> Thanks,
>
> Jun
>
>
>
> On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Hi,
> >
> > As said above - I list again all comments from this thread so we
> > can see what's left and finalize all pending issues.
> >
> > Comments from Jay:
> > 1. This is much needed functionality, but there are a lot of the so let's
> > really think these protocols through. We really want to end up with a set
> > of well thought-out, orthoganol apis. For this reason I think it is
> really
> > important to think through the end state even if that includes APIs we
> > won't implement in the first phase.
> >
> > A: Definitely behind this. Would appreciate if there are concrete
> comments
> > how this can be improved.
> >
> > 2. Let's please please please wait until we have switched the server over
> > to the new java protocol definitions. If we add upteen more ad hoc scala
> > objects that is just generating more work for the conversion we know we
> > have to do.
> >
> > A: Fixed in the latest patch - removed scala protocol classes.
> >
> > 3. This proposal introduces a new type of optional parameter. This is
> > inconsistent with everything else in the protocol where we use -1 or some
> > other marker value. You could argue either way but let's stick with that
> > for consistency. For clients that implemented the protocol in a better
> way
> > than our scala code these basic primitives are hard to change.
> >
> > A: Fixed in the latest patch - removed MaybeOf type and changed protocol
> > accordingly.
> >
> > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest which
> has
> > brokers, topics, and partitions. I think we should rename that request
> > ClusterMetadataRequest (or just MetadataRequest) and include the id of
> the
> > controller. Or are there other things we could add here?
> >
> > A: I agree. Updated the KIP. Let's extends TopicMetadata to version 2 and
> > include controller.
> >
> > 5. We have a tendency to try to make a lot of requests that can only go
> to
> > particular nodes. This adds a lot of burden for client implementations
> (it
> > sounds easy but each discovery can fail in many parts so it ends up
> being a
> > full state machine to do right). I think we should consider making admin
> > commands and ideally as many of the other apis as possible available on
> all
> > brokers and just redirect to the controller on the broker side. Perhaps
> > there would be a general way to encapsulate this re-routing behavior.
> >
> > A: It's a very interesting idea, but seems there are some concerns about
> > this
> > feature (like performance considerations, how this will complicate server
> > etc).
> > I believe this shouldn't be a blocker. If this feature is implemented at
> > some
> > point it won't affect Admin changes - at least no changes to public API
> > will be required.
> >
> > 6. We should probably normalize the key value pairs used for configs
> rather
> > than embedding a new formatting. So two strings rather than one with an
> > internal equals sign.
> >
> > A: Fixed in the latest patch - normalized configs and changed protocol
> > accordingly.
> >
> > 7. Is the postcondition of these APIs that the command has begun or that
> > the command has been completed? It is a lot more usable if the command
> has
> > been completed so you know that if you create a topic and then publish to
> > it you won't get an exception about there being no such topic.
> >
> > A: For long running requests (like reassign partitions) - the post
> > condition is
> > command has begun - so we don't block the client. In case of your
> example -
> > topic commands, this will be refactored and topic commands will be
> executed
> > immediately, since the Controller will serve Admin requests
> > (follow-up ticket KAFKA-1777).
> >
> > 8. Describe topic and list topics duplicate a lot of stuff in the
> metadata
> > request. Is there a reason to give back topics marked for deletion? I
> feel
> > like if we just make the post-condition of the delete command be that the
> > topic is deleted that will get rid of the need for this right? And it
> will
> > be much more intuitive.
> >
> > A: Fixed in the latest patch - removed topics marked for deletion in
> > ListTopicsRequest.
> >
> > 9. Should we consider batching these requests? We have generally tried to
> > allow multiple operations to be batched. My suspicion is that without
> this
> > we will get a lot of code that does something like
> >    for(topic: adminClient.listTopics())
> >       adminClient.describeTopic(topic)
> > this code will work great when you test on 5 topics but not do as well if
> > you have 50k.
> >
> > A: Updated the KIP - please check "Topic Admin Schema" section.
> >
> > 10. I think we should also discuss how we want to expose a programmatic
> JVM
> > client api for these operations. Currently people rely on AdminUtils
> which
> > is totally sketchy. I think we probably need another client under
> clients/
> > that exposes administrative functionality. We will need this just to
> > properly test the new apis, I suspect. We should figure out that API.
> >
> > A: Updated the KIP - please check "Admin Client" section with an initial
> > API proposal.
> >
> > 11. The other information that would be really useful to get would be
> > information about partitions--how much data is in the partition, what are
> > the segment offsets, what is the log-end offset (i.e. last offset), what
> is
> > the compaction point, etc. I think that done right this would be the
> > successor to the very awkward OffsetRequest we have today.
> >
> > A: I removed ConsumerGroupOffsetsRequest in the latest patch. I believe
> > this should
> > be resolved in a separate KIP / jira ticket.
> >
> > 12. Generally we can do good error handling without needing custom
> > server-side
> > messages. I.e. generally the client has the context to know that if it
> got
> > an error that the topic doesn't exist to say "Topic X doesn't exist"
> rather
> > than "error code 14" (or whatever). Maybe there are specific cases where
> > this is hard? If we want to add server-side error messages we really do
> > need to do this in a consistent way across the protocol.
> >
> > A: Updated the KIP - please check "Protocol Errors" section. I added the
> > comprehensive, fine-grained list of error codes.
> >
> > Comments from Guozhang:
> > 13. Describe topic request: it would be great to go beyond just batching
> on
> > topic name regex for this request. For example, a very common use case of
> > the topic command is to list all topics whose config A's value is B. With
> > topic name regex then we have to first retrieve __all__ topics's
> > description info and then filter at the client end, which will be a huge
> > burden on ZK.
> > AND
> > 14. Config K-Vs in create topic: this is related to the previous point;
> > maybe we can add another metadata K-V or just a metadata string along
> side
> > with config K-V in create topic like we did for offset commit request.
> This
> > field can be quite useful in storing information like "owner" of the
> topic
> > who issue the create command, etc, which is quite important for a
> > multi-tenant setting. Then in the describe topic request we can also
> batch
> > on regex of the metadata field.
> >
> > A: As discussed it is very interesting but can be implemented later after
> > we have some basic functionality there.
> >
> > 15. Today all the admin operations are async in the sense that command
> will
> > return once it is written in ZK, and that is why we need extra
> verification
> > like testUtil.waitForTopicCreated() / verify partition reassignment
> > request, etc. With admin requests we could add a flag to enable / disable
> > synchronous requests; when it is turned on, the response will not return
> > until the request has been completed. And for async requests we can add a
> > "token" field in the response, and then only need a general "admin
> > verification request" with the given token to check if the async request
> > has been completed.
> >
> > A: I see your point. My idea was to provide specific Verify...Request per
> > each
> > long running request, where needed. We can do it the way you suggest. The
> > only
> > concern is that introducing a token we again will make schema "dynamic".
> We
> > wanted
> > to do similar thing introducing single AdminRequest for all topic
> commands
> > but rejected
> > this idea because we wanted to have schema defined. So this is more a
> > choice between:
> > a) have fixed schema but introduce each time new Verify...Request for
> > long-running requests
> > b) use one request for verification but generalize it with token
> > I'm fine with whatever decision community come to. Just let me know your
> > thoughts.
> >
> > Comment from Gwen:
> > 16. Specifically for ownership, I think the plan is to add ACL (it sounds
> > like you are describing ACL) via an external system (Argus, Sentry).
> > I remember KIP-11 described this, but I can't find the KIP any longer.
> >
> > A: Okay, no problem. Not sure though how we are going to handle it. Wait
> > which KIP
> > will be committed first and include changes to TopicMetadata from the
> later
> > one?
> > Anyway, I added this note to "Open Questions" section so we don't miss
> this
> > piece.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> > On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Hi all,
> > >
> > > Today I uploaded the patch that covers some of the discussed and agreed
> > > items:
> > > - removed MaybeOf optional type
> > > - switched to java protocol definitions
> > > - simplified messages (normalized configs, removed topic marked for
> > > deletion)
> > >
> > > I also updated the KIP-4 with respective changes and wrote down my
> > > proposal for
> > > pending items:
> > > - Batch Admin Operations -> updated Wire Protocol schema proposal
> > > - Remove ClusterMetadata -> changed to extend TopicMetadataRequest
> > > - Admin Client -> updated my initial proposal to reflect batching
> > > - Error codes -> proposed fine-grained error code instead of
> > > AdminRequestFailed
> > >
> > > I will also send a separate email to cover all comments from this
> thread.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > >
> > > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <gs...@cloudera.com>
> > > wrote:
> > >
> > >> Found KIP-11 (
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> > >> )
> > >> It actually specifies changes to the Metadata protocol, so making sure
> > >> both KIPs are consistent in this regard will be good.
> > >>
> > >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <gshapira@cloudera.com
> >
> > >> wrote:
> > >> > Specifically for ownership, I think the plan is to add ACL (it
> sounds
> > >> > like you are describing ACL) via an external system (Argus, Sentry).
> > >> > I remember KIP-11 described this, but I can't find the KIP any
> longer.
> > >> >
> > >> > Regardless, I think KIP-4 focuses on getting information that
> already
> > >> > exists from Kafka brokers, not on adding information that perhaps
> > >> > should exist but doesn't yet?
> > >> >
> > >> > Gwen
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <wa...@gmail.com>
> > >> wrote:
> > >> >> Folks,
> > >> >>
> > >> >> Just want to elaborate a bit more on the create-topic metadata and
> > >> batching
> > >> >> describe-topic based on config / metadata in my previous email as
> we
> > >> work
> > >> >> on KAFKA-1694. The main motivation is to have some sort of topic
> > >> management
> > >> >> mechanisms, which I think is quite important in a multi-tenant /
> > cloud
> > >> >> architecture: today anyone can create topics in a shared Kafka
> > >> cluster, but
> > >> >> there is no concept or "ownership" of topics that are created by
> > >> different
> > >> >> users. For example, at LinkedIn we basically distinguish topic
> owners
> > >> via
> > >> >> some casual topic name prefix, which is a bit awkward and does not
> > fly
> > >> as
> > >> >> we scale our customers. It would be great to use describe-topics
> such
> > >> as:
> > >> >>
> > >> >> Describe all topics that is created by me.
> > >> >>
> > >> >> Describe all topics whose retention time is overriden to X.
> > >> >>
> > >> >> Describe all topics whose writable group include user Y (this is
> > >> related to
> > >> >> authorization), etc..
> > >> >>
> > >> >> One possible way to achieve this is to add a metadata file in the
> > >> >> create-topic request, whose value will also be written ZK as we
> > create
> > >> the
> > >> >> topic; then describe-topics can choose to batch topics based on 1)
> > name
> > >> >> regex, 2) config K-V matching, 3) metadata regex, etc.
> > >> >>
> > >> >> Thoughts?
> > >> >>
> > >> >> Guozhang
> > >> >>
> > >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com>
> > >> wrote:
> > >> >>
> > >> >>> Thanks for the updated wiki. A few comments below:
> > >> >>>
> > >> >>> 1. Error description in response: I think if some errorCode could
> > >> indicate
> > >> >>> several different error cases then we should really change it to
> > >> multiple
> > >> >>> codes. In general the errorCode itself would be precise and
> > >> sufficient for
> > >> >>> describing the server side errors.
> > >> >>>
> > >> >>> 2. Describe topic request: it would be great to go beyond just
> > >> batching on
> > >> >>> topic name regex for this request. For example, a very common use
> > >> case of
> > >> >>> the topic command is to list all topics whose config A's value is
> B.
> > >> With
> > >> >>> topic name regex then we have to first retrieve __all__ topics's
> > >> >>> description info and then filter at the client end, which will be
> a
> > >> huge
> > >> >>> burden on ZK.
> > >> >>>
> > >> >>> 3. Config K-Vs in create topic: this is related to the previous
> > point;
> > >> >>> maybe we can add another metadata K-V or just a metadata string
> > along
> > >> side
> > >> >>> with config K-V in create topic like we did for offset commit
> > >> request. This
> > >> >>> field can be quite useful in storing information like "owner" of
> the
> > >> topic
> > >> >>> who issue the create command, etc, which is quite important for a
> > >> >>> multi-tenant setting. Then in the describe topic request we can
> also
> > >> batch
> > >> >>> on regex of the metadata field.
> > >> >>>
> > >> >>> 4. Today all the admin operations are async in the sense that
> > command
> > >> will
> > >> >>> return once it is written in ZK, and that is why we need extra
> > >> verification
> > >> >>> like testUtil.waitForTopicCreated() / verify partition
> reassignment
> > >> >>> request, etc. With admin requests we could add a flag to enable /
> > >> disable
> > >> >>> synchronous requests; when it is turned on, the response will not
> > >> return
> > >> >>> until the request has been completed. And for async requests we
> can
> > >> add a
> > >> >>> "token" field in the response, and then only need a general "admin
> > >> >>> verification request" with the given token to check if the async
> > >> request
> > >> >>> has been completed.
> > >> >>>
> > >> >>> 5. +1 for extending Metadata request to include controller /
> > >> coordinator
> > >> >>> information, and then we can remove the ConsumerMetadata /
> > >> ClusterMetadata
> > >> >>> requests.
> > >> >>>
> > >> >>> Guozhang
> > >> >>>
> > >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com>
> > >> wrote:
> > >> >>>
> > >> >>>> Thanks for sending that out Joe - I don't think I will be able to
> > >> make
> > >> >>>> it today, so if notes can be sent out afterward that would be
> > great.
> > >> >>>>
> > >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> > >> >>>> > Thanks for sending this out Joe. Looking forward to chatting
> with
> > >> >>>> everyone :)
> > >> >>>> >
> > >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <
> joe.stein@stealth.ly>
> > >> wrote:
> > >> >>>> > > Hey, I just sent out a google hangout invite to all pmc,
> > >> committers
> > >> >>>> and
> > >> >>>> > > everyone I found working on a KIP. If I missed anyone in the
> > >> invite
> > >> >>>> please
> > >> >>>> > > let me know and can update it, np.
> > >> >>>> > >
> > >> >>>> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we
> > can
> > >> get
> > >> >>>> INFRA
> > >> >>>> > > help to make a google account so we can manage better?
> > >> >>>> > >
> > >> >>>> > > To discuss
> > >> >>>> > >
> > >> >>>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > >> >>>> > > in progress and related JIRA that are interdependent and
> common
> > >> work.
> > >> >>>> > >
> > >> >>>> > > ~ Joe Stein
> > >> >>>> > >
> > >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
> > jay.kreps@gmail.com>
> > >> >>>> wrote:
> > >> >>>> > >
> > >> >>>> > >> Let's stay on Google hangouts that will also record and make
> > the
> > >> >>>> sessions
> > >> >>>> > >> available on youtube.
> > >> >>>> > >>
> > >> >>>> > >> -Jay
> > >> >>>> > >>
> > >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> > >> >>>> jholoman@cloudera.com>
> > >> >>>> > >> wrote:
> > >> >>>> > >>
> > >> >>>> > >> > Jay / Joe
> > >> >>>> > >> >
> > >> >>>> > >> > We're happy to send out a Webex for this purpose. We could
> > >> record
> > >> >>>> the
> > >> >>>> > >> > sessions if there is interest and publish them out.
> > >> >>>> > >> >
> > >> >>>> > >> > Thanks
> > >> >>>> > >> >
> > >> >>>> > >> > Jeff
> > >> >>>> > >> >
> > >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> > >> jay.kreps@gmail.com>
> > >> >>>> wrote:
> > >> >>>> > >> >
> > >> >>>> > >> > > Let's try to get the technical hang-ups sorted out,
> > though.
> > >> I
> > >> >>>> really
> > >> >>>> > >> > think
> > >> >>>> > >> > > there is some benefit to live discussion vs writing. I
> am
> > >> >>>> hopeful that
> > >> >>>> > >> if
> > >> >>>> > >> > > we post instructions and give ourselves a few attempts
> we
> > >> can
> > >> >>>> get it
> > >> >>>> > >> > > working.
> > >> >>>> > >> > >
> > >> >>>> > >> > > Tuesday at that time would work for me...any objections?
> > >> >>>> > >> > >
> > >> >>>> > >> > > -Jay
> > >> >>>> > >> > >
> > >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> > >> joe.stein@stealth.ly
> > >> >>>> >
> > >> >>>> > >> wrote:
> > >> >>>> > >> > >
> > >> >>>> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm
> ET
> > /
> > >> 10am
> > >> >>>> PT
> > >> >>>> > >> ????
> > >> >>>> > >> > > >
> > >> >>>> > >> > > > I don't mind google hangout but there is always some
> > >> issue or
> > >> >>>> > >> whatever
> > >> >>>> > >> > so
> > >> >>>> > >> > > > we know the apache irc channel works. We can start
> there
> > >> and
> > >> >>>> see how
> > >> >>>> > >> it
> > >> >>>> > >> > > > goes? We can pull transcripts too and associate to
> > >> tickets if
> > >> >>>> need be
> > >> >>>> > >> > > makes
> > >> >>>> > >> > > > it helpful for things.
> > >> >>>> > >> > > >
> > >> >>>> > >> > > > ~ Joestein
> > >> >>>> > >> > > >
> > >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> > >> >>>> jay.kreps@gmail.com>
> > >> >>>> > >> > wrote:
> > >> >>>> > >> > > >
> > >> >>>> > >> > > > > We'd talked about doing a Google Hangout to chat
> about
> > >> this.
> > >> >>>> What
> > >> >>>> > >> > about
> > >> >>>> > >> > > > > generalizing that a little further...I actually
> think
> > it
> > >> >>>> would be
> > >> >>>> > >> > good
> > >> >>>> > >> > > > for
> > >> >>>> > >> > > > > everyone spending a reasonable chunk of their week
> on
> > >> Kafka
> > >> >>>> stuff
> > >> >>>> > >> to
> > >> >>>> > >> > > > maybe
> > >> >>>> > >> > > > > sync up once a week. I think we could use time to
> talk
> > >> >>>> through
> > >> >>>> > >> design
> > >> >>>> > >> > > > > stuff, make sure we are on top of code reviews, talk
> > >> through
> > >> >>>> any
> > >> >>>> > >> > tricky
> > >> >>>> > >> > > > > issues, etc.
> > >> >>>> > >> > > > >
> > >> >>>> > >> > > > > We can make it publicly available so that any one
> can
> > >> follow
> > >> >>>> along
> > >> >>>> > >> > who
> > >> >>>> > >> > > > > likes.
> > >> >>>> > >> > > > >
> > >> >>>> > >> > > > > Any interest in doing this? If so I'll try to set it
> > up
> > >> >>>> starting
> > >> >>>> > >> next
> > >> >>>> > >> > > > week.
> > >> >>>> > >> > > > >
> > >> >>>> > >> > > > > -Jay
> > >> >>>> > >> > > > >
> > >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> > >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> >>>> > >> > > > >
> > >> >>>> > >> > > > > > Hi all,
> > >> >>>> > >> > > > > >
> > >> >>>> > >> > > > > > I've updated KIP page, fixed / aligned document
> > >> structure.
> > >> >>>> Also I
> > >> >>>> > >> > > added
> > >> >>>> > >> > > > > > some
> > >> >>>> > >> > > > > > very initial proposal for AdminClient so we have
> > >> something
> > >> >>>> to
> > >> >>>> > >> start
> > >> >>>> > >> > > > from
> > >> >>>> > >> > > > > > while
> > >> >>>> > >> > > > > > discussing the KIP.
> > >> >>>> > >> > > > > >
> > >> >>>> > >> > > > > >
> > >> >>>> > >> > > > > >
> > >> >>>> > >> > > > >
> > >> >>>> > >> > > >
> > >> >>>> > >> > >
> > >> >>>> > >> >
> > >> >>>> > >>
> > >> >>>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >> >>>> > >> > > > > >
> > >> >>>> > >> > > > > > Thanks,
> > >> >>>> > >> > > > > > Andrii Biletskyi
> > >> >>>> > >> > > > > >
> > >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi
> <
> > >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> >>>> > >> > > > > >
> > >> >>>> > >> > > > > > > Jay,
> > >> >>>> > >> > > > > > >
> > >> >>>> > >> > > > > > > Re error messages: you are right, in most cases
> > >> client
> > >> >>>> will
> > >> >>>> > >> have
> > >> >>>> > >> > > > enough
> > >> >>>> > >> > > > > > > context to show descriptive error message. My
> > >> concern is
> > >> >>>> that
> > >> >>>> > >> we
> > >> >>>> > >> > > will
> > >> >>>> > >> > > > > > have
> > >> >>>> > >> > > > > > > to
> > >> >>>> > >> > > > > > > add lots of new error codes for each possible
> > >> error. Of
> > >> >>>> course,
> > >> >>>> > >> > we
> > >> >>>> > >> > > > > could
> > >> >>>> > >> > > > > > > reuse
> > >> >>>> > >> > > > > > > some of existing like
> UknownTopicOrPartitionCode,
> > >> but we
> > >> >>>> will
> > >> >>>> > >> > also
> > >> >>>> > >> > > > need
> > >> >>>> > >> > > > > > to
> > >> >>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> > >> >>>> TopicConfigInvalid (both
> > >> >>>> > >> > for
> > >> >>>> > >> > > > > topic
> > >> >>>> > >> > > > > > > name and config, and probably user would like to
> > >> know
> > >> >>>> what
> > >> >>>> > >> > exactly
> > >> >>>> > >> > > > > > > is wrong in his config),
> InvalidReplicaAssignment,
> > >> >>>> > >> InternalError
> > >> >>>> > >> > > > (e.g.
> > >> >>>> > >> > > > > > > zookeeper failure) etc.
> > >> >>>> > >> > > > > > > And this is only for TopicCommand, we will also
> > >> need to
> > >> >>>> add
> > >> >>>> > >> > similar
> > >> >>>> > >> > > > > stuff
> > >> >>>> > >> > > > > > > for
> > >> >>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll
> end
> > >> up
> > >> >>>> with a
> > >> >>>> > >> > large
> > >> >>>> > >> > > > list
> > >> >>>> > >> > > > > > of
> > >> >>>> > >> > > > > > > error codes, used only in Admin protocol.
> > >> >>>> > >> > > > > > > Having said that, I agree my proposal is not
> > >> consistent
> > >> >>>> with
> > >> >>>> > >> > other
> > >> >>>> > >> > > > > cases.
> > >> >>>> > >> > > > > > > Maybe we can find better solution or something
> > >> >>>> in-between.
> > >> >>>> > >> > > > > > >
> > >> >>>> > >> > > > > > > Re Hangout chat: I think it is a great idea.
> This
> > >> way we
> > >> >>>> can
> > >> >>>> > >> move
> > >> >>>> > >> > > on
> > >> >>>> > >> > > > > > > faster.
> > >> >>>> > >> > > > > > > Let's agree somehow on date/time so people can
> > join.
> > >> >>>> Will work
> > >> >>>> > >> > for
> > >> >>>> > >> > > me
> > >> >>>> > >> > > > > > this
> > >> >>>> > >> > > > > > > and
> > >> >>>> > >> > > > > > > next week almost anytime if agreed in advance.
> > >> >>>> > >> > > > > > >
> > >> >>>> > >> > > > > > > Thanks,
> > >> >>>> > >> > > > > > > Andrii
> > >> >>>> > >> > > > > > >
> > >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> > >> >>>> > >> jay.kreps@gmail.com>
> > >> >>>> > >> > > > > wrote:
> > >> >>>> > >> > > > > > >
> > >> >>>> > >> > > > > > >> Hey Andrii,
> > >> >>>> > >> > > > > > >>
> > >> >>>> > >> > > > > > >> Generally we can do good error handling without
> > >> needing
> > >> >>>> custom
> > >> >>>> > >> > > > > > server-side
> > >> >>>> > >> > > > > > >> messages. I.e. generally the client has the
> > >> context to
> > >> >>>> know
> > >> >>>> > >> that
> > >> >>>> > >> > > if
> > >> >>>> > >> > > > it
> > >> >>>> > >> > > > > > got
> > >> >>>> > >> > > > > > >> an error that the topic doesn't exist to say
> > >> "Topic X
> > >> >>>> doesn't
> > >> >>>> > >> > > exist"
> > >> >>>> > >> > > > > > >> rather
> > >> >>>> > >> > > > > > >> than "error code 14" (or whatever). Maybe there
> > are
> > >> >>>> specific
> > >> >>>> > >> > cases
> > >> >>>> > >> > > > > where
> > >> >>>> > >> > > > > > >> this is hard? If we want to add server-side
> error
> > >> >>>> messages we
> > >> >>>> > >> > > really
> > >> >>>> > >> > > > > do
> > >> >>>> > >> > > > > > >> need to do this in a consistent way across the
> > >> protocol.
> > >> >>>> > >> > > > > > >>
> > >> >>>> > >> > > > > > >> I still have a bunch of open questions here
> from
> > my
> > >> >>>> previous
> > >> >>>> > >> > > list. I
> > >> >>>> > >> > > > > > will
> > >> >>>> > >> > > > > > >> be out for the next few days for Strata though.
> > >> Maybe
> > >> >>>> we could
> > >> >>>> > >> > do
> > >> >>>> > >> > > a
> > >> >>>> > >> > > > > > Google
> > >> >>>> > >> > > > > > >> Hangout chat on any open issues some time
> towards
> > >> the
> > >> >>>> end of
> > >> >>>> > >> > next
> > >> >>>> > >> > > > week
> > >> >>>> > >> > > > > > for
> > >> >>>> > >> > > > > > >> anyone interested in this ticket? I have a
> > feeling
> > >> that
> > >> >>>> might
> > >> >>>> > >> > > > progress
> > >> >>>> > >> > > > > > >> things a little faster than email--I think we
> > >> could talk
> > >> >>>> > >> through
> > >> >>>> > >> > > > those
> > >> >>>> > >> > > > > > >> issues I brought up fairly quickly...
> > >> >>>> > >> > > > > > >>
> > >> >>>> > >> > > > > > >> -Jay
> > >> >>>> > >> > > > > > >>
> > >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii
> > Biletskyi <
> > >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > >> >>>> > >> > > > > > >>
> > >> >>>> > >> > > > > > >> > Hi all,
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >> > I'm trying to address some of the issues
> which
> > >> were
> > >> >>>> > >> mentioned
> > >> >>>> > >> > > > > earlier
> > >> >>>> > >> > > > > > >> about
> > >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of those was about
> > >> batching
> > >> >>>> > >> > operations.
> > >> >>>> > >> > > > What
> > >> >>>> > >> > > > > > if
> > >> >>>> > >> > > > > > >> we
> > >> >>>> > >> > > > > > >> > follow TopicCommand approach and let people
> > >> specify
> > >> >>>> > >> topic-name
> > >> >>>> > >> > > by
> > >> >>>> > >> > > > > > >> regexp -
> > >> >>>> > >> > > > > > >> > would that cover most of the use cases?
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >> > Secondly, is what information should we
> > generally
> > >> >>>> provide in
> > >> >>>> > >> > > Admin
> > >> >>>> > >> > > > > > >> > responses.
> > >> >>>> > >> > > > > > >> > I realize that Admin commands don't imply
> they
> > >> will
> > >> >>>> be used
> > >> >>>> > >> > only
> > >> >>>> > >> > > > in
> > >> >>>> > >> > > > > > CLI
> > >> >>>> > >> > > > > > >> > but,
> > >> >>>> > >> > > > > > >> > it seems to me, CLI is a very important
> client
> > >> of this
> > >> >>>> > >> > feature.
> > >> >>>> > >> > > In
> > >> >>>> > >> > > > > > this
> > >> >>>> > >> > > > > > >> > case,
> > >> >>>> > >> > > > > > >> > seems logical, we would like to provide users
> > >> with
> > >> >>>> rich
> > >> >>>> > >> > > experience
> > >> >>>> > >> > > > > in
> > >> >>>> > >> > > > > > >> terms
> > >> >>>> > >> > > > > > >> > of
> > >> >>>> > >> > > > > > >> > getting results / errors of the executed
> > >> commands.
> > >> >>>> Usually
> > >> >>>> > >> we
> > >> >>>> > >> > > > supply
> > >> >>>> > >> > > > > > >> with
> > >> >>>> > >> > > > > > >> > responses only errorCode, which looks very
> > >> limiting,
> > >> >>>> in case
> > >> >>>> > >> > of
> > >> >>>> > >> > > > CLI
> > >> >>>> > >> > > > > we
> > >> >>>> > >> > > > > > >> may
> > >> >>>> > >> > > > > > >> > want to print human readable error
> description.
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >> > So, taking into account previous item about
> > >> batching,
> > >> >>>> what
> > >> >>>> > >> do
> > >> >>>> > >> > > you
> > >> >>>> > >> > > > > > think
> > >> >>>> > >> > > > > > >> > about
> > >> >>>> > >> > > > > > >> > having smth like:
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >> > ('create' doesn't support regexp)
> > >> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions
> > >> Replicas
> > >> >>>> > >> > > > > ReplicaAssignment
> > >> >>>> > >> > > > > > >> > [Config]
> > >> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode
> > ErrorDescription
> > >> >>>> > >> > > > > > >> >   ErrorCode => int16
> > >> >>>> > >> > > > > > >> >   ErrorDescription => string (empty if
> > >> successful)
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp
> Partitions
> > >> >>>> > >> > > ReplicaAssignment
> > >> >>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> > >> >>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
> > >> >>>> ErrorDescription]
> > >> >>>> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
> > >> >>>> > >> > > > > > >> >   CommandErrorCode => int16
> > >> >>>> > >> > > > > > >> >   CommandErrorDescription => string (nonempty
> > in
> > >> case
> > >> >>>> of
> > >> >>>> > >> fatal
> > >> >>>> > >> > > > > error,
> > >> >>>> > >> > > > > > >> e.g.
> > >> >>>> > >> > > > > > >> > we couldn't get topics by regexp)
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> > >> >>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> > >> TopicDescription
> > >> >>>> > >> ErrorCode
> > >> >>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> > >> >>>> CommandErrorDescription
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >> > Also, any thoughts about our discussion
> > regarding
> > >> >>>> re-routing
> > >> >>>> > >> > > > > facility?
> > >> >>>> > >> > > > > > >> In
> > >> >>>> > >> > > > > > >> > my
> > >> >>>> > >> > > > > > >> > understanding, it is like between augmenting
> > >> >>>> > >> > > TopicMetadataRequest
> > >> >>>> > >> > > > > > >> > (to include at least controllerId) and
> > >> implementing
> > >> >>>> new
> > >> >>>> > >> > generic
> > >> >>>> > >> > > > > > >> re-routing
> > >> >>>> > >> > > > > > >> > facility so sending messages to controller
> will
> > >> be
> > >> >>>> handled
> > >> >>>> > >> by
> > >> >>>> > >> > > it.
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >> > Thanks,
> > >> >>>> > >> > > > > > >> > Andrii Biletskyi
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii
> > >> Biletskyi <
> > >> >>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >> > > @Guozhang:
> > >> >>>> > >> > > > > > >> > > Thanks for your comments, I've answered
> some
> > of
> > >> >>>> those. The
> > >> >>>> > >> > > main
> > >> >>>> > >> > > > > > thing
> > >> >>>> > >> > > > > > >> is
> > >> >>>> > >> > > > > > >> > > having merged request for
> > >> >>>> create-alter-delete-describe - I
> > >> >>>> > >> > > have
> > >> >>>> > >> > > > > some
> > >> >>>> > >> > > > > > >> > > concerns about this approach.
> > >> >>>> > >> > > > > > >> > >
> > >> >>>> > >> > > > > > >> > > @*Jay*:
> > >> >>>> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest
> is
> > >> also
> > >> >>>> one of
> > >> >>>> > >> > the
> > >> >>>> > >> > > > > > >> concerns.
> > >> >>>> > >> > > > > > >> > We
> > >> >>>> > >> > > > > > >> > > can solve it if we implement re-routing
> > >> facility.
> > >> >>>> But I
> > >> >>>> > >> > agree
> > >> >>>> > >> > > > with
> > >> >>>> > >> > > > > > >> > > Guozhang - it will make clients' internals
> a
> > >> little
> > >> >>>> bit
> > >> >>>> > >> > easier
> > >> >>>> > >> > > > but
> > >> >>>> > >> > > > > > >> this
> > >> >>>> > >> > > > > > >> > > seems to be a complex logic to implement
> and
> > >> >>>> support then.
> > >> >>>> > >> > > > > > Especially
> > >> >>>> > >> > > > > > >> for
> > >> >>>> > >> > > > > > >> > > Fetch and Produce (even if we add
> re-routing
> > >> later
> > >> >>>> for
> > >> >>>> > >> these
> > >> >>>> > >> > > > > > >> requests).
> > >> >>>> > >> > > > > > >> > > Also people will tend to avoid this
> > re-routing
> > >> >>>> facility
> > >> >>>> > >> and
> > >> >>>> > >> > > hold
> > >> >>>> > >> > > > > > local
> > >> >>>> > >> > > > > > >> > > cluster cache to ensure their high-priority
> > >> requests
> > >> >>>> > >> (which
> > >> >>>> > >> > > some
> > >> >>>> > >> > > > > of
> > >> >>>> > >> > > > > > >> the
> > >> >>>> > >> > > > > > >> > > admin requests are) not sent to some busy
> > >> broker
> > >> >>>> where
> > >> >>>> > >> they
> > >> >>>> > >> > > wait
> > >> >>>> > >> > > > > to
> > >> >>>> > >> > > > > > be
> > >> >>>> > >> > > > > > >> > > routed to the correct one.
> > >> >>>> > >> > > > > > >> > > As pointed out by Jun here (
> > >> >>>> > >> > > > > > >> > >
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >>
> > >> >>>> > >> > > > > >
> > >> >>>> > >> > > > >
> > >> >>>> > >> > > >
> > >> >>>> > >> > >
> > >> >>>> > >> >
> > >> >>>> > >>
> > >> >>>>
> > >>
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > >> >>>> > >> > > > > > >> > )
> > >> >>>> > >> > > > > > >> > > to solve the issue we might introduce a
> > message
> > >> >>>> type to
> > >> >>>> > >> get
> > >> >>>> > >> > > > > cluster
> > >> >>>> > >> > > > > > >> > state.
> > >> >>>> > >> > > > > > >> > > But I agree we can just update
> > >> >>>> TopicMetadataResponse to
> > >> >>>> > >> > > include
> > >> >>>> > >> > > > > > >> > > controllerId (and probably smth else).
> > >> >>>> > >> > > > > > >> > > What are you thougths?
> > >> >>>> > >> > > > > > >> > >
> > >> >>>> > >> > > > > > >> > > Thanks,
> > >> >>>> > >> > > > > > >> > > Andrii
> > >> >>>> > >> > > > > > >> > >
> > >> >>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang
> > Wang
> > >> <
> > >> >>>> > >> > > > > wangguoz@gmail.com>
> > >> >>>> > >> > > > > > >> > wrote:
> > >> >>>> > >> > > > > > >> > >
> > >> >>>> > >> > > > > > >> > >> I think for the topics commands we can
> > >> actually
> > >> >>>> merge
> > >> >>>> > >> > > > > > >> > >> create/alter/delete/describe as one
> request
> > >> type
> > >> >>>> since
> > >> >>>> > >> > their
> > >> >>>> > >> > > > > > formats
> > >> >>>> > >> > > > > > >> are
> > >> >>>> > >> > > > > > >> > >> very much similar, and keep list-topics
> and
> > >> others
> > >> >>>> like
> > >> >>>> > >> > > > > > >> > >> partition-reassignment /
> > >> preferred-leader-election
> > >> >>>> as
> > >> >>>> > >> > > separate
> > >> >>>> > >> > > > > > >> request
> > >> >>>> > >> > > > > > >> > >> types, I also left some other comments on
> > the
> > >> RB (
> > >> >>>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> > >> >>>> > >> > > > > > >> > >>
> > >> >>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay
> Kreps <
> > >> >>>> > >> > > > jay.kreps@gmail.com>
> > >> >>>> > >> > > > > > >> wrote:
> > >> >>>> > >> > > > > > >> > >>
> > >> >>>> > >> > > > > > >> > >> > Yeah I totally agree that we don't want
> to
> > >> just
> > >> >>>> have
> > >> >>>> > >> one
> > >> >>>> > >> > > "do
> > >> >>>> > >> > > > > > admin
> > >> >>>> > >> > > > > > >> > >> stuff"
> > >> >>>> > >> > > > > > >> > >> > command that has the union of all
> > >> parameters.
> > >> >>>> > >> > > > > > >> > >> >
> > >> >>>> > >> > > > > > >> > >> > What I am saying is that command line
> > tools
> > >> are
> > >> >>>> one
> > >> >>>> > >> > client
> > >> >>>> > >> > > of
> > >> >>>> > >> > > > > the
> > >> >>>> > >> > > > > > >> > >> > administrative apis, but these will be
> > used
> > >> in a
> > >> >>>> number
> > >> >>>> > >> > of
> > >> >>>> > >> > > > > > >> scenarios
> > >> >>>> > >> > > > > > >> > so
> > >> >>>> > >> > > > > > >> > >> > they should make logical sense even in
> the
> > >> >>>> absence of
> > >> >>>> > >> the
> > >> >>>> > >> > > > > command
> > >> >>>> > >> > > > > > >> line
> > >> >>>> > >> > > > > > >> > >> > tool. Hence comments like trying to
> > clarify
> > >> the
> > >> >>>> > >> > > relationship
> > >> >>>> > >> > > > > > >> between
> > >> >>>> > >> > > > > > >> > >> > ClusterMetadata and
> TopicMetadata...these
> > >> kinds
> > >> >>>> of
> > >> >>>> > >> things
> > >> >>>> > >> > > > > really
> > >> >>>> > >> > > > > > >> need
> > >> >>>> > >> > > > > > >> > >> to be
> > >> >>>> > >> > > > > > >> > >> > thought through.
> > >> >>>> > >> > > > > > >> > >> >
> > >> >>>> > >> > > > > > >> > >> > Hope that makes sense.
> > >> >>>> > >> > > > > > >> > >> >
> > >> >>>> > >> > > > > > >> > >> > -Jay
> > >> >>>> > >> > > > > > >> > >> >
> > >> >>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
> > >> >>>> Biletskyi <
> > >> >>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > >> >>>> > >> > > > > > >> > >> >
> > >> >>>> > >> > > > > > >> > >> > > Jay,
> > >> >>>> > >> > > > > > >> > >> > >
> > >> >>>> > >> > > > > > >> > >> > > Thanks for answering. You understood
> > >> >>>> correctly, most
> > >> >>>> > >> of
> > >> >>>> > >> > > my
> > >> >>>> > >> > > > > > >> comments
> > >> >>>> > >> > > > > > >> > >> were
> > >> >>>> > >> > > > > > >> > >> > > related to your point 1) - about "well
> > >> >>>> thought-out"
> > >> >>>> > >> > apis.
> > >> >>>> > >> > > > > Also,
> > >> >>>> > >> > > > > > >> yes,
> > >> >>>> > >> > > > > > >> > >> as I
> > >> >>>> > >> > > > > > >> > >> > > understood we would like to introduce
> a
> > >> single
> > >> >>>> > >> unified
> > >> >>>> > >> > > CLI
> > >> >>>> > >> > > > > tool
> > >> >>>> > >> > > > > > >> with
> > >> >>>> > >> > > > > > >> > >> > > centralized server-side request
> handling
> > >> for
> > >> >>>> lots of
> > >> >>>> > >> > > > existing
> > >> >>>> > >> > > > > > >> ones
> > >> >>>> > >> > > > > > >> > >> (incl.
> > >> >>>> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> > >> >>>> > >> ReassignPartitions,
> > >> >>>> > >> > > smth
> > >> >>>> > >> > > > > > else
> > >> >>>> > >> > > > > > >> if
> > >> >>>> > >> > > > > > >> > >> added
> > >> >>>> > >> > > > > > >> > >> > > in future). In our previous
> discussion (
> > >> >>>> > >> > > > > > >> > >> > >
> > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694)
> > >> >>>> > >> > people
> > >> >>>> > >> > > > > said
> > >> >>>> > >> > > > > > >> > they'd
> > >> >>>> > >> > > > > > >> > >> > > rather
> > >> >>>> > >> > > > > > >> > >> > > have a separate message for each
> > command,
> > >> so,
> > >> >>>> yes,
> > >> >>>> > >> this
> > >> >>>> > >> > > > way I
> > >> >>>> > >> > > > > > >> came
> > >> >>>> > >> > > > > > >> > to
> > >> >>>> > >> > > > > > >> > >> 1-1
> > >> >>>> > >> > > > > > >> > >> > > mapping between commands in the tool
> and
> > >> >>>> protocol
> > >> >>>> > >> > > > additions.
> > >> >>>> > >> > > > > > But
> > >> >>>> > >> > > > > > >> I
> > >> >>>> > >> > > > > > >> > >> might
> > >> >>>> > >> > > > > > >> > >> > be
> > >> >>>> > >> > > > > > >> > >> > > wrong.
> > >> >>>> > >> > > > > > >> > >> > > At the end I just try to start
> > discussion
> > >> how
> > >> >>>> at
> > >> >>>> > >> least
> > >> >>>> > >> > > > > > generally
> > >> >>>> > >> > > > > > >> > this
> > >> >>>> > >> > > > > > >> > >> > > protocol should look like.
> > >> >>>> > >> > > > > > >> > >> > >
> > >> >>>> > >> > > > > > >> > >> > > Thanks,
> > >> >>>> > >> > > > > > >> > >> > > Andrii
> > >> >>>> > >> > > > > > >> > >> > >
> > >> >>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay
> > >> Kreps <
> > >> >>>> > >> > > > > > jay.kreps@gmail.com
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >> > >> wrote:
> > >> >>>> > >> > > > > > >> > >> > >
> > >> >>>> > >> > > > > > >> > >> > > > Hey Andrii,
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> > >> > > > > > >> > >> > > > To answer your earlier question we
> > just
> > >> >>>> really
> > >> >>>> > >> can't
> > >> >>>> > >> > be
> > >> >>>> > >> > > > > > adding
> > >> >>>> > >> > > > > > >> any
> > >> >>>> > >> > > > > > >> > >> more
> > >> >>>> > >> > > > > > >> > >> > > > scala protocol objects. These things
> > are
> > >> >>>> super hard
> > >> >>>> > >> > to
> > >> >>>> > >> > > > > > maintain
> > >> >>>> > >> > > > > > >> > >> because
> > >> >>>> > >> > > > > > >> > >> > > > they hand code the byte parsing and
> > >> don't
> > >> >>>> have good
> > >> >>>> > >> > > > > > versioning
> > >> >>>> > >> > > > > > >> > >> support.
> > >> >>>> > >> > > > > > >> > >> > > > Since we are already planning on
> > >> converting
> > >> >>>> we
> > >> >>>> > >> > > definitely
> > >> >>>> > >> > > > > > don't
> > >> >>>> > >> > > > > > >> > >> want to
> > >> >>>> > >> > > > > > >> > >> > > add
> > >> >>>> > >> > > > > > >> > >> > > > a ton more of these--they are total
> > tech
> > >> >>>> debt.
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> > >> > > > > > >> > >> > > > What does it mean that the changes
> are
> > >> >>>> isolated
> > >> >>>> > >> from
> > >> >>>> > >> > > the
> > >> >>>> > >> > > > > > >> current
> > >> >>>> > >> > > > > > >> > >> code
> > >> >>>> > >> > > > > > >> > >> > > base?
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> > >> > > > > > >> > >> > > > I actually didn't understand the
> > >> remaining
> > >> >>>> > >> comments,
> > >> >>>> > >> > > > which
> > >> >>>> > >> > > > > of
> > >> >>>> > >> > > > > > >> the
> > >> >>>> > >> > > > > > >> > >> > points
> > >> >>>> > >> > > > > > >> > >> > > > are you responding to?
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> > >> > > > > > >> > >> > > > Maybe one sticking point here is
> that
> > it
> > >> >>>> seems like
> > >> >>>> > >> > you
> > >> >>>> > >> > > > > want
> > >> >>>> > >> > > > > > to
> > >> >>>> > >> > > > > > >> > make
> > >> >>>> > >> > > > > > >> > >> > some
> > >> >>>> > >> > > > > > >> > >> > > > kind of tool, and you have made a
> 1-1
> > >> mapping
> > >> >>>> > >> between
> > >> >>>> > >> > > > > > commands
> > >> >>>> > >> > > > > > >> you
> > >> >>>> > >> > > > > > >> > >> > > imagine
> > >> >>>> > >> > > > > > >> > >> > > > in the tool and protocol additions.
> I
> > >> want
> > >> >>>> to make
> > >> >>>> > >> > sure
> > >> >>>> > >> > > > we
> > >> >>>> > >> > > > > > >> don't
> > >> >>>> > >> > > > > > >> > do
> > >> >>>> > >> > > > > > >> > >> > that.
> > >> >>>> > >> > > > > > >> > >> > > > The protocol needs to be really
> really
> > >> well
> > >> >>>> thought
> > >> >>>> > >> > out
> > >> >>>> > >> > > > > > against
> > >> >>>> > >> > > > > > >> > many
> > >> >>>> > >> > > > > > >> > >> > use
> > >> >>>> > >> > > > > > >> > >> > > > cases so it should make perfect
> > logical
> > >> >>>> sense in
> > >> >>>> > >> the
> > >> >>>> > >> > > > > absence
> > >> >>>> > >> > > > > > of
> > >> >>>> > >> > > > > > >> > >> knowing
> > >> >>>> > >> > > > > > >> > >> > > the
> > >> >>>> > >> > > > > > >> > >> > > > command line tool, right?
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> > >> > > > > > >> > >> > > > -Jay
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM,
> > Andrii
> > >> >>>> Biletskyi
> > >> >>>> > >> <
> > >> >>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> > >> > > > > > >> > >> > > > > Hey Jay,
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > I would like to continue this
> > >> discussion
> > >> >>>> as it
> > >> >>>> > >> seem
> > >> >>>> > >> > > > there
> > >> >>>> > >> > > > > > is
> > >> >>>> > >> > > > > > >> no
> > >> >>>> > >> > > > > > >> > >> > > progress
> > >> >>>> > >> > > > > > >> > >> > > > > here.
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > First of all, could you please
> > explain
> > >> >>>> what did
> > >> >>>> > >> you
> > >> >>>> > >> > > > mean
> > >> >>>> > >> > > > > in
> > >> >>>> > >> > > > > > >> 2?
> > >> >>>> > >> > > > > > >> > How
> > >> >>>> > >> > > > > > >> > >> > > > exactly
> > >> >>>> > >> > > > > > >> > >> > > > > are we going to migrate to the new
> > >> java
> > >> >>>> protocol
> > >> >>>> > >> > > > > > definitions.
> > >> >>>> > >> > > > > > >> > And
> > >> >>>> > >> > > > > > >> > >> why
> > >> >>>> > >> > > > > > >> > >> > > > it's
> > >> >>>> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > I agree with you, this feature
> > >> includes
> > >> >>>> lots of
> > >> >>>> > >> > > stuff,
> > >> >>>> > >> > > > > but
> > >> >>>> > >> > > > > > >> > >> thankfully
> > >> >>>> > >> > > > > > >> > >> > > > > almost all changes are isolated
> from
> > >> the
> > >> >>>> current
> > >> >>>> > >> > code
> > >> >>>> > >> > > > > base,
> > >> >>>> > >> > > > > > >> > >> > > > > so the main thing, I think, we
> need
> > to
> > >> >>>> agree is
> > >> >>>> > >> > RQ/RP
> > >> >>>> > >> > > > > > format.
> > >> >>>> > >> > > > > > >> > >> > > > > So how can we start discussion
> about
> > >> the
> > >> >>>> concrete
> > >> >>>> > >> > > > > messages
> > >> >>>> > >> > > > > > >> > format?
> > >> >>>> > >> > > > > > >> > >> > > > > Can we take (
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> > >> > > > > > >> > >> > >
> > >> >>>> > >> > > > > > >> > >> >
> > >> >>>> > >> > > > > > >> > >>
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >>
> > >> >>>> > >> > > > > >
> > >> >>>> > >> > > > >
> > >> >>>> > >> > > >
> > >> >>>> > >> > >
> > >> >>>> > >> >
> > >> >>>> > >>
> > >> >>>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > >> >>>> > >> > > > > > >> > >> > > > > )
> > >> >>>> > >> > > > > > >> > >> > > > > as starting point?
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > We had some doubts earlier whether
> > it
> > >> worth
> > >> >>>> > >> > > introducing
> > >> >>>> > >> > > > > one
> > >> >>>> > >> > > > > > >> > >> generic
> > >> >>>> > >> > > > > > >> > >> > > Admin
> > >> >>>> > >> > > > > > >> > >> > > > > Request for all commands (
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694
> > >> >>>> > >> > > > > > >> > >> > > > > )
> > >> >>>> > >> > > > > > >> > >> > > > > but then everybody agreed it would
> > be
> > >> >>>> better to
> > >> >>>> > >> > have
> > >> >>>> > >> > > > > > separate
> > >> >>>> > >> > > > > > >> > >> message
> > >> >>>> > >> > > > > > >> > >> > > for
> > >> >>>> > >> > > > > > >> > >> > > > > each admin command. The Request
> part
> > >> is
> > >> >>>> really
> > >> >>>> > >> > > dictated
> > >> >>>> > >> > > > > > from
> > >> >>>> > >> > > > > > >> the
> > >> >>>> > >> > > > > > >> > >> > > command
> > >> >>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments
> > itself,
> > >> so
> > >> >>>> the
> > >> >>>> > >> > proposed
> > >> >>>> > >> > > > > > version
> > >> >>>> > >> > > > > > >> > >> should
> > >> >>>> > >> > > > > > >> > >> > be
> > >> >>>> > >> > > > > > >> > >> > > > > fine (let's put aside for now
> > remarks
> > >> about
> > >> >>>> > >> > Optional
> > >> >>>> > >> > > > > type,
> > >> >>>> > >> > > > > > >> > >> batching,
> > >> >>>> > >> > > > > > >> > >> > > > > configs normalization - I agree
> with
> > >> all of
> > >> >>>> > >> them).
> > >> >>>> > >> > > > > > >> > >> > > > > So the second part is Response. I
> > see
> > >> >>>> there are
> > >> >>>> > >> two
> > >> >>>> > >> > > > cases
> > >> >>>> > >> > > > > > >> here.
> > >> >>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> > >> Create/Alter/... ;
> > >> >>>> b)
> > >> >>>> > >> "Get"
> > >> >>>> > >> > > > > > requests -
> > >> >>>> > >> > > > > > >> > >> > > > > List/Describe...
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > a) should only hold request result
> > >> >>>> (regardless
> > >> >>>> > >> what
> > >> >>>> > >> > > we
> > >> >>>> > >> > > > > > decide
> > >> >>>> > >> > > > > > >> > >> about
> > >> >>>> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
> > >> execution).
> > >> >>>> > >> > > > > > >> > >> > > > > Usually we provide error code in
> > >> response
> > >> >>>> but
> > >> >>>> > >> since
> > >> >>>> > >> > > we
> > >> >>>> > >> > > > > will
> > >> >>>> > >> > > > > > >> use
> > >> >>>> > >> > > > > > >> > >> this
> > >> >>>> > >> > > > > > >> > >> > in
> > >> >>>> > >> > > > > > >> > >> > > > > interactive shell we need some
> human
> > >> >>>> readable
> > >> >>>> > >> error
> > >> >>>> > >> > > > > > >> description
> > >> >>>> > >> > > > > > >> > -
> > >> >>>> > >> > > > > > >> > >> so
> > >> >>>> > >> > > > > > >> > >> > I
> > >> >>>> > >> > > > > > >> > >> > > > > added errorDesription field where
> > you
> > >> can
> > >> >>>> at
> > >> >>>> > >> least
> > >> >>>> > >> > > > leave
> > >> >>>> > >> > > > > > >> > >> > > > > exception.getMessage.
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > b) in addition to previous item
> > >> message
> > >> >>>> should
> > >> >>>> > >> hold
> > >> >>>> > >> > > > > command
> > >> >>>> > >> > > > > > >> > >> specific
> > >> >>>> > >> > > > > > >> > >> > > > > response data. We can discuss in
> > >> detail
> > >> >>>> each of
> > >> >>>> > >> > them
> > >> >>>> > >> > > > but
> > >> >>>> > >> > > > > > >> let's
> > >> >>>> > >> > > > > > >> > for
> > >> >>>> > >> > > > > > >> > >> > now
> > >> >>>> > >> > > > > > >> > >> > > > > agree about the overall pattern.
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > Thanks,
> > >> >>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM,
> Jay
> > >> Kreps
> > >> >>>> <
> > >> >>>> > >> > > > > > >> jay.kreps@gmail.com
> > >> >>>> > >> > > > > > >> > >
> > >> >>>> > >> > > > > > >> > >> > > wrote:
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > Hey Joe,
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > This is great. A few comments on
> > >> KIP-4
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > 1. This is much needed
> > >> functionality,
> > >> >>>> but there
> > >> >>>> > >> > > are a
> > >> >>>> > >> > > > > lot
> > >> >>>> > >> > > > > > >> of
> > >> >>>> > >> > > > > > >> > >> the so
> > >> >>>> > >> > > > > > >> > >> > > > let's
> > >> >>>> > >> > > > > > >> > >> > > > > > really think these protocols
> > >> through. We
> > >> >>>> really
> > >> >>>> > >> > > want
> > >> >>>> > >> > > > to
> > >> >>>> > >> > > > > > >> end up
> > >> >>>> > >> > > > > > >> > >> > with a
> > >> >>>> > >> > > > > > >> > >> > > > set
> > >> >>>> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol
> > >> apis.
> > >> >>>> For this
> > >> >>>> > >> > > > reason I
> > >> >>>> > >> > > > > > >> think
> > >> >>>> > >> > > > > > >> > >> it is
> > >> >>>> > >> > > > > > >> > >> > > > > really
> > >> >>>> > >> > > > > > >> > >> > > > > > important to think through the
> end
> > >> state
> > >> >>>> even
> > >> >>>> > >> if
> > >> >>>> > >> > > that
> > >> >>>> > >> > > > > > >> includes
> > >> >>>> > >> > > > > > >> > >> APIs
> > >> >>>> > >> > > > > > >> > >> > > we
> > >> >>>> > >> > > > > > >> > >> > > > > > won't implement in the first
> > phase.
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > 2. Let's please please please
> wait
> > >> until
> > >> >>>> we
> > >> >>>> > >> have
> > >> >>>> > >> > > > > switched
> > >> >>>> > >> > > > > > >> the
> > >> >>>> > >> > > > > > >> > >> > server
> > >> >>>> > >> > > > > > >> > >> > > > over
> > >> >>>> > >> > > > > > >> > >> > > > > > to the new java protocol
> > >> definitions. If
> > >> >>>> we add
> > >> >>>> > >> > > > upteen
> > >> >>>> > >> > > > > > >> more ad
> > >> >>>> > >> > > > > > >> > >> hoc
> > >> >>>> > >> > > > > > >> > >> > > > scala
> > >> >>>> > >> > > > > > >> > >> > > > > > objects that is just generating
> > more
> > >> >>>> work for
> > >> >>>> > >> the
> > >> >>>> > >> > > > > > >> conversion
> > >> >>>> > >> > > > > > >> > we
> > >> >>>> > >> > > > > > >> > >> > know
> > >> >>>> > >> > > > > > >> > >> > > we
> > >> >>>> > >> > > > > > >> > >> > > > > > have to do.
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a
> new
> > >> type of
> > >> >>>> > >> > optional
> > >> >>>> > >> > > > > > >> parameter.
> > >> >>>> > >> > > > > > >> > >> This
> > >> >>>> > >> > > > > > >> > >> > is
> > >> >>>> > >> > > > > > >> > >> > > > > > inconsistent with everything
> else
> > >> in the
> > >> >>>> > >> protocol
> > >> >>>> > >> > > > where
> > >> >>>> > >> > > > > > we
> > >> >>>> > >> > > > > > >> use
> > >> >>>> > >> > > > > > >> > >> -1
> > >> >>>> > >> > > > > > >> > >> > or
> > >> >>>> > >> > > > > > >> > >> > > > some
> > >> >>>> > >> > > > > > >> > >> > > > > > other marker value. You could
> > argue
> > >> >>>> either way
> > >> >>>> > >> > but
> > >> >>>> > >> > > > > let's
> > >> >>>> > >> > > > > > >> stick
> > >> >>>> > >> > > > > > >> > >> with
> > >> >>>> > >> > > > > > >> > >> > > > that
> > >> >>>> > >> > > > > > >> > >> > > > > > for consistency. For clients
> that
> > >> >>>> implemented
> > >> >>>> > >> the
> > >> >>>> > >> > > > > > protocol
> > >> >>>> > >> > > > > > >> in
> > >> >>>> > >> > > > > > >> > a
> > >> >>>> > >> > > > > > >> > >> > > better
> > >> >>>> > >> > > > > > >> > >> > > > > way
> > >> >>>> > >> > > > > > >> > >> > > > > > than our scala code these basic
> > >> >>>> primitives are
> > >> >>>> > >> > hard
> > >> >>>> > >> > > > to
> > >> >>>> > >> > > > > > >> change.
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems
> to
> > >> >>>> duplicate
> > >> >>>> > >> > > > > > >> > TopicMetadataRequest
> > >> >>>> > >> > > > > > >> > >> > > which
> > >> >>>> > >> > > > > > >> > >> > > > > has
> > >> >>>> > >> > > > > > >> > >> > > > > > brokers, topics, and
> partitions. I
> > >> think
> > >> >>>> we
> > >> >>>> > >> > should
> > >> >>>> > >> > > > > rename
> > >> >>>> > >> > > > > > >> that
> > >> >>>> > >> > > > > > >> > >> > > request
> > >> >>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> > >> >>>> > >> MetadataRequest)
> > >> >>>> > >> > > and
> > >> >>>> > >> > > > > > >> include
> > >> >>>> > >> > > > > > >> > >> the id
> > >> >>>> > >> > > > > > >> > >> > > of
> > >> >>>> > >> > > > > > >> > >> > > > > the
> > >> >>>> > >> > > > > > >> > >> > > > > > controller. Or are there other
> > >> things we
> > >> >>>> could
> > >> >>>> > >> > add
> > >> >>>> > >> > > > > here?
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to
> > >> make a
> > >> >>>> lot of
> > >> >>>> > >> > > > requests
> > >> >>>> > >> > > > > > that
> > >> >>>> > >> > > > > > >> > can
> > >> >>>> > >> > > > > > >> > >> > only
> > >> >>>> > >> > > > > > >> > >> > > go
> > >> >>>> > >> > > > > > >> > >> > > > > to
> > >> >>>> > >> > > > > > >> > >> > > > > > particular nodes. This adds a
> lot
> > of
> > >> >>>> burden for
> > >> >>>> > >> > > > client
> > >> >>>> > >> > > > > > >> > >> > > implementations
> > >> >>>> > >> > > > > > >> > >> > > > > (it
> > >> >>>> > >> > > > > > >> > >> > > > > > sounds easy but each discovery
> can
> > >> fail
> > >> >>>> in many
> > >> >>>> > >> > > parts
> > >> >>>> > >> > > > > so
> > >> >>>> > >> > > > > > it
> > >> >>>> > >> > > > > > >> > >> ends up
> > >> >>>> > >> > > > > > >> > >> > > > > being a
> > >> >>>> > >> > > > > > >> > >> > > > > > full state machine to do
> right). I
> > >> think
> > >> >>>> we
> > >> >>>> > >> > should
> > >> >>>> > >> > > > > > consider
> > >> >>>> > >> > > > > > >> > >> making
> > >> >>>> > >> > > > > > >> > >> > > > admin
> > >> >>>> > >> > > > > > >> > >> > > > > > commands and ideally as many of
> > the
> > >> >>>> other apis
> > >> >>>> > >> as
> > >> >>>> > >> > > > > > possible
> > >> >>>> > >> > > > > > >> > >> > available
> > >> >>>> > >> > > > > > >> > >> > > on
> > >> >>>> > >> > > > > > >> > >> > > > > all
> > >> >>>> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
> > >> >>>> controller on
> > >> >>>> > >> > the
> > >> >>>> > >> > > > > broker
> > >> >>>> > >> > > > > > >> > side.
> > >> >>>> > >> > > > > > >> > >> > > Perhaps
> > >> >>>> > >> > > > > > >> > >> > > > > > there would be a general way to
> > >> >>>> encapsulate
> > >> >>>> > >> this
> > >> >>>> > >> > > > > > re-routing
> > >> >>>> > >> > > > > > >> > >> > behavior.
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > 6. We should probably normalize
> > the
> > >> key
> > >> >>>> value
> > >> >>>> > >> > pairs
> > >> >>>> > >> > > > > used
> > >> >>>> > >> > > > > > >> for
> > >> >>>> > >> > > > > > >> > >> > configs
> > >> >>>> > >> > > > > > >> > >> > > > > rather
> > >> >>>> > >> > > > > > >> > >> > > > > > than embedding a new formatting.
> > So
> > >> two
> > >> >>>> strings
> > >> >>>> > >> > > > rather
> > >> >>>> > >> > > > > > than
> > >> >>>> > >> > > > > > >> > one
> > >> >>>> > >> > > > > > >> > >> > with
> > >> >>>> > >> > > > > > >> > >> > > an
> > >> >>>> > >> > > > > > >> > >> > > > > > internal equals sign.
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these
> > >> APIs
> > >> >>>> that the
> > >> >>>> > >> > > > command
> > >> >>>> > >> > > > > > has
> > >> >>>> > >> > > > > > >> > >> begun or
> > >> >>>> > >> > > > > > >> > >> > > > that
> > >> >>>> > >> > > > > > >> > >> > > > > > the command has been completed?
> It
> > >> is a
> > >> >>>> lot
> > >> >>>> > >> more
> > >> >>>> > >> > > > usable
> > >> >>>> > >> > > > > > if
> > >> >>>> > >> > > > > > >> the
> > >> >>>> > >> > > > > > >> > >> > > command
> > >> >>>> > >> > > > > > >> > >> > > > > has
> > >> >>>> > >> > > > > > >> > >> > > > > > been completed so you know that
> if
> > >> you
> > >> >>>> create a
> > >> >>>> > >> > > topic
> > >> >>>> > >> > > > > and
> > >> >>>> > >> > > > > > >> then
> > >> >>>> > >> > > > > > >> > >> > > publish
> > >> >>>> > >> > > > > > >> > >> > > > to
> > >> >>>> > >> > > > > > >> > >> > > > > > it you won't get an exception
> > about
> > >> >>>> there being
> > >> >>>> > >> > no
> > >> >>>> > >> > > > such
> > >> >>>> > >> > > > > > >> topic.
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list
> topics
> > >> >>>> duplicate a
> > >> >>>> > >> lot
> > >> >>>> > >> > > of
> > >> >>>> > >> > > > > > stuff
> > >> >>>> > >> > > > > > >> in
> > >> >>>> > >> > > > > > >> > >> the
> > >> >>>> > >> > > > > > >> > >> > > > > metadata
> > >> >>>> > >> > > > > > >> > >> > > > > > request. Is there a reason to
> give
> > >> back
> > >> >>>> topics
> > >> >>>> > >> > > marked
> > >> >>>> > >> > > > > for
> > >> >>>> > >> > > > > > >> > >> > deletion? I
> > >> >>>> > >> > > > > > >> > >> > > > > feel
> > >> >>>> > >> > > > > > >> > >> > > > > > like if we just make the
> > >> post-condition
> > >> >>>> of the
> > >> >>>> > >> > > delete
> > >> >>>> > >> > > > > > >> command
> > >> >>>> > >> > > > > > >> > be
> > >> >>>> > >> > > > > > >> > >> > that
> > >> >>>> > >> > > > > > >> > >> > > > the
> > >> >>>> > >> > > > > > >> > >> > > > > > topic is deleted that will get
> rid
> > >> of
> > >> >>>> the need
> > >> >>>> > >> > for
> > >> >>>> > >> > > > this
> > >> >>>> > >> > > > > > >> right?
> > >> >>>> > >> > > > > > >> > >> And
> > >> >>>> > >> > > > > > >> > >> > it
> > >> >>>> > >> > > > > > >> > >> > > > > will
> > >> >>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > 9. Should we consider batching
> > these
> > >> >>>> requests?
> > >> >>>> > >> We
> > >> >>>> > >> > > > have
> > >> >>>> > >> > > > > > >> > generally
> > >> >>>> > >> > > > > > >> > >> > > tried
> > >> >>>> > >> > > > > > >> > >> > > > to
> > >> >>>> > >> > > > > > >> > >> > > > > > allow multiple operations to be
> > >> batched.
> > >> >>>> My
> > >> >>>> > >> > > suspicion
> > >> >>>> > >> > > > > is
> > >> >>>> > >> > > > > > >> that
> > >> >>>> > >> > > > > > >> > >> > without
> > >> >>>> > >> > > > > > >> > >> > > > > this
> > >> >>>> > >> > > > > > >> > >> > > > > > we will get a lot of code that
> > does
> > >> >>>> something
> > >> >>>> > >> > like
> > >> >>>> > >> > > > > > >> > >> > > > > >    for(topic:
> > >> adminClient.listTopics())
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >>  adminClient.describeTopic(topic)
> > >> >>>> > >> > > > > > >> > >> > > > > > this code will work great when
> you
> > >> test
> > >> >>>> on 5
> > >> >>>> > >> > topics
> > >> >>>> > >> > > > but
> > >> >>>> > >> > > > > > >> not do
> > >> >>>> > >> > > > > > >> > >> as
> > >> >>>> > >> > > > > > >> > >> > > well
> > >> >>>> > >> > > > > > >> > >> > > > if
> > >> >>>> > >> > > > > > >> > >> > > > > > you have 50k.
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > 10. I think we should also
> discuss
> > >> how
> > >> >>>> we want
> > >> >>>> > >> to
> > >> >>>> > >> > > > > expose
> > >> >>>> > >> > > > > > a
> > >> >>>> > >> > > > > > >> > >> > > programmatic
> > >> >>>> > >> > > > > > >> > >> > > > > JVM
> > >> >>>> > >> > > > > > >> > >> > > > > > client api for these operations.
> > >> >>>> Currently
> > >> >>>> > >> people
> > >> >>>> > >> > > > rely
> > >> >>>> > >> > > > > on
> > >> >>>> > >> > > > > > >> > >> > AdminUtils
> > >> >>>> > >> > > > > > >> > >> > > > > which
> > >> >>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we
> > >> probably
> > >> >>>> need
> > >> >>>> > >> > > another
> > >> >>>> > >> > > > > > client
> > >> >>>> > >> > > > > > >> > >> under
> > >> >>>> > >> > > > > > >> > >> > > > > clients/
> > >> >>>> > >> > > > > > >> > >> > > > > > that exposes administrative
> > >> >>>> functionality. We
> > >> >>>> > >> > will
> > >> >>>> > >> > > > need
> > >> >>>> > >> > > > > > >> this
> > >> >>>> > >> > > > > > >> > >> just
> > >> >>>> > >> > > > > > >> > >> > to
> > >> >>>> > >> > > > > > >> > >> > > > > > properly test the new apis, I
> > >> suspect. We
> > >> >>>> > >> should
> > >> >>>> > >> > > > figure
> > >> >>>> > >> > > > > > out
> > >> >>>> > >> > > > > > >> > that
> > >> >>>> > >> > > > > > >> > >> > API.
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > 11. The other information that
> > >> would be
> > >> >>>> really
> > >> >>>> > >> > > useful
> > >> >>>> > >> > > > > to
> > >> >>>> > >> > > > > > >> get
> > >> >>>> > >> > > > > > >> > >> would
> > >> >>>> > >> > > > > > >> > >> > be
> > >> >>>> > >> > > > > > >> > >> > > > > > information about
> partitions--how
> > >> much
> > >> >>>> data is
> > >> >>>> > >> in
> > >> >>>> > >> > > the
> > >> >>>> > >> > > > > > >> > partition,
> > >> >>>> > >> > > > > > >> > >> > what
> > >> >>>> > >> > > > > > >> > >> > > > are
> > >> >>>> > >> > > > > > >> > >> > > > > > the segment offsets, what is the
> > >> log-end
> > >> >>>> offset
> > >> >>>> > >> > > (i.e.
> > >> >>>> > >> > > > > > last
> > >> >>>> > >> > > > > > >> > >> offset),
> > >> >>>> > >> > > > > > >> > >> > > > what
> > >> >>>> > >> > > > > > >> > >> > > > > is
> > >> >>>> > >> > > > > > >> > >> > > > > > the compaction point, etc. I
> think
> > >> that
> > >> >>>> done
> > >> >>>> > >> > right
> > >> >>>> > >> > > > this
> > >> >>>> > >> > > > > > >> would
> > >> >>>> > >> > > > > > >> > be
> > >> >>>> > >> > > > > > >> > >> > the
> > >> >>>> > >> > > > > > >> > >> > > > > > successor to the very awkward
> > >> >>>> OffsetRequest we
> > >> >>>> > >> > have
> > >> >>>> > >> > > > > > today.
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > -Jay
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27
> PM,
> > >> Joe
> > >> >>>> Stein <
> > >> >>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
> > >> >>>> > >> > > > > > >> > >> > > > > wrote:
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> > >> >>>> > >> > > > > > >> > >> > > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> > >> > > > > > >> > >> > >
> > >> >>>> > >> > > > > > >> > >> >
> > >> >>>> > >> > > > > > >> > >>
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >>
> > >> >>>> > >> > > > > >
> > >> >>>> > >> > > > >
> > >> >>>> > >> > > >
> > >> >>>> > >> > >
> > >> >>>> > >> >
> > >> >>>> > >>
> > >> >>>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >> >>>> > >> > > > > > >> > >> > > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > > JIRA
> > >> >>>> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > >> >>>> > >> > > > > > >> > >> > > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > >
> > >> >>>> /*******************************************
> > >> >>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
> > >> >>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> > >> >>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security
> > LLC
> > >> >>>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> > >> >>>> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> > >> >>>> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> > >> > > > > > >> > >> > > > > > >
> > >> >>>> ********************************************/
> > >> >>>> > >> > > > > > >> > >> > > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > > >
> > >> >>>> > >> > > > > > >> > >> > > > >
> > >> >>>> > >> > > > > > >> > >> > > >
> > >> >>>> > >> > > > > > >> > >> > >
> > >> >>>> > >> > > > > > >> > >> >
> > >> >>>> > >> > > > > > >> > >>
> > >> >>>> > >> > > > > > >> > >>
> > >> >>>> > >> > > > > > >> > >>
> > >> >>>> > >> > > > > > >> > >> --
> > >> >>>> > >> > > > > > >> > >> -- Guozhang
> > >> >>>> > >> > > > > > >> > >>
> > >> >>>> > >> > > > > > >> > >
> > >> >>>> > >> > > > > > >> > >
> > >> >>>> > >> > > > > > >> >
> > >> >>>> > >> > > > > > >>
> > >> >>>> > >> > > > > > >
> > >> >>>> > >> > > > > > >
> > >> >>>> > >> > > > > >
> > >> >>>> > >> > > > >
> > >> >>>> > >> > > >
> > >> >>>> > >> > >
> > >> >>>> > >> >
> > >> >>>> > >> >
> > >> >>>> > >> >
> > >> >>>> > >> > --
> > >> >>>> > >> > Jeff Holoman
> > >> >>>> > >> > Systems Engineer
> > >> >>>> > >> >
> > >> >>>> > >>
> > >> >>>>
> > >> >>>>
> > >> >>>
> > >> >>>
> > >> >>> --
> > >> >>> -- Guozhang
> > >> >>>
> > >> >>
> > >> >>
> > >> >>
> > >> >> --
> > >> >> -- Guozhang
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
Andrii,


A few more comments.

100. There are a few fields such as ReplicaAssignment,
ReassignPartitionRequest,
and PartitionsSerialized that are represented as a string, but contain
composite structures in json. Could we flatten them out directly in the
protocol definition as arrays/records?

101. Does TopicMetadataRequest v1 still trigger auto topic creation? This
will be a bit weird now that we have a separate topic creation api. Have
you thought about how the new createTopicRequest and TopicMetadataRequest
v1 will be used in the producer/consumer client, in addition to admin
tools? For example, ideally, we don't want TopicMetadataRequest from the
consumer to trigger auto topic creation.

2. I think Jay meant getting rid of scala classes
like HeartbeatRequestAndHeader and HeartbeatResponseAndHeader. We did that
as a stop-gap thing when adding the new requests for the consumers.
However, the long term plan is to get rid of all those and just reuse the
java request/response in the client. Since this KIP proposes to add a
significant number of new requests, perhaps we should bite the bullet to
clean up the existing scala requests first before adding new ones?

Thanks,

Jun



On Thu, Mar 12, 2015 at 3:37 PM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Hi,
>
> As said above - I list again all comments from this thread so we
> can see what's left and finalize all pending issues.
>
> Comments from Jay:
> 1. This is much needed functionality, but there are a lot of the so let's
> really think these protocols through. We really want to end up with a set
> of well thought-out, orthoganol apis. For this reason I think it is really
> important to think through the end state even if that includes APIs we
> won't implement in the first phase.
>
> A: Definitely behind this. Would appreciate if there are concrete comments
> how this can be improved.
>
> 2. Let's please please please wait until we have switched the server over
> to the new java protocol definitions. If we add upteen more ad hoc scala
> objects that is just generating more work for the conversion we know we
> have to do.
>
> A: Fixed in the latest patch - removed scala protocol classes.
>
> 3. This proposal introduces a new type of optional parameter. This is
> inconsistent with everything else in the protocol where we use -1 or some
> other marker value. You could argue either way but let's stick with that
> for consistency. For clients that implemented the protocol in a better way
> than our scala code these basic primitives are hard to change.
>
> A: Fixed in the latest patch - removed MaybeOf type and changed protocol
> accordingly.
>
> 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest which has
> brokers, topics, and partitions. I think we should rename that request
> ClusterMetadataRequest (or just MetadataRequest) and include the id of the
> controller. Or are there other things we could add here?
>
> A: I agree. Updated the KIP. Let's extends TopicMetadata to version 2 and
> include controller.
>
> 5. We have a tendency to try to make a lot of requests that can only go to
> particular nodes. This adds a lot of burden for client implementations (it
> sounds easy but each discovery can fail in many parts so it ends up being a
> full state machine to do right). I think we should consider making admin
> commands and ideally as many of the other apis as possible available on all
> brokers and just redirect to the controller on the broker side. Perhaps
> there would be a general way to encapsulate this re-routing behavior.
>
> A: It's a very interesting idea, but seems there are some concerns about
> this
> feature (like performance considerations, how this will complicate server
> etc).
> I believe this shouldn't be a blocker. If this feature is implemented at
> some
> point it won't affect Admin changes - at least no changes to public API
> will be required.
>
> 6. We should probably normalize the key value pairs used for configs rather
> than embedding a new formatting. So two strings rather than one with an
> internal equals sign.
>
> A: Fixed in the latest patch - normalized configs and changed protocol
> accordingly.
>
> 7. Is the postcondition of these APIs that the command has begun or that
> the command has been completed? It is a lot more usable if the command has
> been completed so you know that if you create a topic and then publish to
> it you won't get an exception about there being no such topic.
>
> A: For long running requests (like reassign partitions) - the post
> condition is
> command has begun - so we don't block the client. In case of your example -
> topic commands, this will be refactored and topic commands will be executed
> immediately, since the Controller will serve Admin requests
> (follow-up ticket KAFKA-1777).
>
> 8. Describe topic and list topics duplicate a lot of stuff in the metadata
> request. Is there a reason to give back topics marked for deletion? I feel
> like if we just make the post-condition of the delete command be that the
> topic is deleted that will get rid of the need for this right? And it will
> be much more intuitive.
>
> A: Fixed in the latest patch - removed topics marked for deletion in
> ListTopicsRequest.
>
> 9. Should we consider batching these requests? We have generally tried to
> allow multiple operations to be batched. My suspicion is that without this
> we will get a lot of code that does something like
>    for(topic: adminClient.listTopics())
>       adminClient.describeTopic(topic)
> this code will work great when you test on 5 topics but not do as well if
> you have 50k.
>
> A: Updated the KIP - please check "Topic Admin Schema" section.
>
> 10. I think we should also discuss how we want to expose a programmatic JVM
> client api for these operations. Currently people rely on AdminUtils which
> is totally sketchy. I think we probably need another client under clients/
> that exposes administrative functionality. We will need this just to
> properly test the new apis, I suspect. We should figure out that API.
>
> A: Updated the KIP - please check "Admin Client" section with an initial
> API proposal.
>
> 11. The other information that would be really useful to get would be
> information about partitions--how much data is in the partition, what are
> the segment offsets, what is the log-end offset (i.e. last offset), what is
> the compaction point, etc. I think that done right this would be the
> successor to the very awkward OffsetRequest we have today.
>
> A: I removed ConsumerGroupOffsetsRequest in the latest patch. I believe
> this should
> be resolved in a separate KIP / jira ticket.
>
> 12. Generally we can do good error handling without needing custom
> server-side
> messages. I.e. generally the client has the context to know that if it got
> an error that the topic doesn't exist to say "Topic X doesn't exist" rather
> than "error code 14" (or whatever). Maybe there are specific cases where
> this is hard? If we want to add server-side error messages we really do
> need to do this in a consistent way across the protocol.
>
> A: Updated the KIP - please check "Protocol Errors" section. I added the
> comprehensive, fine-grained list of error codes.
>
> Comments from Guozhang:
> 13. Describe topic request: it would be great to go beyond just batching on
> topic name regex for this request. For example, a very common use case of
> the topic command is to list all topics whose config A's value is B. With
> topic name regex then we have to first retrieve __all__ topics's
> description info and then filter at the client end, which will be a huge
> burden on ZK.
> AND
> 14. Config K-Vs in create topic: this is related to the previous point;
> maybe we can add another metadata K-V or just a metadata string along side
> with config K-V in create topic like we did for offset commit request. This
> field can be quite useful in storing information like "owner" of the topic
> who issue the create command, etc, which is quite important for a
> multi-tenant setting. Then in the describe topic request we can also batch
> on regex of the metadata field.
>
> A: As discussed it is very interesting but can be implemented later after
> we have some basic functionality there.
>
> 15. Today all the admin operations are async in the sense that command will
> return once it is written in ZK, and that is why we need extra verification
> like testUtil.waitForTopicCreated() / verify partition reassignment
> request, etc. With admin requests we could add a flag to enable / disable
> synchronous requests; when it is turned on, the response will not return
> until the request has been completed. And for async requests we can add a
> "token" field in the response, and then only need a general "admin
> verification request" with the given token to check if the async request
> has been completed.
>
> A: I see your point. My idea was to provide specific Verify...Request per
> each
> long running request, where needed. We can do it the way you suggest. The
> only
> concern is that introducing a token we again will make schema "dynamic". We
> wanted
> to do similar thing introducing single AdminRequest for all topic commands
> but rejected
> this idea because we wanted to have schema defined. So this is more a
> choice between:
> a) have fixed schema but introduce each time new Verify...Request for
> long-running requests
> b) use one request for verification but generalize it with token
> I'm fine with whatever decision community come to. Just let me know your
> thoughts.
>
> Comment from Gwen:
> 16. Specifically for ownership, I think the plan is to add ACL (it sounds
> like you are describing ACL) via an external system (Argus, Sentry).
> I remember KIP-11 described this, but I can't find the KIP any longer.
>
> A: Okay, no problem. Not sure though how we are going to handle it. Wait
> which KIP
> will be committed first and include changes to TopicMetadata from the later
> one?
> Anyway, I added this note to "Open Questions" section so we don't miss this
> piece.
>
> Thanks,
> Andrii Biletskyi
>
> On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Hi all,
> >
> > Today I uploaded the patch that covers some of the discussed and agreed
> > items:
> > - removed MaybeOf optional type
> > - switched to java protocol definitions
> > - simplified messages (normalized configs, removed topic marked for
> > deletion)
> >
> > I also updated the KIP-4 with respective changes and wrote down my
> > proposal for
> > pending items:
> > - Batch Admin Operations -> updated Wire Protocol schema proposal
> > - Remove ClusterMetadata -> changed to extend TopicMetadataRequest
> > - Admin Client -> updated my initial proposal to reflect batching
> > - Error codes -> proposed fine-grained error code instead of
> > AdminRequestFailed
> >
> > I will also send a separate email to cover all comments from this thread.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> >
> > On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <gs...@cloudera.com>
> > wrote:
> >
> >> Found KIP-11 (
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> >> )
> >> It actually specifies changes to the Metadata protocol, so making sure
> >> both KIPs are consistent in this regard will be good.
> >>
> >> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <gs...@cloudera.com>
> >> wrote:
> >> > Specifically for ownership, I think the plan is to add ACL (it sounds
> >> > like you are describing ACL) via an external system (Argus, Sentry).
> >> > I remember KIP-11 described this, but I can't find the KIP any longer.
> >> >
> >> > Regardless, I think KIP-4 focuses on getting information that already
> >> > exists from Kafka brokers, not on adding information that perhaps
> >> > should exist but doesn't yet?
> >> >
> >> > Gwen
> >> >
> >> >
> >> >
> >> >
> >> >
> >> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >> >> Folks,
> >> >>
> >> >> Just want to elaborate a bit more on the create-topic metadata and
> >> batching
> >> >> describe-topic based on config / metadata in my previous email as we
> >> work
> >> >> on KAFKA-1694. The main motivation is to have some sort of topic
> >> management
> >> >> mechanisms, which I think is quite important in a multi-tenant /
> cloud
> >> >> architecture: today anyone can create topics in a shared Kafka
> >> cluster, but
> >> >> there is no concept or "ownership" of topics that are created by
> >> different
> >> >> users. For example, at LinkedIn we basically distinguish topic owners
> >> via
> >> >> some casual topic name prefix, which is a bit awkward and does not
> fly
> >> as
> >> >> we scale our customers. It would be great to use describe-topics such
> >> as:
> >> >>
> >> >> Describe all topics that is created by me.
> >> >>
> >> >> Describe all topics whose retention time is overriden to X.
> >> >>
> >> >> Describe all topics whose writable group include user Y (this is
> >> related to
> >> >> authorization), etc..
> >> >>
> >> >> One possible way to achieve this is to add a metadata file in the
> >> >> create-topic request, whose value will also be written ZK as we
> create
> >> the
> >> >> topic; then describe-topics can choose to batch topics based on 1)
> name
> >> >> regex, 2) config K-V matching, 3) metadata regex, etc.
> >> >>
> >> >> Thoughts?
> >> >>
> >> >> Guozhang
> >> >>
> >> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com>
> >> wrote:
> >> >>
> >> >>> Thanks for the updated wiki. A few comments below:
> >> >>>
> >> >>> 1. Error description in response: I think if some errorCode could
> >> indicate
> >> >>> several different error cases then we should really change it to
> >> multiple
> >> >>> codes. In general the errorCode itself would be precise and
> >> sufficient for
> >> >>> describing the server side errors.
> >> >>>
> >> >>> 2. Describe topic request: it would be great to go beyond just
> >> batching on
> >> >>> topic name regex for this request. For example, a very common use
> >> case of
> >> >>> the topic command is to list all topics whose config A's value is B.
> >> With
> >> >>> topic name regex then we have to first retrieve __all__ topics's
> >> >>> description info and then filter at the client end, which will be a
> >> huge
> >> >>> burden on ZK.
> >> >>>
> >> >>> 3. Config K-Vs in create topic: this is related to the previous
> point;
> >> >>> maybe we can add another metadata K-V or just a metadata string
> along
> >> side
> >> >>> with config K-V in create topic like we did for offset commit
> >> request. This
> >> >>> field can be quite useful in storing information like "owner" of the
> >> topic
> >> >>> who issue the create command, etc, which is quite important for a
> >> >>> multi-tenant setting. Then in the describe topic request we can also
> >> batch
> >> >>> on regex of the metadata field.
> >> >>>
> >> >>> 4. Today all the admin operations are async in the sense that
> command
> >> will
> >> >>> return once it is written in ZK, and that is why we need extra
> >> verification
> >> >>> like testUtil.waitForTopicCreated() / verify partition reassignment
> >> >>> request, etc. With admin requests we could add a flag to enable /
> >> disable
> >> >>> synchronous requests; when it is turned on, the response will not
> >> return
> >> >>> until the request has been completed. And for async requests we can
> >> add a
> >> >>> "token" field in the response, and then only need a general "admin
> >> >>> verification request" with the given token to check if the async
> >> request
> >> >>> has been completed.
> >> >>>
> >> >>> 5. +1 for extending Metadata request to include controller /
> >> coordinator
> >> >>> information, and then we can remove the ConsumerMetadata /
> >> ClusterMetadata
> >> >>> requests.
> >> >>>
> >> >>> Guozhang
> >> >>>
> >> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com>
> >> wrote:
> >> >>>
> >> >>>> Thanks for sending that out Joe - I don't think I will be able to
> >> make
> >> >>>> it today, so if notes can be sent out afterward that would be
> great.
> >> >>>>
> >> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> >> >>>> > Thanks for sending this out Joe. Looking forward to chatting with
> >> >>>> everyone :)
> >> >>>> >
> >> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly>
> >> wrote:
> >> >>>> > > Hey, I just sent out a google hangout invite to all pmc,
> >> committers
> >> >>>> and
> >> >>>> > > everyone I found working on a KIP. If I missed anyone in the
> >> invite
> >> >>>> please
> >> >>>> > > let me know and can update it, np.
> >> >>>> > >
> >> >>>> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we
> can
> >> get
> >> >>>> INFRA
> >> >>>> > > help to make a google account so we can manage better?
> >> >>>> > >
> >> >>>> > > To discuss
> >> >>>> > >
> >> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >> >>>> > > in progress and related JIRA that are interdependent and common
> >> work.
> >> >>>> > >
> >> >>>> > > ~ Joe Stein
> >> >>>> > >
> >> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <
> jay.kreps@gmail.com>
> >> >>>> wrote:
> >> >>>> > >
> >> >>>> > >> Let's stay on Google hangouts that will also record and make
> the
> >> >>>> sessions
> >> >>>> > >> available on youtube.
> >> >>>> > >>
> >> >>>> > >> -Jay
> >> >>>> > >>
> >> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> >> >>>> jholoman@cloudera.com>
> >> >>>> > >> wrote:
> >> >>>> > >>
> >> >>>> > >> > Jay / Joe
> >> >>>> > >> >
> >> >>>> > >> > We're happy to send out a Webex for this purpose. We could
> >> record
> >> >>>> the
> >> >>>> > >> > sessions if there is interest and publish them out.
> >> >>>> > >> >
> >> >>>> > >> > Thanks
> >> >>>> > >> >
> >> >>>> > >> > Jeff
> >> >>>> > >> >
> >> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> >> jay.kreps@gmail.com>
> >> >>>> wrote:
> >> >>>> > >> >
> >> >>>> > >> > > Let's try to get the technical hang-ups sorted out,
> though.
> >> I
> >> >>>> really
> >> >>>> > >> > think
> >> >>>> > >> > > there is some benefit to live discussion vs writing. I am
> >> >>>> hopeful that
> >> >>>> > >> if
> >> >>>> > >> > > we post instructions and give ourselves a few attempts we
> >> can
> >> >>>> get it
> >> >>>> > >> > > working.
> >> >>>> > >> > >
> >> >>>> > >> > > Tuesday at that time would work for me...any objections?
> >> >>>> > >> > >
> >> >>>> > >> > > -Jay
> >> >>>> > >> > >
> >> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> >> joe.stein@stealth.ly
> >> >>>> >
> >> >>>> > >> wrote:
> >> >>>> > >> > >
> >> >>>> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET
> /
> >> 10am
> >> >>>> PT
> >> >>>> > >> ????
> >> >>>> > >> > > >
> >> >>>> > >> > > > I don't mind google hangout but there is always some
> >> issue or
> >> >>>> > >> whatever
> >> >>>> > >> > so
> >> >>>> > >> > > > we know the apache irc channel works. We can start there
> >> and
> >> >>>> see how
> >> >>>> > >> it
> >> >>>> > >> > > > goes? We can pull transcripts too and associate to
> >> tickets if
> >> >>>> need be
> >> >>>> > >> > > makes
> >> >>>> > >> > > > it helpful for things.
> >> >>>> > >> > > >
> >> >>>> > >> > > > ~ Joestein
> >> >>>> > >> > > >
> >> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> >> >>>> jay.kreps@gmail.com>
> >> >>>> > >> > wrote:
> >> >>>> > >> > > >
> >> >>>> > >> > > > > We'd talked about doing a Google Hangout to chat about
> >> this.
> >> >>>> What
> >> >>>> > >> > about
> >> >>>> > >> > > > > generalizing that a little further...I actually think
> it
> >> >>>> would be
> >> >>>> > >> > good
> >> >>>> > >> > > > for
> >> >>>> > >> > > > > everyone spending a reasonable chunk of their week on
> >> Kafka
> >> >>>> stuff
> >> >>>> > >> to
> >> >>>> > >> > > > maybe
> >> >>>> > >> > > > > sync up once a week. I think we could use time to talk
> >> >>>> through
> >> >>>> > >> design
> >> >>>> > >> > > > > stuff, make sure we are on top of code reviews, talk
> >> through
> >> >>>> any
> >> >>>> > >> > tricky
> >> >>>> > >> > > > > issues, etc.
> >> >>>> > >> > > > >
> >> >>>> > >> > > > > We can make it publicly available so that any one can
> >> follow
> >> >>>> along
> >> >>>> > >> > who
> >> >>>> > >> > > > > likes.
> >> >>>> > >> > > > >
> >> >>>> > >> > > > > Any interest in doing this? If so I'll try to set it
> up
> >> >>>> starting
> >> >>>> > >> next
> >> >>>> > >> > > > week.
> >> >>>> > >> > > > >
> >> >>>> > >> > > > > -Jay
> >> >>>> > >> > > > >
> >> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> >> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> >> >>>> > >> > > > >
> >> >>>> > >> > > > > > Hi all,
> >> >>>> > >> > > > > >
> >> >>>> > >> > > > > > I've updated KIP page, fixed / aligned document
> >> structure.
> >> >>>> Also I
> >> >>>> > >> > > added
> >> >>>> > >> > > > > > some
> >> >>>> > >> > > > > > very initial proposal for AdminClient so we have
> >> something
> >> >>>> to
> >> >>>> > >> start
> >> >>>> > >> > > > from
> >> >>>> > >> > > > > > while
> >> >>>> > >> > > > > > discussing the KIP.
> >> >>>> > >> > > > > >
> >> >>>> > >> > > > > >
> >> >>>> > >> > > > > >
> >> >>>> > >> > > > >
> >> >>>> > >> > > >
> >> >>>> > >> > >
> >> >>>> > >> >
> >> >>>> > >>
> >> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> >>>> > >> > > > > >
> >> >>>> > >> > > > > > Thanks,
> >> >>>> > >> > > > > > Andrii Biletskyi
> >> >>>> > >> > > > > >
> >> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> >> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> >>>> > >> > > > > >
> >> >>>> > >> > > > > > > Jay,
> >> >>>> > >> > > > > > >
> >> >>>> > >> > > > > > > Re error messages: you are right, in most cases
> >> client
> >> >>>> will
> >> >>>> > >> have
> >> >>>> > >> > > > enough
> >> >>>> > >> > > > > > > context to show descriptive error message. My
> >> concern is
> >> >>>> that
> >> >>>> > >> we
> >> >>>> > >> > > will
> >> >>>> > >> > > > > > have
> >> >>>> > >> > > > > > > to
> >> >>>> > >> > > > > > > add lots of new error codes for each possible
> >> error. Of
> >> >>>> course,
> >> >>>> > >> > we
> >> >>>> > >> > > > > could
> >> >>>> > >> > > > > > > reuse
> >> >>>> > >> > > > > > > some of existing like UknownTopicOrPartitionCode,
> >> but we
> >> >>>> will
> >> >>>> > >> > also
> >> >>>> > >> > > > need
> >> >>>> > >> > > > > > to
> >> >>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> >> >>>> TopicConfigInvalid (both
> >> >>>> > >> > for
> >> >>>> > >> > > > > topic
> >> >>>> > >> > > > > > > name and config, and probably user would like to
> >> know
> >> >>>> what
> >> >>>> > >> > exactly
> >> >>>> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
> >> >>>> > >> InternalError
> >> >>>> > >> > > > (e.g.
> >> >>>> > >> > > > > > > zookeeper failure) etc.
> >> >>>> > >> > > > > > > And this is only for TopicCommand, we will also
> >> need to
> >> >>>> add
> >> >>>> > >> > similar
> >> >>>> > >> > > > > stuff
> >> >>>> > >> > > > > > > for
> >> >>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end
> >> up
> >> >>>> with a
> >> >>>> > >> > large
> >> >>>> > >> > > > list
> >> >>>> > >> > > > > > of
> >> >>>> > >> > > > > > > error codes, used only in Admin protocol.
> >> >>>> > >> > > > > > > Having said that, I agree my proposal is not
> >> consistent
> >> >>>> with
> >> >>>> > >> > other
> >> >>>> > >> > > > > cases.
> >> >>>> > >> > > > > > > Maybe we can find better solution or something
> >> >>>> in-between.
> >> >>>> > >> > > > > > >
> >> >>>> > >> > > > > > > Re Hangout chat: I think it is a great idea. This
> >> way we
> >> >>>> can
> >> >>>> > >> move
> >> >>>> > >> > > on
> >> >>>> > >> > > > > > > faster.
> >> >>>> > >> > > > > > > Let's agree somehow on date/time so people can
> join.
> >> >>>> Will work
> >> >>>> > >> > for
> >> >>>> > >> > > me
> >> >>>> > >> > > > > > this
> >> >>>> > >> > > > > > > and
> >> >>>> > >> > > > > > > next week almost anytime if agreed in advance.
> >> >>>> > >> > > > > > >
> >> >>>> > >> > > > > > > Thanks,
> >> >>>> > >> > > > > > > Andrii
> >> >>>> > >> > > > > > >
> >> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> >> >>>> > >> jay.kreps@gmail.com>
> >> >>>> > >> > > > > wrote:
> >> >>>> > >> > > > > > >
> >> >>>> > >> > > > > > >> Hey Andrii,
> >> >>>> > >> > > > > > >>
> >> >>>> > >> > > > > > >> Generally we can do good error handling without
> >> needing
> >> >>>> custom
> >> >>>> > >> > > > > > server-side
> >> >>>> > >> > > > > > >> messages. I.e. generally the client has the
> >> context to
> >> >>>> know
> >> >>>> > >> that
> >> >>>> > >> > > if
> >> >>>> > >> > > > it
> >> >>>> > >> > > > > > got
> >> >>>> > >> > > > > > >> an error that the topic doesn't exist to say
> >> "Topic X
> >> >>>> doesn't
> >> >>>> > >> > > exist"
> >> >>>> > >> > > > > > >> rather
> >> >>>> > >> > > > > > >> than "error code 14" (or whatever). Maybe there
> are
> >> >>>> specific
> >> >>>> > >> > cases
> >> >>>> > >> > > > > where
> >> >>>> > >> > > > > > >> this is hard? If we want to add server-side error
> >> >>>> messages we
> >> >>>> > >> > > really
> >> >>>> > >> > > > > do
> >> >>>> > >> > > > > > >> need to do this in a consistent way across the
> >> protocol.
> >> >>>> > >> > > > > > >>
> >> >>>> > >> > > > > > >> I still have a bunch of open questions here from
> my
> >> >>>> previous
> >> >>>> > >> > > list. I
> >> >>>> > >> > > > > > will
> >> >>>> > >> > > > > > >> be out for the next few days for Strata though.
> >> Maybe
> >> >>>> we could
> >> >>>> > >> > do
> >> >>>> > >> > > a
> >> >>>> > >> > > > > > Google
> >> >>>> > >> > > > > > >> Hangout chat on any open issues some time towards
> >> the
> >> >>>> end of
> >> >>>> > >> > next
> >> >>>> > >> > > > week
> >> >>>> > >> > > > > > for
> >> >>>> > >> > > > > > >> anyone interested in this ticket? I have a
> feeling
> >> that
> >> >>>> might
> >> >>>> > >> > > > progress
> >> >>>> > >> > > > > > >> things a little faster than email--I think we
> >> could talk
> >> >>>> > >> through
> >> >>>> > >> > > > those
> >> >>>> > >> > > > > > >> issues I brought up fairly quickly...
> >> >>>> > >> > > > > > >>
> >> >>>> > >> > > > > > >> -Jay
> >> >>>> > >> > > > > > >>
> >> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii
> Biletskyi <
> >> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> >> >>>> > >> > > > > > >>
> >> >>>> > >> > > > > > >> > Hi all,
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >> > I'm trying to address some of the issues which
> >> were
> >> >>>> > >> mentioned
> >> >>>> > >> > > > > earlier
> >> >>>> > >> > > > > > >> about
> >> >>>> > >> > > > > > >> > Admin RQ/RP format. One of those was about
> >> batching
> >> >>>> > >> > operations.
> >> >>>> > >> > > > What
> >> >>>> > >> > > > > > if
> >> >>>> > >> > > > > > >> we
> >> >>>> > >> > > > > > >> > follow TopicCommand approach and let people
> >> specify
> >> >>>> > >> topic-name
> >> >>>> > >> > > by
> >> >>>> > >> > > > > > >> regexp -
> >> >>>> > >> > > > > > >> > would that cover most of the use cases?
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >> > Secondly, is what information should we
> generally
> >> >>>> provide in
> >> >>>> > >> > > Admin
> >> >>>> > >> > > > > > >> > responses.
> >> >>>> > >> > > > > > >> > I realize that Admin commands don't imply they
> >> will
> >> >>>> be used
> >> >>>> > >> > only
> >> >>>> > >> > > > in
> >> >>>> > >> > > > > > CLI
> >> >>>> > >> > > > > > >> > but,
> >> >>>> > >> > > > > > >> > it seems to me, CLI is a very important client
> >> of this
> >> >>>> > >> > feature.
> >> >>>> > >> > > In
> >> >>>> > >> > > > > > this
> >> >>>> > >> > > > > > >> > case,
> >> >>>> > >> > > > > > >> > seems logical, we would like to provide users
> >> with
> >> >>>> rich
> >> >>>> > >> > > experience
> >> >>>> > >> > > > > in
> >> >>>> > >> > > > > > >> terms
> >> >>>> > >> > > > > > >> > of
> >> >>>> > >> > > > > > >> > getting results / errors of the executed
> >> commands.
> >> >>>> Usually
> >> >>>> > >> we
> >> >>>> > >> > > > supply
> >> >>>> > >> > > > > > >> with
> >> >>>> > >> > > > > > >> > responses only errorCode, which looks very
> >> limiting,
> >> >>>> in case
> >> >>>> > >> > of
> >> >>>> > >> > > > CLI
> >> >>>> > >> > > > > we
> >> >>>> > >> > > > > > >> may
> >> >>>> > >> > > > > > >> > want to print human readable error description.
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >> > So, taking into account previous item about
> >> batching,
> >> >>>> what
> >> >>>> > >> do
> >> >>>> > >> > > you
> >> >>>> > >> > > > > > think
> >> >>>> > >> > > > > > >> > about
> >> >>>> > >> > > > > > >> > having smth like:
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >> > ('create' doesn't support regexp)
> >> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions
> >> Replicas
> >> >>>> > >> > > > > ReplicaAssignment
> >> >>>> > >> > > > > > >> > [Config]
> >> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode
> ErrorDescription
> >> >>>> > >> > > > > > >> >   ErrorCode => int16
> >> >>>> > >> > > > > > >> >   ErrorDescription => string (empty if
> >> successful)
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> >> >>>> > >> > > ReplicaAssignment
> >> >>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> >> >>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
> >> >>>> ErrorDescription]
> >> >>>> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
> >> >>>> > >> > > > > > >> >   CommandErrorCode => int16
> >> >>>> > >> > > > > > >> >   CommandErrorDescription => string (nonempty
> in
> >> case
> >> >>>> of
> >> >>>> > >> fatal
> >> >>>> > >> > > > > error,
> >> >>>> > >> > > > > > >> e.g.
> >> >>>> > >> > > > > > >> > we couldn't get topics by regexp)
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> >> >>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> >> TopicDescription
> >> >>>> > >> ErrorCode
> >> >>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> >> >>>> CommandErrorDescription
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >> > Also, any thoughts about our discussion
> regarding
> >> >>>> re-routing
> >> >>>> > >> > > > > facility?
> >> >>>> > >> > > > > > >> In
> >> >>>> > >> > > > > > >> > my
> >> >>>> > >> > > > > > >> > understanding, it is like between augmenting
> >> >>>> > >> > > TopicMetadataRequest
> >> >>>> > >> > > > > > >> > (to include at least controllerId) and
> >> implementing
> >> >>>> new
> >> >>>> > >> > generic
> >> >>>> > >> > > > > > >> re-routing
> >> >>>> > >> > > > > > >> > facility so sending messages to controller will
> >> be
> >> >>>> handled
> >> >>>> > >> by
> >> >>>> > >> > > it.
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >> > Thanks,
> >> >>>> > >> > > > > > >> > Andrii Biletskyi
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii
> >> Biletskyi <
> >> >>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >> > > @Guozhang:
> >> >>>> > >> > > > > > >> > > Thanks for your comments, I've answered some
> of
> >> >>>> those. The
> >> >>>> > >> > > main
> >> >>>> > >> > > > > > thing
> >> >>>> > >> > > > > > >> is
> >> >>>> > >> > > > > > >> > > having merged request for
> >> >>>> create-alter-delete-describe - I
> >> >>>> > >> > > have
> >> >>>> > >> > > > > some
> >> >>>> > >> > > > > > >> > > concerns about this approach.
> >> >>>> > >> > > > > > >> > >
> >> >>>> > >> > > > > > >> > > @*Jay*:
> >> >>>> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is
> >> also
> >> >>>> one of
> >> >>>> > >> > the
> >> >>>> > >> > > > > > >> concerns.
> >> >>>> > >> > > > > > >> > We
> >> >>>> > >> > > > > > >> > > can solve it if we implement re-routing
> >> facility.
> >> >>>> But I
> >> >>>> > >> > agree
> >> >>>> > >> > > > with
> >> >>>> > >> > > > > > >> > > Guozhang - it will make clients' internals a
> >> little
> >> >>>> bit
> >> >>>> > >> > easier
> >> >>>> > >> > > > but
> >> >>>> > >> > > > > > >> this
> >> >>>> > >> > > > > > >> > > seems to be a complex logic to implement and
> >> >>>> support then.
> >> >>>> > >> > > > > > Especially
> >> >>>> > >> > > > > > >> for
> >> >>>> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing
> >> later
> >> >>>> for
> >> >>>> > >> these
> >> >>>> > >> > > > > > >> requests).
> >> >>>> > >> > > > > > >> > > Also people will tend to avoid this
> re-routing
> >> >>>> facility
> >> >>>> > >> and
> >> >>>> > >> > > hold
> >> >>>> > >> > > > > > local
> >> >>>> > >> > > > > > >> > > cluster cache to ensure their high-priority
> >> requests
> >> >>>> > >> (which
> >> >>>> > >> > > some
> >> >>>> > >> > > > > of
> >> >>>> > >> > > > > > >> the
> >> >>>> > >> > > > > > >> > > admin requests are) not sent to some busy
> >> broker
> >> >>>> where
> >> >>>> > >> they
> >> >>>> > >> > > wait
> >> >>>> > >> > > > > to
> >> >>>> > >> > > > > > be
> >> >>>> > >> > > > > > >> > > routed to the correct one.
> >> >>>> > >> > > > > > >> > > As pointed out by Jun here (
> >> >>>> > >> > > > > > >> > >
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >>
> >> >>>> > >> > > > > >
> >> >>>> > >> > > > >
> >> >>>> > >> > > >
> >> >>>> > >> > >
> >> >>>> > >> >
> >> >>>> > >>
> >> >>>>
> >>
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> >> >>>> > >> > > > > > >> > )
> >> >>>> > >> > > > > > >> > > to solve the issue we might introduce a
> message
> >> >>>> type to
> >> >>>> > >> get
> >> >>>> > >> > > > > cluster
> >> >>>> > >> > > > > > >> > state.
> >> >>>> > >> > > > > > >> > > But I agree we can just update
> >> >>>> TopicMetadataResponse to
> >> >>>> > >> > > include
> >> >>>> > >> > > > > > >> > > controllerId (and probably smth else).
> >> >>>> > >> > > > > > >> > > What are you thougths?
> >> >>>> > >> > > > > > >> > >
> >> >>>> > >> > > > > > >> > > Thanks,
> >> >>>> > >> > > > > > >> > > Andrii
> >> >>>> > >> > > > > > >> > >
> >> >>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang
> Wang
> >> <
> >> >>>> > >> > > > > wangguoz@gmail.com>
> >> >>>> > >> > > > > > >> > wrote:
> >> >>>> > >> > > > > > >> > >
> >> >>>> > >> > > > > > >> > >> I think for the topics commands we can
> >> actually
> >> >>>> merge
> >> >>>> > >> > > > > > >> > >> create/alter/delete/describe as one request
> >> type
> >> >>>> since
> >> >>>> > >> > their
> >> >>>> > >> > > > > > formats
> >> >>>> > >> > > > > > >> are
> >> >>>> > >> > > > > > >> > >> very much similar, and keep list-topics and
> >> others
> >> >>>> like
> >> >>>> > >> > > > > > >> > >> partition-reassignment /
> >> preferred-leader-election
> >> >>>> as
> >> >>>> > >> > > separate
> >> >>>> > >> > > > > > >> request
> >> >>>> > >> > > > > > >> > >> types, I also left some other comments on
> the
> >> RB (
> >> >>>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> >> >>>> > >> > > > > > >> > >>
> >> >>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> >> >>>> > >> > > > jay.kreps@gmail.com>
> >> >>>> > >> > > > > > >> wrote:
> >> >>>> > >> > > > > > >> > >>
> >> >>>> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to
> >> just
> >> >>>> have
> >> >>>> > >> one
> >> >>>> > >> > > "do
> >> >>>> > >> > > > > > admin
> >> >>>> > >> > > > > > >> > >> stuff"
> >> >>>> > >> > > > > > >> > >> > command that has the union of all
> >> parameters.
> >> >>>> > >> > > > > > >> > >> >
> >> >>>> > >> > > > > > >> > >> > What I am saying is that command line
> tools
> >> are
> >> >>>> one
> >> >>>> > >> > client
> >> >>>> > >> > > of
> >> >>>> > >> > > > > the
> >> >>>> > >> > > > > > >> > >> > administrative apis, but these will be
> used
> >> in a
> >> >>>> number
> >> >>>> > >> > of
> >> >>>> > >> > > > > > >> scenarios
> >> >>>> > >> > > > > > >> > so
> >> >>>> > >> > > > > > >> > >> > they should make logical sense even in the
> >> >>>> absence of
> >> >>>> > >> the
> >> >>>> > >> > > > > command
> >> >>>> > >> > > > > > >> line
> >> >>>> > >> > > > > > >> > >> > tool. Hence comments like trying to
> clarify
> >> the
> >> >>>> > >> > > relationship
> >> >>>> > >> > > > > > >> between
> >> >>>> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these
> >> kinds
> >> >>>> of
> >> >>>> > >> things
> >> >>>> > >> > > > > really
> >> >>>> > >> > > > > > >> need
> >> >>>> > >> > > > > > >> > >> to be
> >> >>>> > >> > > > > > >> > >> > thought through.
> >> >>>> > >> > > > > > >> > >> >
> >> >>>> > >> > > > > > >> > >> > Hope that makes sense.
> >> >>>> > >> > > > > > >> > >> >
> >> >>>> > >> > > > > > >> > >> > -Jay
> >> >>>> > >> > > > > > >> > >> >
> >> >>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
> >> >>>> Biletskyi <
> >> >>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> >> >>>> > >> > > > > > >> > >> >
> >> >>>> > >> > > > > > >> > >> > > Jay,
> >> >>>> > >> > > > > > >> > >> > >
> >> >>>> > >> > > > > > >> > >> > > Thanks for answering. You understood
> >> >>>> correctly, most
> >> >>>> > >> of
> >> >>>> > >> > > my
> >> >>>> > >> > > > > > >> comments
> >> >>>> > >> > > > > > >> > >> were
> >> >>>> > >> > > > > > >> > >> > > related to your point 1) - about "well
> >> >>>> thought-out"
> >> >>>> > >> > apis.
> >> >>>> > >> > > > > Also,
> >> >>>> > >> > > > > > >> yes,
> >> >>>> > >> > > > > > >> > >> as I
> >> >>>> > >> > > > > > >> > >> > > understood we would like to introduce a
> >> single
> >> >>>> > >> unified
> >> >>>> > >> > > CLI
> >> >>>> > >> > > > > tool
> >> >>>> > >> > > > > > >> with
> >> >>>> > >> > > > > > >> > >> > > centralized server-side request handling
> >> for
> >> >>>> lots of
> >> >>>> > >> > > > existing
> >> >>>> > >> > > > > > >> ones
> >> >>>> > >> > > > > > >> > >> (incl.
> >> >>>> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> >> >>>> > >> ReassignPartitions,
> >> >>>> > >> > > smth
> >> >>>> > >> > > > > > else
> >> >>>> > >> > > > > > >> if
> >> >>>> > >> > > > > > >> > >> added
> >> >>>> > >> > > > > > >> > >> > > in future). In our previous discussion (
> >> >>>> > >> > > > > > >> > >> > >
> >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694)
> >> >>>> > >> > people
> >> >>>> > >> > > > > said
> >> >>>> > >> > > > > > >> > they'd
> >> >>>> > >> > > > > > >> > >> > > rather
> >> >>>> > >> > > > > > >> > >> > > have a separate message for each
> command,
> >> so,
> >> >>>> yes,
> >> >>>> > >> this
> >> >>>> > >> > > > way I
> >> >>>> > >> > > > > > >> came
> >> >>>> > >> > > > > > >> > to
> >> >>>> > >> > > > > > >> > >> 1-1
> >> >>>> > >> > > > > > >> > >> > > mapping between commands in the tool and
> >> >>>> protocol
> >> >>>> > >> > > > additions.
> >> >>>> > >> > > > > > But
> >> >>>> > >> > > > > > >> I
> >> >>>> > >> > > > > > >> > >> might
> >> >>>> > >> > > > > > >> > >> > be
> >> >>>> > >> > > > > > >> > >> > > wrong.
> >> >>>> > >> > > > > > >> > >> > > At the end I just try to start
> discussion
> >> how
> >> >>>> at
> >> >>>> > >> least
> >> >>>> > >> > > > > > generally
> >> >>>> > >> > > > > > >> > this
> >> >>>> > >> > > > > > >> > >> > > protocol should look like.
> >> >>>> > >> > > > > > >> > >> > >
> >> >>>> > >> > > > > > >> > >> > > Thanks,
> >> >>>> > >> > > > > > >> > >> > > Andrii
> >> >>>> > >> > > > > > >> > >> > >
> >> >>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay
> >> Kreps <
> >> >>>> > >> > > > > > jay.kreps@gmail.com
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >> > >> wrote:
> >> >>>> > >> > > > > > >> > >> > >
> >> >>>> > >> > > > > > >> > >> > > > Hey Andrii,
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> > >> > > > > > >> > >> > > > To answer your earlier question we
> just
> >> >>>> really
> >> >>>> > >> can't
> >> >>>> > >> > be
> >> >>>> > >> > > > > > adding
> >> >>>> > >> > > > > > >> any
> >> >>>> > >> > > > > > >> > >> more
> >> >>>> > >> > > > > > >> > >> > > > scala protocol objects. These things
> are
> >> >>>> super hard
> >> >>>> > >> > to
> >> >>>> > >> > > > > > maintain
> >> >>>> > >> > > > > > >> > >> because
> >> >>>> > >> > > > > > >> > >> > > > they hand code the byte parsing and
> >> don't
> >> >>>> have good
> >> >>>> > >> > > > > > versioning
> >> >>>> > >> > > > > > >> > >> support.
> >> >>>> > >> > > > > > >> > >> > > > Since we are already planning on
> >> converting
> >> >>>> we
> >> >>>> > >> > > definitely
> >> >>>> > >> > > > > > don't
> >> >>>> > >> > > > > > >> > >> want to
> >> >>>> > >> > > > > > >> > >> > > add
> >> >>>> > >> > > > > > >> > >> > > > a ton more of these--they are total
> tech
> >> >>>> debt.
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> > >> > > > > > >> > >> > > > What does it mean that the changes are
> >> >>>> isolated
> >> >>>> > >> from
> >> >>>> > >> > > the
> >> >>>> > >> > > > > > >> current
> >> >>>> > >> > > > > > >> > >> code
> >> >>>> > >> > > > > > >> > >> > > base?
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> > >> > > > > > >> > >> > > > I actually didn't understand the
> >> remaining
> >> >>>> > >> comments,
> >> >>>> > >> > > > which
> >> >>>> > >> > > > > of
> >> >>>> > >> > > > > > >> the
> >> >>>> > >> > > > > > >> > >> > points
> >> >>>> > >> > > > > > >> > >> > > > are you responding to?
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> > >> > > > > > >> > >> > > > Maybe one sticking point here is that
> it
> >> >>>> seems like
> >> >>>> > >> > you
> >> >>>> > >> > > > > want
> >> >>>> > >> > > > > > to
> >> >>>> > >> > > > > > >> > make
> >> >>>> > >> > > > > > >> > >> > some
> >> >>>> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1
> >> mapping
> >> >>>> > >> between
> >> >>>> > >> > > > > > commands
> >> >>>> > >> > > > > > >> you
> >> >>>> > >> > > > > > >> > >> > > imagine
> >> >>>> > >> > > > > > >> > >> > > > in the tool and protocol additions. I
> >> want
> >> >>>> to make
> >> >>>> > >> > sure
> >> >>>> > >> > > > we
> >> >>>> > >> > > > > > >> don't
> >> >>>> > >> > > > > > >> > do
> >> >>>> > >> > > > > > >> > >> > that.
> >> >>>> > >> > > > > > >> > >> > > > The protocol needs to be really really
> >> well
> >> >>>> thought
> >> >>>> > >> > out
> >> >>>> > >> > > > > > against
> >> >>>> > >> > > > > > >> > many
> >> >>>> > >> > > > > > >> > >> > use
> >> >>>> > >> > > > > > >> > >> > > > cases so it should make perfect
> logical
> >> >>>> sense in
> >> >>>> > >> the
> >> >>>> > >> > > > > absence
> >> >>>> > >> > > > > > of
> >> >>>> > >> > > > > > >> > >> knowing
> >> >>>> > >> > > > > > >> > >> > > the
> >> >>>> > >> > > > > > >> > >> > > > command line tool, right?
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> > >> > > > > > >> > >> > > > -Jay
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM,
> Andrii
> >> >>>> Biletskyi
> >> >>>> > >> <
> >> >>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> > >> > > > > > >> > >> > > > > Hey Jay,
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > > > I would like to continue this
> >> discussion
> >> >>>> as it
> >> >>>> > >> seem
> >> >>>> > >> > > > there
> >> >>>> > >> > > > > > is
> >> >>>> > >> > > > > > >> no
> >> >>>> > >> > > > > > >> > >> > > progress
> >> >>>> > >> > > > > > >> > >> > > > > here.
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > > > First of all, could you please
> explain
> >> >>>> what did
> >> >>>> > >> you
> >> >>>> > >> > > > mean
> >> >>>> > >> > > > > in
> >> >>>> > >> > > > > > >> 2?
> >> >>>> > >> > > > > > >> > How
> >> >>>> > >> > > > > > >> > >> > > > exactly
> >> >>>> > >> > > > > > >> > >> > > > > are we going to migrate to the new
> >> java
> >> >>>> protocol
> >> >>>> > >> > > > > > definitions.
> >> >>>> > >> > > > > > >> > And
> >> >>>> > >> > > > > > >> > >> why
> >> >>>> > >> > > > > > >> > >> > > > it's
> >> >>>> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > > > I agree with you, this feature
> >> includes
> >> >>>> lots of
> >> >>>> > >> > > stuff,
> >> >>>> > >> > > > > but
> >> >>>> > >> > > > > > >> > >> thankfully
> >> >>>> > >> > > > > > >> > >> > > > > almost all changes are isolated from
> >> the
> >> >>>> current
> >> >>>> > >> > code
> >> >>>> > >> > > > > base,
> >> >>>> > >> > > > > > >> > >> > > > > so the main thing, I think, we need
> to
> >> >>>> agree is
> >> >>>> > >> > RQ/RP
> >> >>>> > >> > > > > > format.
> >> >>>> > >> > > > > > >> > >> > > > > So how can we start discussion about
> >> the
> >> >>>> concrete
> >> >>>> > >> > > > > messages
> >> >>>> > >> > > > > > >> > format?
> >> >>>> > >> > > > > > >> > >> > > > > Can we take (
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> > >> > > > > > >> > >> > >
> >> >>>> > >> > > > > > >> > >> >
> >> >>>> > >> > > > > > >> > >>
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >>
> >> >>>> > >> > > > > >
> >> >>>> > >> > > > >
> >> >>>> > >> > > >
> >> >>>> > >> > >
> >> >>>> > >> >
> >> >>>> > >>
> >> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> >> >>>> > >> > > > > > >> > >> > > > > )
> >> >>>> > >> > > > > > >> > >> > > > > as starting point?
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > > > We had some doubts earlier whether
> it
> >> worth
> >> >>>> > >> > > introducing
> >> >>>> > >> > > > > one
> >> >>>> > >> > > > > > >> > >> generic
> >> >>>> > >> > > > > > >> > >> > > Admin
> >> >>>> > >> > > > > > >> > >> > > > > Request for all commands (
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> https://issues.apache.org/jira/browse/KAFKA-1694
> >> >>>> > >> > > > > > >> > >> > > > > )
> >> >>>> > >> > > > > > >> > >> > > > > but then everybody agreed it would
> be
> >> >>>> better to
> >> >>>> > >> > have
> >> >>>> > >> > > > > > separate
> >> >>>> > >> > > > > > >> > >> message
> >> >>>> > >> > > > > > >> > >> > > for
> >> >>>> > >> > > > > > >> > >> > > > > each admin command. The Request part
> >> is
> >> >>>> really
> >> >>>> > >> > > dictated
> >> >>>> > >> > > > > > from
> >> >>>> > >> > > > > > >> the
> >> >>>> > >> > > > > > >> > >> > > command
> >> >>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments
> itself,
> >> so
> >> >>>> the
> >> >>>> > >> > proposed
> >> >>>> > >> > > > > > version
> >> >>>> > >> > > > > > >> > >> should
> >> >>>> > >> > > > > > >> > >> > be
> >> >>>> > >> > > > > > >> > >> > > > > fine (let's put aside for now
> remarks
> >> about
> >> >>>> > >> > Optional
> >> >>>> > >> > > > > type,
> >> >>>> > >> > > > > > >> > >> batching,
> >> >>>> > >> > > > > > >> > >> > > > > configs normalization - I agree with
> >> all of
> >> >>>> > >> them).
> >> >>>> > >> > > > > > >> > >> > > > > So the second part is Response. I
> see
> >> >>>> there are
> >> >>>> > >> two
> >> >>>> > >> > > > cases
> >> >>>> > >> > > > > > >> here.
> >> >>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> >> Create/Alter/... ;
> >> >>>> b)
> >> >>>> > >> "Get"
> >> >>>> > >> > > > > > requests -
> >> >>>> > >> > > > > > >> > >> > > > > List/Describe...
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > > > a) should only hold request result
> >> >>>> (regardless
> >> >>>> > >> what
> >> >>>> > >> > > we
> >> >>>> > >> > > > > > decide
> >> >>>> > >> > > > > > >> > >> about
> >> >>>> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
> >> execution).
> >> >>>> > >> > > > > > >> > >> > > > > Usually we provide error code in
> >> response
> >> >>>> but
> >> >>>> > >> since
> >> >>>> > >> > > we
> >> >>>> > >> > > > > will
> >> >>>> > >> > > > > > >> use
> >> >>>> > >> > > > > > >> > >> this
> >> >>>> > >> > > > > > >> > >> > in
> >> >>>> > >> > > > > > >> > >> > > > > interactive shell we need some human
> >> >>>> readable
> >> >>>> > >> error
> >> >>>> > >> > > > > > >> description
> >> >>>> > >> > > > > > >> > -
> >> >>>> > >> > > > > > >> > >> so
> >> >>>> > >> > > > > > >> > >> > I
> >> >>>> > >> > > > > > >> > >> > > > > added errorDesription field where
> you
> >> can
> >> >>>> at
> >> >>>> > >> least
> >> >>>> > >> > > > leave
> >> >>>> > >> > > > > > >> > >> > > > > exception.getMessage.
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > > > b) in addition to previous item
> >> message
> >> >>>> should
> >> >>>> > >> hold
> >> >>>> > >> > > > > command
> >> >>>> > >> > > > > > >> > >> specific
> >> >>>> > >> > > > > > >> > >> > > > > response data. We can discuss in
> >> detail
> >> >>>> each of
> >> >>>> > >> > them
> >> >>>> > >> > > > but
> >> >>>> > >> > > > > > >> let's
> >> >>>> > >> > > > > > >> > for
> >> >>>> > >> > > > > > >> > >> > now
> >> >>>> > >> > > > > > >> > >> > > > > agree about the overall pattern.
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > > > Thanks,
> >> >>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay
> >> Kreps
> >> >>>> <
> >> >>>> > >> > > > > > >> jay.kreps@gmail.com
> >> >>>> > >> > > > > > >> > >
> >> >>>> > >> > > > > > >> > >> > > wrote:
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > Hey Joe,
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > This is great. A few comments on
> >> KIP-4
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > 1. This is much needed
> >> functionality,
> >> >>>> but there
> >> >>>> > >> > > are a
> >> >>>> > >> > > > > lot
> >> >>>> > >> > > > > > >> of
> >> >>>> > >> > > > > > >> > >> the so
> >> >>>> > >> > > > > > >> > >> > > > let's
> >> >>>> > >> > > > > > >> > >> > > > > > really think these protocols
> >> through. We
> >> >>>> really
> >> >>>> > >> > > want
> >> >>>> > >> > > > to
> >> >>>> > >> > > > > > >> end up
> >> >>>> > >> > > > > > >> > >> > with a
> >> >>>> > >> > > > > > >> > >> > > > set
> >> >>>> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol
> >> apis.
> >> >>>> For this
> >> >>>> > >> > > > reason I
> >> >>>> > >> > > > > > >> think
> >> >>>> > >> > > > > > >> > >> it is
> >> >>>> > >> > > > > > >> > >> > > > > really
> >> >>>> > >> > > > > > >> > >> > > > > > important to think through the end
> >> state
> >> >>>> even
> >> >>>> > >> if
> >> >>>> > >> > > that
> >> >>>> > >> > > > > > >> includes
> >> >>>> > >> > > > > > >> > >> APIs
> >> >>>> > >> > > > > > >> > >> > > we
> >> >>>> > >> > > > > > >> > >> > > > > > won't implement in the first
> phase.
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait
> >> until
> >> >>>> we
> >> >>>> > >> have
> >> >>>> > >> > > > > switched
> >> >>>> > >> > > > > > >> the
> >> >>>> > >> > > > > > >> > >> > server
> >> >>>> > >> > > > > > >> > >> > > > over
> >> >>>> > >> > > > > > >> > >> > > > > > to the new java protocol
> >> definitions. If
> >> >>>> we add
> >> >>>> > >> > > > upteen
> >> >>>> > >> > > > > > >> more ad
> >> >>>> > >> > > > > > >> > >> hoc
> >> >>>> > >> > > > > > >> > >> > > > scala
> >> >>>> > >> > > > > > >> > >> > > > > > objects that is just generating
> more
> >> >>>> work for
> >> >>>> > >> the
> >> >>>> > >> > > > > > >> conversion
> >> >>>> > >> > > > > > >> > we
> >> >>>> > >> > > > > > >> > >> > know
> >> >>>> > >> > > > > > >> > >> > > we
> >> >>>> > >> > > > > > >> > >> > > > > > have to do.
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new
> >> type of
> >> >>>> > >> > optional
> >> >>>> > >> > > > > > >> parameter.
> >> >>>> > >> > > > > > >> > >> This
> >> >>>> > >> > > > > > >> > >> > is
> >> >>>> > >> > > > > > >> > >> > > > > > inconsistent with everything else
> >> in the
> >> >>>> > >> protocol
> >> >>>> > >> > > > where
> >> >>>> > >> > > > > > we
> >> >>>> > >> > > > > > >> use
> >> >>>> > >> > > > > > >> > >> -1
> >> >>>> > >> > > > > > >> > >> > or
> >> >>>> > >> > > > > > >> > >> > > > some
> >> >>>> > >> > > > > > >> > >> > > > > > other marker value. You could
> argue
> >> >>>> either way
> >> >>>> > >> > but
> >> >>>> > >> > > > > let's
> >> >>>> > >> > > > > > >> stick
> >> >>>> > >> > > > > > >> > >> with
> >> >>>> > >> > > > > > >> > >> > > > that
> >> >>>> > >> > > > > > >> > >> > > > > > for consistency. For clients that
> >> >>>> implemented
> >> >>>> > >> the
> >> >>>> > >> > > > > > protocol
> >> >>>> > >> > > > > > >> in
> >> >>>> > >> > > > > > >> > a
> >> >>>> > >> > > > > > >> > >> > > better
> >> >>>> > >> > > > > > >> > >> > > > > way
> >> >>>> > >> > > > > > >> > >> > > > > > than our scala code these basic
> >> >>>> primitives are
> >> >>>> > >> > hard
> >> >>>> > >> > > > to
> >> >>>> > >> > > > > > >> change.
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
> >> >>>> duplicate
> >> >>>> > >> > > > > > >> > TopicMetadataRequest
> >> >>>> > >> > > > > > >> > >> > > which
> >> >>>> > >> > > > > > >> > >> > > > > has
> >> >>>> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I
> >> think
> >> >>>> we
> >> >>>> > >> > should
> >> >>>> > >> > > > > rename
> >> >>>> > >> > > > > > >> that
> >> >>>> > >> > > > > > >> > >> > > request
> >> >>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> >> >>>> > >> MetadataRequest)
> >> >>>> > >> > > and
> >> >>>> > >> > > > > > >> include
> >> >>>> > >> > > > > > >> > >> the id
> >> >>>> > >> > > > > > >> > >> > > of
> >> >>>> > >> > > > > > >> > >> > > > > the
> >> >>>> > >> > > > > > >> > >> > > > > > controller. Or are there other
> >> things we
> >> >>>> could
> >> >>>> > >> > add
> >> >>>> > >> > > > > here?
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to
> >> make a
> >> >>>> lot of
> >> >>>> > >> > > > requests
> >> >>>> > >> > > > > > that
> >> >>>> > >> > > > > > >> > can
> >> >>>> > >> > > > > > >> > >> > only
> >> >>>> > >> > > > > > >> > >> > > go
> >> >>>> > >> > > > > > >> > >> > > > > to
> >> >>>> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot
> of
> >> >>>> burden for
> >> >>>> > >> > > > client
> >> >>>> > >> > > > > > >> > >> > > implementations
> >> >>>> > >> > > > > > >> > >> > > > > (it
> >> >>>> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can
> >> fail
> >> >>>> in many
> >> >>>> > >> > > parts
> >> >>>> > >> > > > > so
> >> >>>> > >> > > > > > it
> >> >>>> > >> > > > > > >> > >> ends up
> >> >>>> > >> > > > > > >> > >> > > > > being a
> >> >>>> > >> > > > > > >> > >> > > > > > full state machine to do right). I
> >> think
> >> >>>> we
> >> >>>> > >> > should
> >> >>>> > >> > > > > > consider
> >> >>>> > >> > > > > > >> > >> making
> >> >>>> > >> > > > > > >> > >> > > > admin
> >> >>>> > >> > > > > > >> > >> > > > > > commands and ideally as many of
> the
> >> >>>> other apis
> >> >>>> > >> as
> >> >>>> > >> > > > > > possible
> >> >>>> > >> > > > > > >> > >> > available
> >> >>>> > >> > > > > > >> > >> > > on
> >> >>>> > >> > > > > > >> > >> > > > > all
> >> >>>> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
> >> >>>> controller on
> >> >>>> > >> > the
> >> >>>> > >> > > > > broker
> >> >>>> > >> > > > > > >> > side.
> >> >>>> > >> > > > > > >> > >> > > Perhaps
> >> >>>> > >> > > > > > >> > >> > > > > > there would be a general way to
> >> >>>> encapsulate
> >> >>>> > >> this
> >> >>>> > >> > > > > > re-routing
> >> >>>> > >> > > > > > >> > >> > behavior.
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > 6. We should probably normalize
> the
> >> key
> >> >>>> value
> >> >>>> > >> > pairs
> >> >>>> > >> > > > > used
> >> >>>> > >> > > > > > >> for
> >> >>>> > >> > > > > > >> > >> > configs
> >> >>>> > >> > > > > > >> > >> > > > > rather
> >> >>>> > >> > > > > > >> > >> > > > > > than embedding a new formatting.
> So
> >> two
> >> >>>> strings
> >> >>>> > >> > > > rather
> >> >>>> > >> > > > > > than
> >> >>>> > >> > > > > > >> > one
> >> >>>> > >> > > > > > >> > >> > with
> >> >>>> > >> > > > > > >> > >> > > an
> >> >>>> > >> > > > > > >> > >> > > > > > internal equals sign.
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these
> >> APIs
> >> >>>> that the
> >> >>>> > >> > > > command
> >> >>>> > >> > > > > > has
> >> >>>> > >> > > > > > >> > >> begun or
> >> >>>> > >> > > > > > >> > >> > > > that
> >> >>>> > >> > > > > > >> > >> > > > > > the command has been completed? It
> >> is a
> >> >>>> lot
> >> >>>> > >> more
> >> >>>> > >> > > > usable
> >> >>>> > >> > > > > > if
> >> >>>> > >> > > > > > >> the
> >> >>>> > >> > > > > > >> > >> > > command
> >> >>>> > >> > > > > > >> > >> > > > > has
> >> >>>> > >> > > > > > >> > >> > > > > > been completed so you know that if
> >> you
> >> >>>> create a
> >> >>>> > >> > > topic
> >> >>>> > >> > > > > and
> >> >>>> > >> > > > > > >> then
> >> >>>> > >> > > > > > >> > >> > > publish
> >> >>>> > >> > > > > > >> > >> > > > to
> >> >>>> > >> > > > > > >> > >> > > > > > it you won't get an exception
> about
> >> >>>> there being
> >> >>>> > >> > no
> >> >>>> > >> > > > such
> >> >>>> > >> > > > > > >> topic.
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
> >> >>>> duplicate a
> >> >>>> > >> lot
> >> >>>> > >> > > of
> >> >>>> > >> > > > > > stuff
> >> >>>> > >> > > > > > >> in
> >> >>>> > >> > > > > > >> > >> the
> >> >>>> > >> > > > > > >> > >> > > > > metadata
> >> >>>> > >> > > > > > >> > >> > > > > > request. Is there a reason to give
> >> back
> >> >>>> topics
> >> >>>> > >> > > marked
> >> >>>> > >> > > > > for
> >> >>>> > >> > > > > > >> > >> > deletion? I
> >> >>>> > >> > > > > > >> > >> > > > > feel
> >> >>>> > >> > > > > > >> > >> > > > > > like if we just make the
> >> post-condition
> >> >>>> of the
> >> >>>> > >> > > delete
> >> >>>> > >> > > > > > >> command
> >> >>>> > >> > > > > > >> > be
> >> >>>> > >> > > > > > >> > >> > that
> >> >>>> > >> > > > > > >> > >> > > > the
> >> >>>> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid
> >> of
> >> >>>> the need
> >> >>>> > >> > for
> >> >>>> > >> > > > this
> >> >>>> > >> > > > > > >> right?
> >> >>>> > >> > > > > > >> > >> And
> >> >>>> > >> > > > > > >> > >> > it
> >> >>>> > >> > > > > > >> > >> > > > > will
> >> >>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > 9. Should we consider batching
> these
> >> >>>> requests?
> >> >>>> > >> We
> >> >>>> > >> > > > have
> >> >>>> > >> > > > > > >> > generally
> >> >>>> > >> > > > > > >> > >> > > tried
> >> >>>> > >> > > > > > >> > >> > > > to
> >> >>>> > >> > > > > > >> > >> > > > > > allow multiple operations to be
> >> batched.
> >> >>>> My
> >> >>>> > >> > > suspicion
> >> >>>> > >> > > > > is
> >> >>>> > >> > > > > > >> that
> >> >>>> > >> > > > > > >> > >> > without
> >> >>>> > >> > > > > > >> > >> > > > > this
> >> >>>> > >> > > > > > >> > >> > > > > > we will get a lot of code that
> does
> >> >>>> something
> >> >>>> > >> > like
> >> >>>> > >> > > > > > >> > >> > > > > >    for(topic:
> >> adminClient.listTopics())
> >> >>>> > >> > > > > > >> > >> > > > > >
> >>  adminClient.describeTopic(topic)
> >> >>>> > >> > > > > > >> > >> > > > > > this code will work great when you
> >> test
> >> >>>> on 5
> >> >>>> > >> > topics
> >> >>>> > >> > > > but
> >> >>>> > >> > > > > > >> not do
> >> >>>> > >> > > > > > >> > >> as
> >> >>>> > >> > > > > > >> > >> > > well
> >> >>>> > >> > > > > > >> > >> > > > if
> >> >>>> > >> > > > > > >> > >> > > > > > you have 50k.
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss
> >> how
> >> >>>> we want
> >> >>>> > >> to
> >> >>>> > >> > > > > expose
> >> >>>> > >> > > > > > a
> >> >>>> > >> > > > > > >> > >> > > programmatic
> >> >>>> > >> > > > > > >> > >> > > > > JVM
> >> >>>> > >> > > > > > >> > >> > > > > > client api for these operations.
> >> >>>> Currently
> >> >>>> > >> people
> >> >>>> > >> > > > rely
> >> >>>> > >> > > > > on
> >> >>>> > >> > > > > > >> > >> > AdminUtils
> >> >>>> > >> > > > > > >> > >> > > > > which
> >> >>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we
> >> probably
> >> >>>> need
> >> >>>> > >> > > another
> >> >>>> > >> > > > > > client
> >> >>>> > >> > > > > > >> > >> under
> >> >>>> > >> > > > > > >> > >> > > > > clients/
> >> >>>> > >> > > > > > >> > >> > > > > > that exposes administrative
> >> >>>> functionality. We
> >> >>>> > >> > will
> >> >>>> > >> > > > need
> >> >>>> > >> > > > > > >> this
> >> >>>> > >> > > > > > >> > >> just
> >> >>>> > >> > > > > > >> > >> > to
> >> >>>> > >> > > > > > >> > >> > > > > > properly test the new apis, I
> >> suspect. We
> >> >>>> > >> should
> >> >>>> > >> > > > figure
> >> >>>> > >> > > > > > out
> >> >>>> > >> > > > > > >> > that
> >> >>>> > >> > > > > > >> > >> > API.
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > 11. The other information that
> >> would be
> >> >>>> really
> >> >>>> > >> > > useful
> >> >>>> > >> > > > > to
> >> >>>> > >> > > > > > >> get
> >> >>>> > >> > > > > > >> > >> would
> >> >>>> > >> > > > > > >> > >> > be
> >> >>>> > >> > > > > > >> > >> > > > > > information about partitions--how
> >> much
> >> >>>> data is
> >> >>>> > >> in
> >> >>>> > >> > > the
> >> >>>> > >> > > > > > >> > partition,
> >> >>>> > >> > > > > > >> > >> > what
> >> >>>> > >> > > > > > >> > >> > > > are
> >> >>>> > >> > > > > > >> > >> > > > > > the segment offsets, what is the
> >> log-end
> >> >>>> offset
> >> >>>> > >> > > (i.e.
> >> >>>> > >> > > > > > last
> >> >>>> > >> > > > > > >> > >> offset),
> >> >>>> > >> > > > > > >> > >> > > > what
> >> >>>> > >> > > > > > >> > >> > > > > is
> >> >>>> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think
> >> that
> >> >>>> done
> >> >>>> > >> > right
> >> >>>> > >> > > > this
> >> >>>> > >> > > > > > >> would
> >> >>>> > >> > > > > > >> > be
> >> >>>> > >> > > > > > >> > >> > the
> >> >>>> > >> > > > > > >> > >> > > > > > successor to the very awkward
> >> >>>> OffsetRequest we
> >> >>>> > >> > have
> >> >>>> > >> > > > > > today.
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > -Jay
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM,
> >> Joe
> >> >>>> Stein <
> >> >>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
> >> >>>> > >> > > > > > >> > >> > > > > wrote:
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> >> >>>> > >> > > > > > >> > >> > > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> > >> > > > > > >> > >> > >
> >> >>>> > >> > > > > > >> > >> >
> >> >>>> > >> > > > > > >> > >>
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >>
> >> >>>> > >> > > > > >
> >> >>>> > >> > > > >
> >> >>>> > >> > > >
> >> >>>> > >> > >
> >> >>>> > >> >
> >> >>>> > >>
> >> >>>>
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> >>>> > >> > > > > > >> > >> > > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > > JIRA
> >> >>>> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> >> >>>> > >> > > > > > >> > >> > > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > > >
> >> >>>> /*******************************************
> >> >>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
> >> >>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> >> >>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security
> LLC
> >> >>>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> >> >>>> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> >> >>>> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> > >> > > > > > >> > >> > > > > > >
> >> >>>> ********************************************/
> >> >>>> > >> > > > > > >> > >> > > > > > >
> >> >>>> > >> > > > > > >> > >> > > > > >
> >> >>>> > >> > > > > > >> > >> > > > >
> >> >>>> > >> > > > > > >> > >> > > >
> >> >>>> > >> > > > > > >> > >> > >
> >> >>>> > >> > > > > > >> > >> >
> >> >>>> > >> > > > > > >> > >>
> >> >>>> > >> > > > > > >> > >>
> >> >>>> > >> > > > > > >> > >>
> >> >>>> > >> > > > > > >> > >> --
> >> >>>> > >> > > > > > >> > >> -- Guozhang
> >> >>>> > >> > > > > > >> > >>
> >> >>>> > >> > > > > > >> > >
> >> >>>> > >> > > > > > >> > >
> >> >>>> > >> > > > > > >> >
> >> >>>> > >> > > > > > >>
> >> >>>> > >> > > > > > >
> >> >>>> > >> > > > > > >
> >> >>>> > >> > > > > >
> >> >>>> > >> > > > >
> >> >>>> > >> > > >
> >> >>>> > >> > >
> >> >>>> > >> >
> >> >>>> > >> >
> >> >>>> > >> >
> >> >>>> > >> > --
> >> >>>> > >> > Jeff Holoman
> >> >>>> > >> > Systems Engineer
> >> >>>> > >> >
> >> >>>> > >>
> >> >>>>
> >> >>>>
> >> >>>
> >> >>>
> >> >>> --
> >> >>> -- Guozhang
> >> >>>
> >> >>
> >> >>
> >> >>
> >> >> --
> >> >> -- Guozhang
> >>
> >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Hi,

As said above - I list again all comments from this thread so we
can see what's left and finalize all pending issues.

Comments from Jay:
1. This is much needed functionality, but there are a lot of the so let's
really think these protocols through. We really want to end up with a set
of well thought-out, orthoganol apis. For this reason I think it is really
important to think through the end state even if that includes APIs we
won't implement in the first phase.

A: Definitely behind this. Would appreciate if there are concrete comments
how this can be improved.

2. Let's please please please wait until we have switched the server over
to the new java protocol definitions. If we add upteen more ad hoc scala
objects that is just generating more work for the conversion we know we
have to do.

A: Fixed in the latest patch - removed scala protocol classes.

3. This proposal introduces a new type of optional parameter. This is
inconsistent with everything else in the protocol where we use -1 or some
other marker value. You could argue either way but let's stick with that
for consistency. For clients that implemented the protocol in a better way
than our scala code these basic primitives are hard to change.

A: Fixed in the latest patch - removed MaybeOf type and changed protocol
accordingly.

4. ClusterMetadata: This seems to duplicate TopicMetadataRequest which has
brokers, topics, and partitions. I think we should rename that request
ClusterMetadataRequest (or just MetadataRequest) and include the id of the
controller. Or are there other things we could add here?

A: I agree. Updated the KIP. Let's extends TopicMetadata to version 2 and
include controller.

5. We have a tendency to try to make a lot of requests that can only go to
particular nodes. This adds a lot of burden for client implementations (it
sounds easy but each discovery can fail in many parts so it ends up being a
full state machine to do right). I think we should consider making admin
commands and ideally as many of the other apis as possible available on all
brokers and just redirect to the controller on the broker side. Perhaps
there would be a general way to encapsulate this re-routing behavior.

A: It's a very interesting idea, but seems there are some concerns about
this
feature (like performance considerations, how this will complicate server
etc).
I believe this shouldn't be a blocker. If this feature is implemented at
some
point it won't affect Admin changes - at least no changes to public API
will be required.

6. We should probably normalize the key value pairs used for configs rather
than embedding a new formatting. So two strings rather than one with an
internal equals sign.

A: Fixed in the latest patch - normalized configs and changed protocol
accordingly.

7. Is the postcondition of these APIs that the command has begun or that
the command has been completed? It is a lot more usable if the command has
been completed so you know that if you create a topic and then publish to
it you won't get an exception about there being no such topic.

A: For long running requests (like reassign partitions) - the post
condition is
command has begun - so we don't block the client. In case of your example -
topic commands, this will be refactored and topic commands will be executed
immediately, since the Controller will serve Admin requests
(follow-up ticket KAFKA-1777).

8. Describe topic and list topics duplicate a lot of stuff in the metadata
request. Is there a reason to give back topics marked for deletion? I feel
like if we just make the post-condition of the delete command be that the
topic is deleted that will get rid of the need for this right? And it will
be much more intuitive.

A: Fixed in the latest patch - removed topics marked for deletion in
ListTopicsRequest.

9. Should we consider batching these requests? We have generally tried to
allow multiple operations to be batched. My suspicion is that without this
we will get a lot of code that does something like
   for(topic: adminClient.listTopics())
      adminClient.describeTopic(topic)
this code will work great when you test on 5 topics but not do as well if
you have 50k.

A: Updated the KIP - please check "Topic Admin Schema" section.

10. I think we should also discuss how we want to expose a programmatic JVM
client api for these operations. Currently people rely on AdminUtils which
is totally sketchy. I think we probably need another client under clients/
that exposes administrative functionality. We will need this just to
properly test the new apis, I suspect. We should figure out that API.

A: Updated the KIP - please check "Admin Client" section with an initial
API proposal.

11. The other information that would be really useful to get would be
information about partitions--how much data is in the partition, what are
the segment offsets, what is the log-end offset (i.e. last offset), what is
the compaction point, etc. I think that done right this would be the
successor to the very awkward OffsetRequest we have today.

A: I removed ConsumerGroupOffsetsRequest in the latest patch. I believe
this should
be resolved in a separate KIP / jira ticket.

12. Generally we can do good error handling without needing custom
server-side
messages. I.e. generally the client has the context to know that if it got
an error that the topic doesn't exist to say "Topic X doesn't exist" rather
than "error code 14" (or whatever). Maybe there are specific cases where
this is hard? If we want to add server-side error messages we really do
need to do this in a consistent way across the protocol.

A: Updated the KIP - please check "Protocol Errors" section. I added the
comprehensive, fine-grained list of error codes.

Comments from Guozhang:
13. Describe topic request: it would be great to go beyond just batching on
topic name regex for this request. For example, a very common use case of
the topic command is to list all topics whose config A's value is B. With
topic name regex then we have to first retrieve __all__ topics's
description info and then filter at the client end, which will be a huge
burden on ZK.
AND
14. Config K-Vs in create topic: this is related to the previous point;
maybe we can add another metadata K-V or just a metadata string along side
with config K-V in create topic like we did for offset commit request. This
field can be quite useful in storing information like "owner" of the topic
who issue the create command, etc, which is quite important for a
multi-tenant setting. Then in the describe topic request we can also batch
on regex of the metadata field.

A: As discussed it is very interesting but can be implemented later after
we have some basic functionality there.

15. Today all the admin operations are async in the sense that command will
return once it is written in ZK, and that is why we need extra verification
like testUtil.waitForTopicCreated() / verify partition reassignment
request, etc. With admin requests we could add a flag to enable / disable
synchronous requests; when it is turned on, the response will not return
until the request has been completed. And for async requests we can add a
"token" field in the response, and then only need a general "admin
verification request" with the given token to check if the async request
has been completed.

A: I see your point. My idea was to provide specific Verify...Request per
each
long running request, where needed. We can do it the way you suggest. The
only
concern is that introducing a token we again will make schema "dynamic". We
wanted
to do similar thing introducing single AdminRequest for all topic commands
but rejected
this idea because we wanted to have schema defined. So this is more a
choice between:
a) have fixed schema but introduce each time new Verify...Request for
long-running requests
b) use one request for verification but generalize it with token
I'm fine with whatever decision community come to. Just let me know your
thoughts.

Comment from Gwen:
16. Specifically for ownership, I think the plan is to add ACL (it sounds
like you are describing ACL) via an external system (Argus, Sentry).
I remember KIP-11 described this, but I can't find the KIP any longer.

A: Okay, no problem. Not sure though how we are going to handle it. Wait
which KIP
will be committed first and include changes to TopicMetadata from the later
one?
Anyway, I added this note to "Open Questions" section so we don't miss this
piece.

Thanks,
Andrii Biletskyi

On Fri, Mar 13, 2015 at 12:34 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Hi all,
>
> Today I uploaded the patch that covers some of the discussed and agreed
> items:
> - removed MaybeOf optional type
> - switched to java protocol definitions
> - simplified messages (normalized configs, removed topic marked for
> deletion)
>
> I also updated the KIP-4 with respective changes and wrote down my
> proposal for
> pending items:
> - Batch Admin Operations -> updated Wire Protocol schema proposal
> - Remove ClusterMetadata -> changed to extend TopicMetadataRequest
> - Admin Client -> updated my initial proposal to reflect batching
> - Error codes -> proposed fine-grained error code instead of
> AdminRequestFailed
>
> I will also send a separate email to cover all comments from this thread.
>
> Thanks,
> Andrii Biletskyi
>
>
> On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <gs...@cloudera.com>
> wrote:
>
>> Found KIP-11 (
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
>> )
>> It actually specifies changes to the Metadata protocol, so making sure
>> both KIPs are consistent in this regard will be good.
>>
>> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <gs...@cloudera.com>
>> wrote:
>> > Specifically for ownership, I think the plan is to add ACL (it sounds
>> > like you are describing ACL) via an external system (Argus, Sentry).
>> > I remember KIP-11 described this, but I can't find the KIP any longer.
>> >
>> > Regardless, I think KIP-4 focuses on getting information that already
>> > exists from Kafka brokers, not on adding information that perhaps
>> > should exist but doesn't yet?
>> >
>> > Gwen
>> >
>> >
>> >
>> >
>> >
>> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <wa...@gmail.com>
>> wrote:
>> >> Folks,
>> >>
>> >> Just want to elaborate a bit more on the create-topic metadata and
>> batching
>> >> describe-topic based on config / metadata in my previous email as we
>> work
>> >> on KAFKA-1694. The main motivation is to have some sort of topic
>> management
>> >> mechanisms, which I think is quite important in a multi-tenant / cloud
>> >> architecture: today anyone can create topics in a shared Kafka
>> cluster, but
>> >> there is no concept or "ownership" of topics that are created by
>> different
>> >> users. For example, at LinkedIn we basically distinguish topic owners
>> via
>> >> some casual topic name prefix, which is a bit awkward and does not fly
>> as
>> >> we scale our customers. It would be great to use describe-topics such
>> as:
>> >>
>> >> Describe all topics that is created by me.
>> >>
>> >> Describe all topics whose retention time is overriden to X.
>> >>
>> >> Describe all topics whose writable group include user Y (this is
>> related to
>> >> authorization), etc..
>> >>
>> >> One possible way to achieve this is to add a metadata file in the
>> >> create-topic request, whose value will also be written ZK as we create
>> the
>> >> topic; then describe-topics can choose to batch topics based on 1) name
>> >> regex, 2) config K-V matching, 3) metadata regex, etc.
>> >>
>> >> Thoughts?
>> >>
>> >> Guozhang
>> >>
>> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com>
>> wrote:
>> >>
>> >>> Thanks for the updated wiki. A few comments below:
>> >>>
>> >>> 1. Error description in response: I think if some errorCode could
>> indicate
>> >>> several different error cases then we should really change it to
>> multiple
>> >>> codes. In general the errorCode itself would be precise and
>> sufficient for
>> >>> describing the server side errors.
>> >>>
>> >>> 2. Describe topic request: it would be great to go beyond just
>> batching on
>> >>> topic name regex for this request. For example, a very common use
>> case of
>> >>> the topic command is to list all topics whose config A's value is B.
>> With
>> >>> topic name regex then we have to first retrieve __all__ topics's
>> >>> description info and then filter at the client end, which will be a
>> huge
>> >>> burden on ZK.
>> >>>
>> >>> 3. Config K-Vs in create topic: this is related to the previous point;
>> >>> maybe we can add another metadata K-V or just a metadata string along
>> side
>> >>> with config K-V in create topic like we did for offset commit
>> request. This
>> >>> field can be quite useful in storing information like "owner" of the
>> topic
>> >>> who issue the create command, etc, which is quite important for a
>> >>> multi-tenant setting. Then in the describe topic request we can also
>> batch
>> >>> on regex of the metadata field.
>> >>>
>> >>> 4. Today all the admin operations are async in the sense that command
>> will
>> >>> return once it is written in ZK, and that is why we need extra
>> verification
>> >>> like testUtil.waitForTopicCreated() / verify partition reassignment
>> >>> request, etc. With admin requests we could add a flag to enable /
>> disable
>> >>> synchronous requests; when it is turned on, the response will not
>> return
>> >>> until the request has been completed. And for async requests we can
>> add a
>> >>> "token" field in the response, and then only need a general "admin
>> >>> verification request" with the given token to check if the async
>> request
>> >>> has been completed.
>> >>>
>> >>> 5. +1 for extending Metadata request to include controller /
>> coordinator
>> >>> information, and then we can remove the ConsumerMetadata /
>> ClusterMetadata
>> >>> requests.
>> >>>
>> >>> Guozhang
>> >>>
>> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com>
>> wrote:
>> >>>
>> >>>> Thanks for sending that out Joe - I don't think I will be able to
>> make
>> >>>> it today, so if notes can be sent out afterward that would be great.
>> >>>>
>> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
>> >>>> > Thanks for sending this out Joe. Looking forward to chatting with
>> >>>> everyone :)
>> >>>> >
>> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly>
>> wrote:
>> >>>> > > Hey, I just sent out a google hangout invite to all pmc,
>> committers
>> >>>> and
>> >>>> > > everyone I found working on a KIP. If I missed anyone in the
>> invite
>> >>>> please
>> >>>> > > let me know and can update it, np.
>> >>>> > >
>> >>>> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can
>> get
>> >>>> INFRA
>> >>>> > > help to make a google account so we can manage better?
>> >>>> > >
>> >>>> > > To discuss
>> >>>> > >
>> >>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>> >>>> > > in progress and related JIRA that are interdependent and common
>> work.
>> >>>> > >
>> >>>> > > ~ Joe Stein
>> >>>> > >
>> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com>
>> >>>> wrote:
>> >>>> > >
>> >>>> > >> Let's stay on Google hangouts that will also record and make the
>> >>>> sessions
>> >>>> > >> available on youtube.
>> >>>> > >>
>> >>>> > >> -Jay
>> >>>> > >>
>> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
>> >>>> jholoman@cloudera.com>
>> >>>> > >> wrote:
>> >>>> > >>
>> >>>> > >> > Jay / Joe
>> >>>> > >> >
>> >>>> > >> > We're happy to send out a Webex for this purpose. We could
>> record
>> >>>> the
>> >>>> > >> > sessions if there is interest and publish them out.
>> >>>> > >> >
>> >>>> > >> > Thanks
>> >>>> > >> >
>> >>>> > >> > Jeff
>> >>>> > >> >
>> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
>> jay.kreps@gmail.com>
>> >>>> wrote:
>> >>>> > >> >
>> >>>> > >> > > Let's try to get the technical hang-ups sorted out, though.
>> I
>> >>>> really
>> >>>> > >> > think
>> >>>> > >> > > there is some benefit to live discussion vs writing. I am
>> >>>> hopeful that
>> >>>> > >> if
>> >>>> > >> > > we post instructions and give ourselves a few attempts we
>> can
>> >>>> get it
>> >>>> > >> > > working.
>> >>>> > >> > >
>> >>>> > >> > > Tuesday at that time would work for me...any objections?
>> >>>> > >> > >
>> >>>> > >> > > -Jay
>> >>>> > >> > >
>> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
>> joe.stein@stealth.ly
>> >>>> >
>> >>>> > >> wrote:
>> >>>> > >> > >
>> >>>> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET /
>> 10am
>> >>>> PT
>> >>>> > >> ????
>> >>>> > >> > > >
>> >>>> > >> > > > I don't mind google hangout but there is always some
>> issue or
>> >>>> > >> whatever
>> >>>> > >> > so
>> >>>> > >> > > > we know the apache irc channel works. We can start there
>> and
>> >>>> see how
>> >>>> > >> it
>> >>>> > >> > > > goes? We can pull transcripts too and associate to
>> tickets if
>> >>>> need be
>> >>>> > >> > > makes
>> >>>> > >> > > > it helpful for things.
>> >>>> > >> > > >
>> >>>> > >> > > > ~ Joestein
>> >>>> > >> > > >
>> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
>> >>>> jay.kreps@gmail.com>
>> >>>> > >> > wrote:
>> >>>> > >> > > >
>> >>>> > >> > > > > We'd talked about doing a Google Hangout to chat about
>> this.
>> >>>> What
>> >>>> > >> > about
>> >>>> > >> > > > > generalizing that a little further...I actually think it
>> >>>> would be
>> >>>> > >> > good
>> >>>> > >> > > > for
>> >>>> > >> > > > > everyone spending a reasonable chunk of their week on
>> Kafka
>> >>>> stuff
>> >>>> > >> to
>> >>>> > >> > > > maybe
>> >>>> > >> > > > > sync up once a week. I think we could use time to talk
>> >>>> through
>> >>>> > >> design
>> >>>> > >> > > > > stuff, make sure we are on top of code reviews, talk
>> through
>> >>>> any
>> >>>> > >> > tricky
>> >>>> > >> > > > > issues, etc.
>> >>>> > >> > > > >
>> >>>> > >> > > > > We can make it publicly available so that any one can
>> follow
>> >>>> along
>> >>>> > >> > who
>> >>>> > >> > > > > likes.
>> >>>> > >> > > > >
>> >>>> > >> > > > > Any interest in doing this? If so I'll try to set it up
>> >>>> starting
>> >>>> > >> next
>> >>>> > >> > > > week.
>> >>>> > >> > > > >
>> >>>> > >> > > > > -Jay
>> >>>> > >> > > > >
>> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
>> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
>> >>>> > >> > > > >
>> >>>> > >> > > > > > Hi all,
>> >>>> > >> > > > > >
>> >>>> > >> > > > > > I've updated KIP page, fixed / aligned document
>> structure.
>> >>>> Also I
>> >>>> > >> > > added
>> >>>> > >> > > > > > some
>> >>>> > >> > > > > > very initial proposal for AdminClient so we have
>> something
>> >>>> to
>> >>>> > >> start
>> >>>> > >> > > > from
>> >>>> > >> > > > > > while
>> >>>> > >> > > > > > discussing the KIP.
>> >>>> > >> > > > > >
>> >>>> > >> > > > > >
>> >>>> > >> > > > > >
>> >>>> > >> > > > >
>> >>>> > >> > > >
>> >>>> > >> > >
>> >>>> > >> >
>> >>>> > >>
>> >>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>> >>>> > >> > > > > >
>> >>>> > >> > > > > > Thanks,
>> >>>> > >> > > > > > Andrii Biletskyi
>> >>>> > >> > > > > >
>> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
>> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
>> >>>> > >> > > > > >
>> >>>> > >> > > > > > > Jay,
>> >>>> > >> > > > > > >
>> >>>> > >> > > > > > > Re error messages: you are right, in most cases
>> client
>> >>>> will
>> >>>> > >> have
>> >>>> > >> > > > enough
>> >>>> > >> > > > > > > context to show descriptive error message. My
>> concern is
>> >>>> that
>> >>>> > >> we
>> >>>> > >> > > will
>> >>>> > >> > > > > > have
>> >>>> > >> > > > > > > to
>> >>>> > >> > > > > > > add lots of new error codes for each possible
>> error. Of
>> >>>> course,
>> >>>> > >> > we
>> >>>> > >> > > > > could
>> >>>> > >> > > > > > > reuse
>> >>>> > >> > > > > > > some of existing like UknownTopicOrPartitionCode,
>> but we
>> >>>> will
>> >>>> > >> > also
>> >>>> > >> > > > need
>> >>>> > >> > > > > > to
>> >>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
>> >>>> TopicConfigInvalid (both
>> >>>> > >> > for
>> >>>> > >> > > > > topic
>> >>>> > >> > > > > > > name and config, and probably user would like to
>> know
>> >>>> what
>> >>>> > >> > exactly
>> >>>> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
>> >>>> > >> InternalError
>> >>>> > >> > > > (e.g.
>> >>>> > >> > > > > > > zookeeper failure) etc.
>> >>>> > >> > > > > > > And this is only for TopicCommand, we will also
>> need to
>> >>>> add
>> >>>> > >> > similar
>> >>>> > >> > > > > stuff
>> >>>> > >> > > > > > > for
>> >>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end
>> up
>> >>>> with a
>> >>>> > >> > large
>> >>>> > >> > > > list
>> >>>> > >> > > > > > of
>> >>>> > >> > > > > > > error codes, used only in Admin protocol.
>> >>>> > >> > > > > > > Having said that, I agree my proposal is not
>> consistent
>> >>>> with
>> >>>> > >> > other
>> >>>> > >> > > > > cases.
>> >>>> > >> > > > > > > Maybe we can find better solution or something
>> >>>> in-between.
>> >>>> > >> > > > > > >
>> >>>> > >> > > > > > > Re Hangout chat: I think it is a great idea. This
>> way we
>> >>>> can
>> >>>> > >> move
>> >>>> > >> > > on
>> >>>> > >> > > > > > > faster.
>> >>>> > >> > > > > > > Let's agree somehow on date/time so people can join.
>> >>>> Will work
>> >>>> > >> > for
>> >>>> > >> > > me
>> >>>> > >> > > > > > this
>> >>>> > >> > > > > > > and
>> >>>> > >> > > > > > > next week almost anytime if agreed in advance.
>> >>>> > >> > > > > > >
>> >>>> > >> > > > > > > Thanks,
>> >>>> > >> > > > > > > Andrii
>> >>>> > >> > > > > > >
>> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
>> >>>> > >> jay.kreps@gmail.com>
>> >>>> > >> > > > > wrote:
>> >>>> > >> > > > > > >
>> >>>> > >> > > > > > >> Hey Andrii,
>> >>>> > >> > > > > > >>
>> >>>> > >> > > > > > >> Generally we can do good error handling without
>> needing
>> >>>> custom
>> >>>> > >> > > > > > server-side
>> >>>> > >> > > > > > >> messages. I.e. generally the client has the
>> context to
>> >>>> know
>> >>>> > >> that
>> >>>> > >> > > if
>> >>>> > >> > > > it
>> >>>> > >> > > > > > got
>> >>>> > >> > > > > > >> an error that the topic doesn't exist to say
>> "Topic X
>> >>>> doesn't
>> >>>> > >> > > exist"
>> >>>> > >> > > > > > >> rather
>> >>>> > >> > > > > > >> than "error code 14" (or whatever). Maybe there are
>> >>>> specific
>> >>>> > >> > cases
>> >>>> > >> > > > > where
>> >>>> > >> > > > > > >> this is hard? If we want to add server-side error
>> >>>> messages we
>> >>>> > >> > > really
>> >>>> > >> > > > > do
>> >>>> > >> > > > > > >> need to do this in a consistent way across the
>> protocol.
>> >>>> > >> > > > > > >>
>> >>>> > >> > > > > > >> I still have a bunch of open questions here from my
>> >>>> previous
>> >>>> > >> > > list. I
>> >>>> > >> > > > > > will
>> >>>> > >> > > > > > >> be out for the next few days for Strata though.
>> Maybe
>> >>>> we could
>> >>>> > >> > do
>> >>>> > >> > > a
>> >>>> > >> > > > > > Google
>> >>>> > >> > > > > > >> Hangout chat on any open issues some time towards
>> the
>> >>>> end of
>> >>>> > >> > next
>> >>>> > >> > > > week
>> >>>> > >> > > > > > for
>> >>>> > >> > > > > > >> anyone interested in this ticket? I have a feeling
>> that
>> >>>> might
>> >>>> > >> > > > progress
>> >>>> > >> > > > > > >> things a little faster than email--I think we
>> could talk
>> >>>> > >> through
>> >>>> > >> > > > those
>> >>>> > >> > > > > > >> issues I brought up fairly quickly...
>> >>>> > >> > > > > > >>
>> >>>> > >> > > > > > >> -Jay
>> >>>> > >> > > > > > >>
>> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
>> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
>> >>>> > >> > > > > > >>
>> >>>> > >> > > > > > >> > Hi all,
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >> > I'm trying to address some of the issues which
>> were
>> >>>> > >> mentioned
>> >>>> > >> > > > > earlier
>> >>>> > >> > > > > > >> about
>> >>>> > >> > > > > > >> > Admin RQ/RP format. One of those was about
>> batching
>> >>>> > >> > operations.
>> >>>> > >> > > > What
>> >>>> > >> > > > > > if
>> >>>> > >> > > > > > >> we
>> >>>> > >> > > > > > >> > follow TopicCommand approach and let people
>> specify
>> >>>> > >> topic-name
>> >>>> > >> > > by
>> >>>> > >> > > > > > >> regexp -
>> >>>> > >> > > > > > >> > would that cover most of the use cases?
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >> > Secondly, is what information should we generally
>> >>>> provide in
>> >>>> > >> > > Admin
>> >>>> > >> > > > > > >> > responses.
>> >>>> > >> > > > > > >> > I realize that Admin commands don't imply they
>> will
>> >>>> be used
>> >>>> > >> > only
>> >>>> > >> > > > in
>> >>>> > >> > > > > > CLI
>> >>>> > >> > > > > > >> > but,
>> >>>> > >> > > > > > >> > it seems to me, CLI is a very important client
>> of this
>> >>>> > >> > feature.
>> >>>> > >> > > In
>> >>>> > >> > > > > > this
>> >>>> > >> > > > > > >> > case,
>> >>>> > >> > > > > > >> > seems logical, we would like to provide users
>> with
>> >>>> rich
>> >>>> > >> > > experience
>> >>>> > >> > > > > in
>> >>>> > >> > > > > > >> terms
>> >>>> > >> > > > > > >> > of
>> >>>> > >> > > > > > >> > getting results / errors of the executed
>> commands.
>> >>>> Usually
>> >>>> > >> we
>> >>>> > >> > > > supply
>> >>>> > >> > > > > > >> with
>> >>>> > >> > > > > > >> > responses only errorCode, which looks very
>> limiting,
>> >>>> in case
>> >>>> > >> > of
>> >>>> > >> > > > CLI
>> >>>> > >> > > > > we
>> >>>> > >> > > > > > >> may
>> >>>> > >> > > > > > >> > want to print human readable error description.
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >> > So, taking into account previous item about
>> batching,
>> >>>> what
>> >>>> > >> do
>> >>>> > >> > > you
>> >>>> > >> > > > > > think
>> >>>> > >> > > > > > >> > about
>> >>>> > >> > > > > > >> > having smth like:
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >> > ('create' doesn't support regexp)
>> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions
>> Replicas
>> >>>> > >> > > > > ReplicaAssignment
>> >>>> > >> > > > > > >> > [Config]
>> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
>> >>>> > >> > > > > > >> >   ErrorCode => int16
>> >>>> > >> > > > > > >> >   ErrorDescription => string (empty if
>> successful)
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
>> >>>> > >> > > ReplicaAssignment
>> >>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
>> >>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
>> >>>> ErrorDescription]
>> >>>> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
>> >>>> > >> > > > > > >> >   CommandErrorCode => int16
>> >>>> > >> > > > > > >> >   CommandErrorDescription => string (nonempty in
>> case
>> >>>> of
>> >>>> > >> fatal
>> >>>> > >> > > > > error,
>> >>>> > >> > > > > > >> e.g.
>> >>>> > >> > > > > > >> > we couldn't get topics by regexp)
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
>> >>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
>> TopicDescription
>> >>>> > >> ErrorCode
>> >>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
>> >>>> CommandErrorDescription
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >> > Also, any thoughts about our discussion regarding
>> >>>> re-routing
>> >>>> > >> > > > > facility?
>> >>>> > >> > > > > > >> In
>> >>>> > >> > > > > > >> > my
>> >>>> > >> > > > > > >> > understanding, it is like between augmenting
>> >>>> > >> > > TopicMetadataRequest
>> >>>> > >> > > > > > >> > (to include at least controllerId) and
>> implementing
>> >>>> new
>> >>>> > >> > generic
>> >>>> > >> > > > > > >> re-routing
>> >>>> > >> > > > > > >> > facility so sending messages to controller will
>> be
>> >>>> handled
>> >>>> > >> by
>> >>>> > >> > > it.
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >> > Thanks,
>> >>>> > >> > > > > > >> > Andrii Biletskyi
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii
>> Biletskyi <
>> >>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >> > > @Guozhang:
>> >>>> > >> > > > > > >> > > Thanks for your comments, I've answered some of
>> >>>> those. The
>> >>>> > >> > > main
>> >>>> > >> > > > > > thing
>> >>>> > >> > > > > > >> is
>> >>>> > >> > > > > > >> > > having merged request for
>> >>>> create-alter-delete-describe - I
>> >>>> > >> > > have
>> >>>> > >> > > > > some
>> >>>> > >> > > > > > >> > > concerns about this approach.
>> >>>> > >> > > > > > >> > >
>> >>>> > >> > > > > > >> > > @*Jay*:
>> >>>> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is
>> also
>> >>>> one of
>> >>>> > >> > the
>> >>>> > >> > > > > > >> concerns.
>> >>>> > >> > > > > > >> > We
>> >>>> > >> > > > > > >> > > can solve it if we implement re-routing
>> facility.
>> >>>> But I
>> >>>> > >> > agree
>> >>>> > >> > > > with
>> >>>> > >> > > > > > >> > > Guozhang - it will make clients' internals a
>> little
>> >>>> bit
>> >>>> > >> > easier
>> >>>> > >> > > > but
>> >>>> > >> > > > > > >> this
>> >>>> > >> > > > > > >> > > seems to be a complex logic to implement and
>> >>>> support then.
>> >>>> > >> > > > > > Especially
>> >>>> > >> > > > > > >> for
>> >>>> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing
>> later
>> >>>> for
>> >>>> > >> these
>> >>>> > >> > > > > > >> requests).
>> >>>> > >> > > > > > >> > > Also people will tend to avoid this re-routing
>> >>>> facility
>> >>>> > >> and
>> >>>> > >> > > hold
>> >>>> > >> > > > > > local
>> >>>> > >> > > > > > >> > > cluster cache to ensure their high-priority
>> requests
>> >>>> > >> (which
>> >>>> > >> > > some
>> >>>> > >> > > > > of
>> >>>> > >> > > > > > >> the
>> >>>> > >> > > > > > >> > > admin requests are) not sent to some busy
>> broker
>> >>>> where
>> >>>> > >> they
>> >>>> > >> > > wait
>> >>>> > >> > > > > to
>> >>>> > >> > > > > > be
>> >>>> > >> > > > > > >> > > routed to the correct one.
>> >>>> > >> > > > > > >> > > As pointed out by Jun here (
>> >>>> > >> > > > > > >> > >
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >>
>> >>>> > >> > > > > >
>> >>>> > >> > > > >
>> >>>> > >> > > >
>> >>>> > >> > >
>> >>>> > >> >
>> >>>> > >>
>> >>>>
>> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
>> >>>> > >> > > > > > >> > )
>> >>>> > >> > > > > > >> > > to solve the issue we might introduce a message
>> >>>> type to
>> >>>> > >> get
>> >>>> > >> > > > > cluster
>> >>>> > >> > > > > > >> > state.
>> >>>> > >> > > > > > >> > > But I agree we can just update
>> >>>> TopicMetadataResponse to
>> >>>> > >> > > include
>> >>>> > >> > > > > > >> > > controllerId (and probably smth else).
>> >>>> > >> > > > > > >> > > What are you thougths?
>> >>>> > >> > > > > > >> > >
>> >>>> > >> > > > > > >> > > Thanks,
>> >>>> > >> > > > > > >> > > Andrii
>> >>>> > >> > > > > > >> > >
>> >>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang
>> <
>> >>>> > >> > > > > wangguoz@gmail.com>
>> >>>> > >> > > > > > >> > wrote:
>> >>>> > >> > > > > > >> > >
>> >>>> > >> > > > > > >> > >> I think for the topics commands we can
>> actually
>> >>>> merge
>> >>>> > >> > > > > > >> > >> create/alter/delete/describe as one request
>> type
>> >>>> since
>> >>>> > >> > their
>> >>>> > >> > > > > > formats
>> >>>> > >> > > > > > >> are
>> >>>> > >> > > > > > >> > >> very much similar, and keep list-topics and
>> others
>> >>>> like
>> >>>> > >> > > > > > >> > >> partition-reassignment /
>> preferred-leader-election
>> >>>> as
>> >>>> > >> > > separate
>> >>>> > >> > > > > > >> request
>> >>>> > >> > > > > > >> > >> types, I also left some other comments on the
>> RB (
>> >>>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
>> >>>> > >> > > > > > >> > >>
>> >>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
>> >>>> > >> > > > jay.kreps@gmail.com>
>> >>>> > >> > > > > > >> wrote:
>> >>>> > >> > > > > > >> > >>
>> >>>> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to
>> just
>> >>>> have
>> >>>> > >> one
>> >>>> > >> > > "do
>> >>>> > >> > > > > > admin
>> >>>> > >> > > > > > >> > >> stuff"
>> >>>> > >> > > > > > >> > >> > command that has the union of all
>> parameters.
>> >>>> > >> > > > > > >> > >> >
>> >>>> > >> > > > > > >> > >> > What I am saying is that command line tools
>> are
>> >>>> one
>> >>>> > >> > client
>> >>>> > >> > > of
>> >>>> > >> > > > > the
>> >>>> > >> > > > > > >> > >> > administrative apis, but these will be used
>> in a
>> >>>> number
>> >>>> > >> > of
>> >>>> > >> > > > > > >> scenarios
>> >>>> > >> > > > > > >> > so
>> >>>> > >> > > > > > >> > >> > they should make logical sense even in the
>> >>>> absence of
>> >>>> > >> the
>> >>>> > >> > > > > command
>> >>>> > >> > > > > > >> line
>> >>>> > >> > > > > > >> > >> > tool. Hence comments like trying to clarify
>> the
>> >>>> > >> > > relationship
>> >>>> > >> > > > > > >> between
>> >>>> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these
>> kinds
>> >>>> of
>> >>>> > >> things
>> >>>> > >> > > > > really
>> >>>> > >> > > > > > >> need
>> >>>> > >> > > > > > >> > >> to be
>> >>>> > >> > > > > > >> > >> > thought through.
>> >>>> > >> > > > > > >> > >> >
>> >>>> > >> > > > > > >> > >> > Hope that makes sense.
>> >>>> > >> > > > > > >> > >> >
>> >>>> > >> > > > > > >> > >> > -Jay
>> >>>> > >> > > > > > >> > >> >
>> >>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
>> >>>> Biletskyi <
>> >>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
>> >>>> > >> > > > > > >> > >> >
>> >>>> > >> > > > > > >> > >> > > Jay,
>> >>>> > >> > > > > > >> > >> > >
>> >>>> > >> > > > > > >> > >> > > Thanks for answering. You understood
>> >>>> correctly, most
>> >>>> > >> of
>> >>>> > >> > > my
>> >>>> > >> > > > > > >> comments
>> >>>> > >> > > > > > >> > >> were
>> >>>> > >> > > > > > >> > >> > > related to your point 1) - about "well
>> >>>> thought-out"
>> >>>> > >> > apis.
>> >>>> > >> > > > > Also,
>> >>>> > >> > > > > > >> yes,
>> >>>> > >> > > > > > >> > >> as I
>> >>>> > >> > > > > > >> > >> > > understood we would like to introduce a
>> single
>> >>>> > >> unified
>> >>>> > >> > > CLI
>> >>>> > >> > > > > tool
>> >>>> > >> > > > > > >> with
>> >>>> > >> > > > > > >> > >> > > centralized server-side request handling
>> for
>> >>>> lots of
>> >>>> > >> > > > existing
>> >>>> > >> > > > > > >> ones
>> >>>> > >> > > > > > >> > >> (incl.
>> >>>> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
>> >>>> > >> ReassignPartitions,
>> >>>> > >> > > smth
>> >>>> > >> > > > > > else
>> >>>> > >> > > > > > >> if
>> >>>> > >> > > > > > >> > >> added
>> >>>> > >> > > > > > >> > >> > > in future). In our previous discussion (
>> >>>> > >> > > > > > >> > >> > >
>> >>>> https://issues.apache.org/jira/browse/KAFKA-1694)
>> >>>> > >> > people
>> >>>> > >> > > > > said
>> >>>> > >> > > > > > >> > they'd
>> >>>> > >> > > > > > >> > >> > > rather
>> >>>> > >> > > > > > >> > >> > > have a separate message for each command,
>> so,
>> >>>> yes,
>> >>>> > >> this
>> >>>> > >> > > > way I
>> >>>> > >> > > > > > >> came
>> >>>> > >> > > > > > >> > to
>> >>>> > >> > > > > > >> > >> 1-1
>> >>>> > >> > > > > > >> > >> > > mapping between commands in the tool and
>> >>>> protocol
>> >>>> > >> > > > additions.
>> >>>> > >> > > > > > But
>> >>>> > >> > > > > > >> I
>> >>>> > >> > > > > > >> > >> might
>> >>>> > >> > > > > > >> > >> > be
>> >>>> > >> > > > > > >> > >> > > wrong.
>> >>>> > >> > > > > > >> > >> > > At the end I just try to start discussion
>> how
>> >>>> at
>> >>>> > >> least
>> >>>> > >> > > > > > generally
>> >>>> > >> > > > > > >> > this
>> >>>> > >> > > > > > >> > >> > > protocol should look like.
>> >>>> > >> > > > > > >> > >> > >
>> >>>> > >> > > > > > >> > >> > > Thanks,
>> >>>> > >> > > > > > >> > >> > > Andrii
>> >>>> > >> > > > > > >> > >> > >
>> >>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay
>> Kreps <
>> >>>> > >> > > > > > jay.kreps@gmail.com
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >> > >> wrote:
>> >>>> > >> > > > > > >> > >> > >
>> >>>> > >> > > > > > >> > >> > > > Hey Andrii,
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> > >> > > > > > >> > >> > > > To answer your earlier question we just
>> >>>> really
>> >>>> > >> can't
>> >>>> > >> > be
>> >>>> > >> > > > > > adding
>> >>>> > >> > > > > > >> any
>> >>>> > >> > > > > > >> > >> more
>> >>>> > >> > > > > > >> > >> > > > scala protocol objects. These things are
>> >>>> super hard
>> >>>> > >> > to
>> >>>> > >> > > > > > maintain
>> >>>> > >> > > > > > >> > >> because
>> >>>> > >> > > > > > >> > >> > > > they hand code the byte parsing and
>> don't
>> >>>> have good
>> >>>> > >> > > > > > versioning
>> >>>> > >> > > > > > >> > >> support.
>> >>>> > >> > > > > > >> > >> > > > Since we are already planning on
>> converting
>> >>>> we
>> >>>> > >> > > definitely
>> >>>> > >> > > > > > don't
>> >>>> > >> > > > > > >> > >> want to
>> >>>> > >> > > > > > >> > >> > > add
>> >>>> > >> > > > > > >> > >> > > > a ton more of these--they are total tech
>> >>>> debt.
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> > >> > > > > > >> > >> > > > What does it mean that the changes are
>> >>>> isolated
>> >>>> > >> from
>> >>>> > >> > > the
>> >>>> > >> > > > > > >> current
>> >>>> > >> > > > > > >> > >> code
>> >>>> > >> > > > > > >> > >> > > base?
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> > >> > > > > > >> > >> > > > I actually didn't understand the
>> remaining
>> >>>> > >> comments,
>> >>>> > >> > > > which
>> >>>> > >> > > > > of
>> >>>> > >> > > > > > >> the
>> >>>> > >> > > > > > >> > >> > points
>> >>>> > >> > > > > > >> > >> > > > are you responding to?
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> > >> > > > > > >> > >> > > > Maybe one sticking point here is that it
>> >>>> seems like
>> >>>> > >> > you
>> >>>> > >> > > > > want
>> >>>> > >> > > > > > to
>> >>>> > >> > > > > > >> > make
>> >>>> > >> > > > > > >> > >> > some
>> >>>> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1
>> mapping
>> >>>> > >> between
>> >>>> > >> > > > > > commands
>> >>>> > >> > > > > > >> you
>> >>>> > >> > > > > > >> > >> > > imagine
>> >>>> > >> > > > > > >> > >> > > > in the tool and protocol additions. I
>> want
>> >>>> to make
>> >>>> > >> > sure
>> >>>> > >> > > > we
>> >>>> > >> > > > > > >> don't
>> >>>> > >> > > > > > >> > do
>> >>>> > >> > > > > > >> > >> > that.
>> >>>> > >> > > > > > >> > >> > > > The protocol needs to be really really
>> well
>> >>>> thought
>> >>>> > >> > out
>> >>>> > >> > > > > > against
>> >>>> > >> > > > > > >> > many
>> >>>> > >> > > > > > >> > >> > use
>> >>>> > >> > > > > > >> > >> > > > cases so it should make perfect logical
>> >>>> sense in
>> >>>> > >> the
>> >>>> > >> > > > > absence
>> >>>> > >> > > > > > of
>> >>>> > >> > > > > > >> > >> knowing
>> >>>> > >> > > > > > >> > >> > > the
>> >>>> > >> > > > > > >> > >> > > > command line tool, right?
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> > >> > > > > > >> > >> > > > -Jay
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii
>> >>>> Biletskyi
>> >>>> > >> <
>> >>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> > >> > > > > > >> > >> > > > > Hey Jay,
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > > > I would like to continue this
>> discussion
>> >>>> as it
>> >>>> > >> seem
>> >>>> > >> > > > there
>> >>>> > >> > > > > > is
>> >>>> > >> > > > > > >> no
>> >>>> > >> > > > > > >> > >> > > progress
>> >>>> > >> > > > > > >> > >> > > > > here.
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > > > First of all, could you please explain
>> >>>> what did
>> >>>> > >> you
>> >>>> > >> > > > mean
>> >>>> > >> > > > > in
>> >>>> > >> > > > > > >> 2?
>> >>>> > >> > > > > > >> > How
>> >>>> > >> > > > > > >> > >> > > > exactly
>> >>>> > >> > > > > > >> > >> > > > > are we going to migrate to the new
>> java
>> >>>> protocol
>> >>>> > >> > > > > > definitions.
>> >>>> > >> > > > > > >> > And
>> >>>> > >> > > > > > >> > >> why
>> >>>> > >> > > > > > >> > >> > > > it's
>> >>>> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > > > I agree with you, this feature
>> includes
>> >>>> lots of
>> >>>> > >> > > stuff,
>> >>>> > >> > > > > but
>> >>>> > >> > > > > > >> > >> thankfully
>> >>>> > >> > > > > > >> > >> > > > > almost all changes are isolated from
>> the
>> >>>> current
>> >>>> > >> > code
>> >>>> > >> > > > > base,
>> >>>> > >> > > > > > >> > >> > > > > so the main thing, I think, we need to
>> >>>> agree is
>> >>>> > >> > RQ/RP
>> >>>> > >> > > > > > format.
>> >>>> > >> > > > > > >> > >> > > > > So how can we start discussion about
>> the
>> >>>> concrete
>> >>>> > >> > > > > messages
>> >>>> > >> > > > > > >> > format?
>> >>>> > >> > > > > > >> > >> > > > > Can we take (
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> > >> > > > > > >> > >> > >
>> >>>> > >> > > > > > >> > >> >
>> >>>> > >> > > > > > >> > >>
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >>
>> >>>> > >> > > > > >
>> >>>> > >> > > > >
>> >>>> > >> > > >
>> >>>> > >> > >
>> >>>> > >> >
>> >>>> > >>
>> >>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
>> >>>> > >> > > > > > >> > >> > > > > )
>> >>>> > >> > > > > > >> > >> > > > > as starting point?
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > > > We had some doubts earlier whether it
>> worth
>> >>>> > >> > > introducing
>> >>>> > >> > > > > one
>> >>>> > >> > > > > > >> > >> generic
>> >>>> > >> > > > > > >> > >> > > Admin
>> >>>> > >> > > > > > >> > >> > > > > Request for all commands (
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> https://issues.apache.org/jira/browse/KAFKA-1694
>> >>>> > >> > > > > > >> > >> > > > > )
>> >>>> > >> > > > > > >> > >> > > > > but then everybody agreed it would be
>> >>>> better to
>> >>>> > >> > have
>> >>>> > >> > > > > > separate
>> >>>> > >> > > > > > >> > >> message
>> >>>> > >> > > > > > >> > >> > > for
>> >>>> > >> > > > > > >> > >> > > > > each admin command. The Request part
>> is
>> >>>> really
>> >>>> > >> > > dictated
>> >>>> > >> > > > > > from
>> >>>> > >> > > > > > >> the
>> >>>> > >> > > > > > >> > >> > > command
>> >>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself,
>> so
>> >>>> the
>> >>>> > >> > proposed
>> >>>> > >> > > > > > version
>> >>>> > >> > > > > > >> > >> should
>> >>>> > >> > > > > > >> > >> > be
>> >>>> > >> > > > > > >> > >> > > > > fine (let's put aside for now remarks
>> about
>> >>>> > >> > Optional
>> >>>> > >> > > > > type,
>> >>>> > >> > > > > > >> > >> batching,
>> >>>> > >> > > > > > >> > >> > > > > configs normalization - I agree with
>> all of
>> >>>> > >> them).
>> >>>> > >> > > > > > >> > >> > > > > So the second part is Response. I see
>> >>>> there are
>> >>>> > >> two
>> >>>> > >> > > > cases
>> >>>> > >> > > > > > >> here.
>> >>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
>> Create/Alter/... ;
>> >>>> b)
>> >>>> > >> "Get"
>> >>>> > >> > > > > > requests -
>> >>>> > >> > > > > > >> > >> > > > > List/Describe...
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > > > a) should only hold request result
>> >>>> (regardless
>> >>>> > >> what
>> >>>> > >> > > we
>> >>>> > >> > > > > > decide
>> >>>> > >> > > > > > >> > >> about
>> >>>> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
>> execution).
>> >>>> > >> > > > > > >> > >> > > > > Usually we provide error code in
>> response
>> >>>> but
>> >>>> > >> since
>> >>>> > >> > > we
>> >>>> > >> > > > > will
>> >>>> > >> > > > > > >> use
>> >>>> > >> > > > > > >> > >> this
>> >>>> > >> > > > > > >> > >> > in
>> >>>> > >> > > > > > >> > >> > > > > interactive shell we need some human
>> >>>> readable
>> >>>> > >> error
>> >>>> > >> > > > > > >> description
>> >>>> > >> > > > > > >> > -
>> >>>> > >> > > > > > >> > >> so
>> >>>> > >> > > > > > >> > >> > I
>> >>>> > >> > > > > > >> > >> > > > > added errorDesription field where you
>> can
>> >>>> at
>> >>>> > >> least
>> >>>> > >> > > > leave
>> >>>> > >> > > > > > >> > >> > > > > exception.getMessage.
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > > > b) in addition to previous item
>> message
>> >>>> should
>> >>>> > >> hold
>> >>>> > >> > > > > command
>> >>>> > >> > > > > > >> > >> specific
>> >>>> > >> > > > > > >> > >> > > > > response data. We can discuss in
>> detail
>> >>>> each of
>> >>>> > >> > them
>> >>>> > >> > > > but
>> >>>> > >> > > > > > >> let's
>> >>>> > >> > > > > > >> > for
>> >>>> > >> > > > > > >> > >> > now
>> >>>> > >> > > > > > >> > >> > > > > agree about the overall pattern.
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > > > Thanks,
>> >>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay
>> Kreps
>> >>>> <
>> >>>> > >> > > > > > >> jay.kreps@gmail.com
>> >>>> > >> > > > > > >> > >
>> >>>> > >> > > > > > >> > >> > > wrote:
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > > > > Hey Joe,
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > This is great. A few comments on
>> KIP-4
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > 1. This is much needed
>> functionality,
>> >>>> but there
>> >>>> > >> > > are a
>> >>>> > >> > > > > lot
>> >>>> > >> > > > > > >> of
>> >>>> > >> > > > > > >> > >> the so
>> >>>> > >> > > > > > >> > >> > > > let's
>> >>>> > >> > > > > > >> > >> > > > > > really think these protocols
>> through. We
>> >>>> really
>> >>>> > >> > > want
>> >>>> > >> > > > to
>> >>>> > >> > > > > > >> end up
>> >>>> > >> > > > > > >> > >> > with a
>> >>>> > >> > > > > > >> > >> > > > set
>> >>>> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol
>> apis.
>> >>>> For this
>> >>>> > >> > > > reason I
>> >>>> > >> > > > > > >> think
>> >>>> > >> > > > > > >> > >> it is
>> >>>> > >> > > > > > >> > >> > > > > really
>> >>>> > >> > > > > > >> > >> > > > > > important to think through the end
>> state
>> >>>> even
>> >>>> > >> if
>> >>>> > >> > > that
>> >>>> > >> > > > > > >> includes
>> >>>> > >> > > > > > >> > >> APIs
>> >>>> > >> > > > > > >> > >> > > we
>> >>>> > >> > > > > > >> > >> > > > > > won't implement in the first phase.
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait
>> until
>> >>>> we
>> >>>> > >> have
>> >>>> > >> > > > > switched
>> >>>> > >> > > > > > >> the
>> >>>> > >> > > > > > >> > >> > server
>> >>>> > >> > > > > > >> > >> > > > over
>> >>>> > >> > > > > > >> > >> > > > > > to the new java protocol
>> definitions. If
>> >>>> we add
>> >>>> > >> > > > upteen
>> >>>> > >> > > > > > >> more ad
>> >>>> > >> > > > > > >> > >> hoc
>> >>>> > >> > > > > > >> > >> > > > scala
>> >>>> > >> > > > > > >> > >> > > > > > objects that is just generating more
>> >>>> work for
>> >>>> > >> the
>> >>>> > >> > > > > > >> conversion
>> >>>> > >> > > > > > >> > we
>> >>>> > >> > > > > > >> > >> > know
>> >>>> > >> > > > > > >> > >> > > we
>> >>>> > >> > > > > > >> > >> > > > > > have to do.
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new
>> type of
>> >>>> > >> > optional
>> >>>> > >> > > > > > >> parameter.
>> >>>> > >> > > > > > >> > >> This
>> >>>> > >> > > > > > >> > >> > is
>> >>>> > >> > > > > > >> > >> > > > > > inconsistent with everything else
>> in the
>> >>>> > >> protocol
>> >>>> > >> > > > where
>> >>>> > >> > > > > > we
>> >>>> > >> > > > > > >> use
>> >>>> > >> > > > > > >> > >> -1
>> >>>> > >> > > > > > >> > >> > or
>> >>>> > >> > > > > > >> > >> > > > some
>> >>>> > >> > > > > > >> > >> > > > > > other marker value. You could argue
>> >>>> either way
>> >>>> > >> > but
>> >>>> > >> > > > > let's
>> >>>> > >> > > > > > >> stick
>> >>>> > >> > > > > > >> > >> with
>> >>>> > >> > > > > > >> > >> > > > that
>> >>>> > >> > > > > > >> > >> > > > > > for consistency. For clients that
>> >>>> implemented
>> >>>> > >> the
>> >>>> > >> > > > > > protocol
>> >>>> > >> > > > > > >> in
>> >>>> > >> > > > > > >> > a
>> >>>> > >> > > > > > >> > >> > > better
>> >>>> > >> > > > > > >> > >> > > > > way
>> >>>> > >> > > > > > >> > >> > > > > > than our scala code these basic
>> >>>> primitives are
>> >>>> > >> > hard
>> >>>> > >> > > > to
>> >>>> > >> > > > > > >> change.
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
>> >>>> duplicate
>> >>>> > >> > > > > > >> > TopicMetadataRequest
>> >>>> > >> > > > > > >> > >> > > which
>> >>>> > >> > > > > > >> > >> > > > > has
>> >>>> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I
>> think
>> >>>> we
>> >>>> > >> > should
>> >>>> > >> > > > > rename
>> >>>> > >> > > > > > >> that
>> >>>> > >> > > > > > >> > >> > > request
>> >>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
>> >>>> > >> MetadataRequest)
>> >>>> > >> > > and
>> >>>> > >> > > > > > >> include
>> >>>> > >> > > > > > >> > >> the id
>> >>>> > >> > > > > > >> > >> > > of
>> >>>> > >> > > > > > >> > >> > > > > the
>> >>>> > >> > > > > > >> > >> > > > > > controller. Or are there other
>> things we
>> >>>> could
>> >>>> > >> > add
>> >>>> > >> > > > > here?
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to
>> make a
>> >>>> lot of
>> >>>> > >> > > > requests
>> >>>> > >> > > > > > that
>> >>>> > >> > > > > > >> > can
>> >>>> > >> > > > > > >> > >> > only
>> >>>> > >> > > > > > >> > >> > > go
>> >>>> > >> > > > > > >> > >> > > > > to
>> >>>> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of
>> >>>> burden for
>> >>>> > >> > > > client
>> >>>> > >> > > > > > >> > >> > > implementations
>> >>>> > >> > > > > > >> > >> > > > > (it
>> >>>> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can
>> fail
>> >>>> in many
>> >>>> > >> > > parts
>> >>>> > >> > > > > so
>> >>>> > >> > > > > > it
>> >>>> > >> > > > > > >> > >> ends up
>> >>>> > >> > > > > > >> > >> > > > > being a
>> >>>> > >> > > > > > >> > >> > > > > > full state machine to do right). I
>> think
>> >>>> we
>> >>>> > >> > should
>> >>>> > >> > > > > > consider
>> >>>> > >> > > > > > >> > >> making
>> >>>> > >> > > > > > >> > >> > > > admin
>> >>>> > >> > > > > > >> > >> > > > > > commands and ideally as many of the
>> >>>> other apis
>> >>>> > >> as
>> >>>> > >> > > > > > possible
>> >>>> > >> > > > > > >> > >> > available
>> >>>> > >> > > > > > >> > >> > > on
>> >>>> > >> > > > > > >> > >> > > > > all
>> >>>> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
>> >>>> controller on
>> >>>> > >> > the
>> >>>> > >> > > > > broker
>> >>>> > >> > > > > > >> > side.
>> >>>> > >> > > > > > >> > >> > > Perhaps
>> >>>> > >> > > > > > >> > >> > > > > > there would be a general way to
>> >>>> encapsulate
>> >>>> > >> this
>> >>>> > >> > > > > > re-routing
>> >>>> > >> > > > > > >> > >> > behavior.
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > 6. We should probably normalize the
>> key
>> >>>> value
>> >>>> > >> > pairs
>> >>>> > >> > > > > used
>> >>>> > >> > > > > > >> for
>> >>>> > >> > > > > > >> > >> > configs
>> >>>> > >> > > > > > >> > >> > > > > rather
>> >>>> > >> > > > > > >> > >> > > > > > than embedding a new formatting. So
>> two
>> >>>> strings
>> >>>> > >> > > > rather
>> >>>> > >> > > > > > than
>> >>>> > >> > > > > > >> > one
>> >>>> > >> > > > > > >> > >> > with
>> >>>> > >> > > > > > >> > >> > > an
>> >>>> > >> > > > > > >> > >> > > > > > internal equals sign.
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these
>> APIs
>> >>>> that the
>> >>>> > >> > > > command
>> >>>> > >> > > > > > has
>> >>>> > >> > > > > > >> > >> begun or
>> >>>> > >> > > > > > >> > >> > > > that
>> >>>> > >> > > > > > >> > >> > > > > > the command has been completed? It
>> is a
>> >>>> lot
>> >>>> > >> more
>> >>>> > >> > > > usable
>> >>>> > >> > > > > > if
>> >>>> > >> > > > > > >> the
>> >>>> > >> > > > > > >> > >> > > command
>> >>>> > >> > > > > > >> > >> > > > > has
>> >>>> > >> > > > > > >> > >> > > > > > been completed so you know that if
>> you
>> >>>> create a
>> >>>> > >> > > topic
>> >>>> > >> > > > > and
>> >>>> > >> > > > > > >> then
>> >>>> > >> > > > > > >> > >> > > publish
>> >>>> > >> > > > > > >> > >> > > > to
>> >>>> > >> > > > > > >> > >> > > > > > it you won't get an exception about
>> >>>> there being
>> >>>> > >> > no
>> >>>> > >> > > > such
>> >>>> > >> > > > > > >> topic.
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
>> >>>> duplicate a
>> >>>> > >> lot
>> >>>> > >> > > of
>> >>>> > >> > > > > > stuff
>> >>>> > >> > > > > > >> in
>> >>>> > >> > > > > > >> > >> the
>> >>>> > >> > > > > > >> > >> > > > > metadata
>> >>>> > >> > > > > > >> > >> > > > > > request. Is there a reason to give
>> back
>> >>>> topics
>> >>>> > >> > > marked
>> >>>> > >> > > > > for
>> >>>> > >> > > > > > >> > >> > deletion? I
>> >>>> > >> > > > > > >> > >> > > > > feel
>> >>>> > >> > > > > > >> > >> > > > > > like if we just make the
>> post-condition
>> >>>> of the
>> >>>> > >> > > delete
>> >>>> > >> > > > > > >> command
>> >>>> > >> > > > > > >> > be
>> >>>> > >> > > > > > >> > >> > that
>> >>>> > >> > > > > > >> > >> > > > the
>> >>>> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid
>> of
>> >>>> the need
>> >>>> > >> > for
>> >>>> > >> > > > this
>> >>>> > >> > > > > > >> right?
>> >>>> > >> > > > > > >> > >> And
>> >>>> > >> > > > > > >> > >> > it
>> >>>> > >> > > > > > >> > >> > > > > will
>> >>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > 9. Should we consider batching these
>> >>>> requests?
>> >>>> > >> We
>> >>>> > >> > > > have
>> >>>> > >> > > > > > >> > generally
>> >>>> > >> > > > > > >> > >> > > tried
>> >>>> > >> > > > > > >> > >> > > > to
>> >>>> > >> > > > > > >> > >> > > > > > allow multiple operations to be
>> batched.
>> >>>> My
>> >>>> > >> > > suspicion
>> >>>> > >> > > > > is
>> >>>> > >> > > > > > >> that
>> >>>> > >> > > > > > >> > >> > without
>> >>>> > >> > > > > > >> > >> > > > > this
>> >>>> > >> > > > > > >> > >> > > > > > we will get a lot of code that does
>> >>>> something
>> >>>> > >> > like
>> >>>> > >> > > > > > >> > >> > > > > >    for(topic:
>> adminClient.listTopics())
>> >>>> > >> > > > > > >> > >> > > > > >
>>  adminClient.describeTopic(topic)
>> >>>> > >> > > > > > >> > >> > > > > > this code will work great when you
>> test
>> >>>> on 5
>> >>>> > >> > topics
>> >>>> > >> > > > but
>> >>>> > >> > > > > > >> not do
>> >>>> > >> > > > > > >> > >> as
>> >>>> > >> > > > > > >> > >> > > well
>> >>>> > >> > > > > > >> > >> > > > if
>> >>>> > >> > > > > > >> > >> > > > > > you have 50k.
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss
>> how
>> >>>> we want
>> >>>> > >> to
>> >>>> > >> > > > > expose
>> >>>> > >> > > > > > a
>> >>>> > >> > > > > > >> > >> > > programmatic
>> >>>> > >> > > > > > >> > >> > > > > JVM
>> >>>> > >> > > > > > >> > >> > > > > > client api for these operations.
>> >>>> Currently
>> >>>> > >> people
>> >>>> > >> > > > rely
>> >>>> > >> > > > > on
>> >>>> > >> > > > > > >> > >> > AdminUtils
>> >>>> > >> > > > > > >> > >> > > > > which
>> >>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we
>> probably
>> >>>> need
>> >>>> > >> > > another
>> >>>> > >> > > > > > client
>> >>>> > >> > > > > > >> > >> under
>> >>>> > >> > > > > > >> > >> > > > > clients/
>> >>>> > >> > > > > > >> > >> > > > > > that exposes administrative
>> >>>> functionality. We
>> >>>> > >> > will
>> >>>> > >> > > > need
>> >>>> > >> > > > > > >> this
>> >>>> > >> > > > > > >> > >> just
>> >>>> > >> > > > > > >> > >> > to
>> >>>> > >> > > > > > >> > >> > > > > > properly test the new apis, I
>> suspect. We
>> >>>> > >> should
>> >>>> > >> > > > figure
>> >>>> > >> > > > > > out
>> >>>> > >> > > > > > >> > that
>> >>>> > >> > > > > > >> > >> > API.
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > 11. The other information that
>> would be
>> >>>> really
>> >>>> > >> > > useful
>> >>>> > >> > > > > to
>> >>>> > >> > > > > > >> get
>> >>>> > >> > > > > > >> > >> would
>> >>>> > >> > > > > > >> > >> > be
>> >>>> > >> > > > > > >> > >> > > > > > information about partitions--how
>> much
>> >>>> data is
>> >>>> > >> in
>> >>>> > >> > > the
>> >>>> > >> > > > > > >> > partition,
>> >>>> > >> > > > > > >> > >> > what
>> >>>> > >> > > > > > >> > >> > > > are
>> >>>> > >> > > > > > >> > >> > > > > > the segment offsets, what is the
>> log-end
>> >>>> offset
>> >>>> > >> > > (i.e.
>> >>>> > >> > > > > > last
>> >>>> > >> > > > > > >> > >> offset),
>> >>>> > >> > > > > > >> > >> > > > what
>> >>>> > >> > > > > > >> > >> > > > > is
>> >>>> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think
>> that
>> >>>> done
>> >>>> > >> > right
>> >>>> > >> > > > this
>> >>>> > >> > > > > > >> would
>> >>>> > >> > > > > > >> > be
>> >>>> > >> > > > > > >> > >> > the
>> >>>> > >> > > > > > >> > >> > > > > > successor to the very awkward
>> >>>> OffsetRequest we
>> >>>> > >> > have
>> >>>> > >> > > > > > today.
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > -Jay
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM,
>> Joe
>> >>>> Stein <
>> >>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
>> >>>> > >> > > > > > >> > >> > > > > wrote:
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
>> >>>> > >> > > > > > >> > >> > > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > >
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> > >> > > > > > >> > >> > >
>> >>>> > >> > > > > > >> > >> >
>> >>>> > >> > > > > > >> > >>
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >>
>> >>>> > >> > > > > >
>> >>>> > >> > > > >
>> >>>> > >> > > >
>> >>>> > >> > >
>> >>>> > >> >
>> >>>> > >>
>> >>>>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>> >>>> > >> > > > > > >> > >> > > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > > JIRA
>> >>>> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
>> >>>> > >> > > > > > >> > >> > > > > > >
>> >>>> > >> > > > > > >> > >> > > > > > >
>> >>>> /*******************************************
>> >>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
>> >>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
>> >>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
>> >>>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
>> >>>> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
>> >>>> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> > >> > > > > > >> > >> > > > > > >
>> >>>> ********************************************/
>> >>>> > >> > > > > > >> > >> > > > > > >
>> >>>> > >> > > > > > >> > >> > > > > >
>> >>>> > >> > > > > > >> > >> > > > >
>> >>>> > >> > > > > > >> > >> > > >
>> >>>> > >> > > > > > >> > >> > >
>> >>>> > >> > > > > > >> > >> >
>> >>>> > >> > > > > > >> > >>
>> >>>> > >> > > > > > >> > >>
>> >>>> > >> > > > > > >> > >>
>> >>>> > >> > > > > > >> > >> --
>> >>>> > >> > > > > > >> > >> -- Guozhang
>> >>>> > >> > > > > > >> > >>
>> >>>> > >> > > > > > >> > >
>> >>>> > >> > > > > > >> > >
>> >>>> > >> > > > > > >> >
>> >>>> > >> > > > > > >>
>> >>>> > >> > > > > > >
>> >>>> > >> > > > > > >
>> >>>> > >> > > > > >
>> >>>> > >> > > > >
>> >>>> > >> > > >
>> >>>> > >> > >
>> >>>> > >> >
>> >>>> > >> >
>> >>>> > >> >
>> >>>> > >> > --
>> >>>> > >> > Jeff Holoman
>> >>>> > >> > Systems Engineer
>> >>>> > >> >
>> >>>> > >>
>> >>>>
>> >>>>
>> >>>
>> >>>
>> >>> --
>> >>> -- Guozhang
>> >>>
>> >>
>> >>
>> >>
>> >> --
>> >> -- Guozhang
>>
>
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Hi all,

Today I uploaded the patch that covers some of the discussed and agreed
items:
- removed MaybeOf optional type
- switched to java protocol definitions
- simplified messages (normalized configs, removed topic marked for
deletion)

I also updated the KIP-4 with respective changes and wrote down my proposal
for
pending items:
- Batch Admin Operations -> updated Wire Protocol schema proposal
- Remove ClusterMetadata -> changed to extend TopicMetadataRequest
- Admin Client -> updated my initial proposal to reflect batching
- Error codes -> proposed fine-grained error code instead of
AdminRequestFailed

I will also send a separate email to cover all comments from this thread.

Thanks,
Andrii Biletskyi


On Thu, Mar 12, 2015 at 9:26 PM, Gwen Shapira <gs...@cloudera.com> wrote:

> Found KIP-11 (
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> )
> It actually specifies changes to the Metadata protocol, so making sure
> both KIPs are consistent in this regard will be good.
>
> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <gs...@cloudera.com>
> wrote:
> > Specifically for ownership, I think the plan is to add ACL (it sounds
> > like you are describing ACL) via an external system (Argus, Sentry).
> > I remember KIP-11 described this, but I can't find the KIP any longer.
> >
> > Regardless, I think KIP-4 focuses on getting information that already
> > exists from Kafka brokers, not on adding information that perhaps
> > should exist but doesn't yet?
> >
> > Gwen
> >
> >
> >
> >
> >
> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >> Folks,
> >>
> >> Just want to elaborate a bit more on the create-topic metadata and
> batching
> >> describe-topic based on config / metadata in my previous email as we
> work
> >> on KAFKA-1694. The main motivation is to have some sort of topic
> management
> >> mechanisms, which I think is quite important in a multi-tenant / cloud
> >> architecture: today anyone can create topics in a shared Kafka cluster,
> but
> >> there is no concept or "ownership" of topics that are created by
> different
> >> users. For example, at LinkedIn we basically distinguish topic owners
> via
> >> some casual topic name prefix, which is a bit awkward and does not fly
> as
> >> we scale our customers. It would be great to use describe-topics such
> as:
> >>
> >> Describe all topics that is created by me.
> >>
> >> Describe all topics whose retention time is overriden to X.
> >>
> >> Describe all topics whose writable group include user Y (this is
> related to
> >> authorization), etc..
> >>
> >> One possible way to achieve this is to add a metadata file in the
> >> create-topic request, whose value will also be written ZK as we create
> the
> >> topic; then describe-topics can choose to batch topics based on 1) name
> >> regex, 2) config K-V matching, 3) metadata regex, etc.
> >>
> >> Thoughts?
> >>
> >> Guozhang
> >>
> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >>
> >>> Thanks for the updated wiki. A few comments below:
> >>>
> >>> 1. Error description in response: I think if some errorCode could
> indicate
> >>> several different error cases then we should really change it to
> multiple
> >>> codes. In general the errorCode itself would be precise and sufficient
> for
> >>> describing the server side errors.
> >>>
> >>> 2. Describe topic request: it would be great to go beyond just
> batching on
> >>> topic name regex for this request. For example, a very common use case
> of
> >>> the topic command is to list all topics whose config A's value is B.
> With
> >>> topic name regex then we have to first retrieve __all__ topics's
> >>> description info and then filter at the client end, which will be a
> huge
> >>> burden on ZK.
> >>>
> >>> 3. Config K-Vs in create topic: this is related to the previous point;
> >>> maybe we can add another metadata K-V or just a metadata string along
> side
> >>> with config K-V in create topic like we did for offset commit request.
> This
> >>> field can be quite useful in storing information like "owner" of the
> topic
> >>> who issue the create command, etc, which is quite important for a
> >>> multi-tenant setting. Then in the describe topic request we can also
> batch
> >>> on regex of the metadata field.
> >>>
> >>> 4. Today all the admin operations are async in the sense that command
> will
> >>> return once it is written in ZK, and that is why we need extra
> verification
> >>> like testUtil.waitForTopicCreated() / verify partition reassignment
> >>> request, etc. With admin requests we could add a flag to enable /
> disable
> >>> synchronous requests; when it is turned on, the response will not
> return
> >>> until the request has been completed. And for async requests we can
> add a
> >>> "token" field in the response, and then only need a general "admin
> >>> verification request" with the given token to check if the async
> request
> >>> has been completed.
> >>>
> >>> 5. +1 for extending Metadata request to include controller /
> coordinator
> >>> information, and then we can remove the ConsumerMetadata /
> ClusterMetadata
> >>> requests.
> >>>
> >>> Guozhang
> >>>
> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com>
> wrote:
> >>>
> >>>> Thanks for sending that out Joe - I don't think I will be able to make
> >>>> it today, so if notes can be sent out afterward that would be great.
> >>>>
> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> >>>> > Thanks for sending this out Joe. Looking forward to chatting with
> >>>> everyone :)
> >>>> >
> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly>
> wrote:
> >>>> > > Hey, I just sent out a google hangout invite to all pmc,
> committers
> >>>> and
> >>>> > > everyone I found working on a KIP. If I missed anyone in the
> invite
> >>>> please
> >>>> > > let me know and can update it, np.
> >>>> > >
> >>>> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can
> get
> >>>> INFRA
> >>>> > > help to make a google account so we can manage better?
> >>>> > >
> >>>> > > To discuss
> >>>> > >
> >>>>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >>>> > > in progress and related JIRA that are interdependent and common
> work.
> >>>> > >
> >>>> > > ~ Joe Stein
> >>>> > >
> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com>
> >>>> wrote:
> >>>> > >
> >>>> > >> Let's stay on Google hangouts that will also record and make the
> >>>> sessions
> >>>> > >> available on youtube.
> >>>> > >>
> >>>> > >> -Jay
> >>>> > >>
> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> >>>> jholoman@cloudera.com>
> >>>> > >> wrote:
> >>>> > >>
> >>>> > >> > Jay / Joe
> >>>> > >> >
> >>>> > >> > We're happy to send out a Webex for this purpose. We could
> record
> >>>> the
> >>>> > >> > sessions if there is interest and publish them out.
> >>>> > >> >
> >>>> > >> > Thanks
> >>>> > >> >
> >>>> > >> > Jeff
> >>>> > >> >
> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> jay.kreps@gmail.com>
> >>>> wrote:
> >>>> > >> >
> >>>> > >> > > Let's try to get the technical hang-ups sorted out, though. I
> >>>> really
> >>>> > >> > think
> >>>> > >> > > there is some benefit to live discussion vs writing. I am
> >>>> hopeful that
> >>>> > >> if
> >>>> > >> > > we post instructions and give ourselves a few attempts we can
> >>>> get it
> >>>> > >> > > working.
> >>>> > >> > >
> >>>> > >> > > Tuesday at that time would work for me...any objections?
> >>>> > >> > >
> >>>> > >> > > -Jay
> >>>> > >> > >
> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> joe.stein@stealth.ly
> >>>> >
> >>>> > >> wrote:
> >>>> > >> > >
> >>>> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET /
> 10am
> >>>> PT
> >>>> > >> ????
> >>>> > >> > > >
> >>>> > >> > > > I don't mind google hangout but there is always some issue
> or
> >>>> > >> whatever
> >>>> > >> > so
> >>>> > >> > > > we know the apache irc channel works. We can start there
> and
> >>>> see how
> >>>> > >> it
> >>>> > >> > > > goes? We can pull transcripts too and associate to tickets
> if
> >>>> need be
> >>>> > >> > > makes
> >>>> > >> > > > it helpful for things.
> >>>> > >> > > >
> >>>> > >> > > > ~ Joestein
> >>>> > >> > > >
> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> >>>> jay.kreps@gmail.com>
> >>>> > >> > wrote:
> >>>> > >> > > >
> >>>> > >> > > > > We'd talked about doing a Google Hangout to chat about
> this.
> >>>> What
> >>>> > >> > about
> >>>> > >> > > > > generalizing that a little further...I actually think it
> >>>> would be
> >>>> > >> > good
> >>>> > >> > > > for
> >>>> > >> > > > > everyone spending a reasonable chunk of their week on
> Kafka
> >>>> stuff
> >>>> > >> to
> >>>> > >> > > > maybe
> >>>> > >> > > > > sync up once a week. I think we could use time to talk
> >>>> through
> >>>> > >> design
> >>>> > >> > > > > stuff, make sure we are on top of code reviews, talk
> through
> >>>> any
> >>>> > >> > tricky
> >>>> > >> > > > > issues, etc.
> >>>> > >> > > > >
> >>>> > >> > > > > We can make it publicly available so that any one can
> follow
> >>>> along
> >>>> > >> > who
> >>>> > >> > > > > likes.
> >>>> > >> > > > >
> >>>> > >> > > > > Any interest in doing this? If so I'll try to set it up
> >>>> starting
> >>>> > >> next
> >>>> > >> > > > week.
> >>>> > >> > > > >
> >>>> > >> > > > > -Jay
> >>>> > >> > > > >
> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > >
> >>>> > >> > > > > > Hi all,
> >>>> > >> > > > > >
> >>>> > >> > > > > > I've updated KIP page, fixed / aligned document
> structure.
> >>>> Also I
> >>>> > >> > > added
> >>>> > >> > > > > > some
> >>>> > >> > > > > > very initial proposal for AdminClient so we have
> something
> >>>> to
> >>>> > >> start
> >>>> > >> > > > from
> >>>> > >> > > > > > while
> >>>> > >> > > > > > discussing the KIP.
> >>>> > >> > > > > >
> >>>> > >> > > > > >
> >>>> > >> > > > > >
> >>>> > >> > > > >
> >>>> > >> > > >
> >>>> > >> > >
> >>>> > >> >
> >>>> > >>
> >>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >>>> > >> > > > > >
> >>>> > >> > > > > > Thanks,
> >>>> > >> > > > > > Andrii Biletskyi
> >>>> > >> > > > > >
> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > > >
> >>>> > >> > > > > > > Jay,
> >>>> > >> > > > > > >
> >>>> > >> > > > > > > Re error messages: you are right, in most cases
> client
> >>>> will
> >>>> > >> have
> >>>> > >> > > > enough
> >>>> > >> > > > > > > context to show descriptive error message. My
> concern is
> >>>> that
> >>>> > >> we
> >>>> > >> > > will
> >>>> > >> > > > > > have
> >>>> > >> > > > > > > to
> >>>> > >> > > > > > > add lots of new error codes for each possible error.
> Of
> >>>> course,
> >>>> > >> > we
> >>>> > >> > > > > could
> >>>> > >> > > > > > > reuse
> >>>> > >> > > > > > > some of existing like UknownTopicOrPartitionCode,
> but we
> >>>> will
> >>>> > >> > also
> >>>> > >> > > > need
> >>>> > >> > > > > > to
> >>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> >>>> TopicConfigInvalid (both
> >>>> > >> > for
> >>>> > >> > > > > topic
> >>>> > >> > > > > > > name and config, and probably user would like to know
> >>>> what
> >>>> > >> > exactly
> >>>> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
> >>>> > >> InternalError
> >>>> > >> > > > (e.g.
> >>>> > >> > > > > > > zookeeper failure) etc.
> >>>> > >> > > > > > > And this is only for TopicCommand, we will also need
> to
> >>>> add
> >>>> > >> > similar
> >>>> > >> > > > > stuff
> >>>> > >> > > > > > > for
> >>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up
> >>>> with a
> >>>> > >> > large
> >>>> > >> > > > list
> >>>> > >> > > > > > of
> >>>> > >> > > > > > > error codes, used only in Admin protocol.
> >>>> > >> > > > > > > Having said that, I agree my proposal is not
> consistent
> >>>> with
> >>>> > >> > other
> >>>> > >> > > > > cases.
> >>>> > >> > > > > > > Maybe we can find better solution or something
> >>>> in-between.
> >>>> > >> > > > > > >
> >>>> > >> > > > > > > Re Hangout chat: I think it is a great idea. This
> way we
> >>>> can
> >>>> > >> move
> >>>> > >> > > on
> >>>> > >> > > > > > > faster.
> >>>> > >> > > > > > > Let's agree somehow on date/time so people can join.
> >>>> Will work
> >>>> > >> > for
> >>>> > >> > > me
> >>>> > >> > > > > > this
> >>>> > >> > > > > > > and
> >>>> > >> > > > > > > next week almost anytime if agreed in advance.
> >>>> > >> > > > > > >
> >>>> > >> > > > > > > Thanks,
> >>>> > >> > > > > > > Andrii
> >>>> > >> > > > > > >
> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> >>>> > >> jay.kreps@gmail.com>
> >>>> > >> > > > > wrote:
> >>>> > >> > > > > > >
> >>>> > >> > > > > > >> Hey Andrii,
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >> Generally we can do good error handling without
> needing
> >>>> custom
> >>>> > >> > > > > > server-side
> >>>> > >> > > > > > >> messages. I.e. generally the client has the context
> to
> >>>> know
> >>>> > >> that
> >>>> > >> > > if
> >>>> > >> > > > it
> >>>> > >> > > > > > got
> >>>> > >> > > > > > >> an error that the topic doesn't exist to say "Topic
> X
> >>>> doesn't
> >>>> > >> > > exist"
> >>>> > >> > > > > > >> rather
> >>>> > >> > > > > > >> than "error code 14" (or whatever). Maybe there are
> >>>> specific
> >>>> > >> > cases
> >>>> > >> > > > > where
> >>>> > >> > > > > > >> this is hard? If we want to add server-side error
> >>>> messages we
> >>>> > >> > > really
> >>>> > >> > > > > do
> >>>> > >> > > > > > >> need to do this in a consistent way across the
> protocol.
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >> I still have a bunch of open questions here from my
> >>>> previous
> >>>> > >> > > list. I
> >>>> > >> > > > > > will
> >>>> > >> > > > > > >> be out for the next few days for Strata though.
> Maybe
> >>>> we could
> >>>> > >> > do
> >>>> > >> > > a
> >>>> > >> > > > > > Google
> >>>> > >> > > > > > >> Hangout chat on any open issues some time towards
> the
> >>>> end of
> >>>> > >> > next
> >>>> > >> > > > week
> >>>> > >> > > > > > for
> >>>> > >> > > > > > >> anyone interested in this ticket? I have a feeling
> that
> >>>> might
> >>>> > >> > > > progress
> >>>> > >> > > > > > >> things a little faster than email--I think we could
> talk
> >>>> > >> through
> >>>> > >> > > > those
> >>>> > >> > > > > > >> issues I brought up fairly quickly...
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >> -Jay
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >> > Hi all,
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > I'm trying to address some of the issues which
> were
> >>>> > >> mentioned
> >>>> > >> > > > > earlier
> >>>> > >> > > > > > >> about
> >>>> > >> > > > > > >> > Admin RQ/RP format. One of those was about
> batching
> >>>> > >> > operations.
> >>>> > >> > > > What
> >>>> > >> > > > > > if
> >>>> > >> > > > > > >> we
> >>>> > >> > > > > > >> > follow TopicCommand approach and let people
> specify
> >>>> > >> topic-name
> >>>> > >> > > by
> >>>> > >> > > > > > >> regexp -
> >>>> > >> > > > > > >> > would that cover most of the use cases?
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > Secondly, is what information should we generally
> >>>> provide in
> >>>> > >> > > Admin
> >>>> > >> > > > > > >> > responses.
> >>>> > >> > > > > > >> > I realize that Admin commands don't imply they
> will
> >>>> be used
> >>>> > >> > only
> >>>> > >> > > > in
> >>>> > >> > > > > > CLI
> >>>> > >> > > > > > >> > but,
> >>>> > >> > > > > > >> > it seems to me, CLI is a very important client of
> this
> >>>> > >> > feature.
> >>>> > >> > > In
> >>>> > >> > > > > > this
> >>>> > >> > > > > > >> > case,
> >>>> > >> > > > > > >> > seems logical, we would like to provide users with
> >>>> rich
> >>>> > >> > > experience
> >>>> > >> > > > > in
> >>>> > >> > > > > > >> terms
> >>>> > >> > > > > > >> > of
> >>>> > >> > > > > > >> > getting results / errors of the executed commands.
> >>>> Usually
> >>>> > >> we
> >>>> > >> > > > supply
> >>>> > >> > > > > > >> with
> >>>> > >> > > > > > >> > responses only errorCode, which looks very
> limiting,
> >>>> in case
> >>>> > >> > of
> >>>> > >> > > > CLI
> >>>> > >> > > > > we
> >>>> > >> > > > > > >> may
> >>>> > >> > > > > > >> > want to print human readable error description.
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > So, taking into account previous item about
> batching,
> >>>> what
> >>>> > >> do
> >>>> > >> > > you
> >>>> > >> > > > > > think
> >>>> > >> > > > > > >> > about
> >>>> > >> > > > > > >> > having smth like:
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > ('create' doesn't support regexp)
> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions
> Replicas
> >>>> > >> > > > > ReplicaAssignment
> >>>> > >> > > > > > >> > [Config]
> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
> >>>> > >> > > > > > >> >   ErrorCode => int16
> >>>> > >> > > > > > >> >   ErrorDescription => string (empty if successful)
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> >>>> > >> > > ReplicaAssignment
> >>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> >>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
> >>>> ErrorDescription]
> >>>> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
> >>>> > >> > > > > > >> >   CommandErrorCode => int16
> >>>> > >> > > > > > >> >   CommandErrorDescription => string (nonempty in
> case
> >>>> of
> >>>> > >> fatal
> >>>> > >> > > > > error,
> >>>> > >> > > > > > >> e.g.
> >>>> > >> > > > > > >> > we couldn't get topics by regexp)
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> >>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> TopicDescription
> >>>> > >> ErrorCode
> >>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> >>>> CommandErrorDescription
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > Also, any thoughts about our discussion regarding
> >>>> re-routing
> >>>> > >> > > > > facility?
> >>>> > >> > > > > > >> In
> >>>> > >> > > > > > >> > my
> >>>> > >> > > > > > >> > understanding, it is like between augmenting
> >>>> > >> > > TopicMetadataRequest
> >>>> > >> > > > > > >> > (to include at least controllerId) and
> implementing
> >>>> new
> >>>> > >> > generic
> >>>> > >> > > > > > >> re-routing
> >>>> > >> > > > > > >> > facility so sending messages to controller will be
> >>>> handled
> >>>> > >> by
> >>>> > >> > > it.
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > Thanks,
> >>>> > >> > > > > > >> > Andrii Biletskyi
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi
> <
> >>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > > @Guozhang:
> >>>> > >> > > > > > >> > > Thanks for your comments, I've answered some of
> >>>> those. The
> >>>> > >> > > main
> >>>> > >> > > > > > thing
> >>>> > >> > > > > > >> is
> >>>> > >> > > > > > >> > > having merged request for
> >>>> create-alter-delete-describe - I
> >>>> > >> > > have
> >>>> > >> > > > > some
> >>>> > >> > > > > > >> > > concerns about this approach.
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > > @*Jay*:
> >>>> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is
> also
> >>>> one of
> >>>> > >> > the
> >>>> > >> > > > > > >> concerns.
> >>>> > >> > > > > > >> > We
> >>>> > >> > > > > > >> > > can solve it if we implement re-routing
> facility.
> >>>> But I
> >>>> > >> > agree
> >>>> > >> > > > with
> >>>> > >> > > > > > >> > > Guozhang - it will make clients' internals a
> little
> >>>> bit
> >>>> > >> > easier
> >>>> > >> > > > but
> >>>> > >> > > > > > >> this
> >>>> > >> > > > > > >> > > seems to be a complex logic to implement and
> >>>> support then.
> >>>> > >> > > > > > Especially
> >>>> > >> > > > > > >> for
> >>>> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing
> later
> >>>> for
> >>>> > >> these
> >>>> > >> > > > > > >> requests).
> >>>> > >> > > > > > >> > > Also people will tend to avoid this re-routing
> >>>> facility
> >>>> > >> and
> >>>> > >> > > hold
> >>>> > >> > > > > > local
> >>>> > >> > > > > > >> > > cluster cache to ensure their high-priority
> requests
> >>>> > >> (which
> >>>> > >> > > some
> >>>> > >> > > > > of
> >>>> > >> > > > > > >> the
> >>>> > >> > > > > > >> > > admin requests are) not sent to some busy broker
> >>>> where
> >>>> > >> they
> >>>> > >> > > wait
> >>>> > >> > > > > to
> >>>> > >> > > > > > be
> >>>> > >> > > > > > >> > > routed to the correct one.
> >>>> > >> > > > > > >> > > As pointed out by Jun here (
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >>
> >>>> > >> > > > > >
> >>>> > >> > > > >
> >>>> > >> > > >
> >>>> > >> > >
> >>>> > >> >
> >>>> > >>
> >>>>
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> >>>> > >> > > > > > >> > )
> >>>> > >> > > > > > >> > > to solve the issue we might introduce a message
> >>>> type to
> >>>> > >> get
> >>>> > >> > > > > cluster
> >>>> > >> > > > > > >> > state.
> >>>> > >> > > > > > >> > > But I agree we can just update
> >>>> TopicMetadataResponse to
> >>>> > >> > > include
> >>>> > >> > > > > > >> > > controllerId (and probably smth else).
> >>>> > >> > > > > > >> > > What are you thougths?
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > > Thanks,
> >>>> > >> > > > > > >> > > Andrii
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> >>>> > >> > > > > wangguoz@gmail.com>
> >>>> > >> > > > > > >> > wrote:
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > >> I think for the topics commands we can actually
> >>>> merge
> >>>> > >> > > > > > >> > >> create/alter/delete/describe as one request
> type
> >>>> since
> >>>> > >> > their
> >>>> > >> > > > > > formats
> >>>> > >> > > > > > >> are
> >>>> > >> > > > > > >> > >> very much similar, and keep list-topics and
> others
> >>>> like
> >>>> > >> > > > > > >> > >> partition-reassignment /
> preferred-leader-election
> >>>> as
> >>>> > >> > > separate
> >>>> > >> > > > > > >> request
> >>>> > >> > > > > > >> > >> types, I also left some other comments on the
> RB (
> >>>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> >>>> > >> > > > jay.kreps@gmail.com>
> >>>> > >> > > > > > >> wrote:
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to
> just
> >>>> have
> >>>> > >> one
> >>>> > >> > > "do
> >>>> > >> > > > > > admin
> >>>> > >> > > > > > >> > >> stuff"
> >>>> > >> > > > > > >> > >> > command that has the union of all parameters.
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >> > What I am saying is that command line tools
> are
> >>>> one
> >>>> > >> > client
> >>>> > >> > > of
> >>>> > >> > > > > the
> >>>> > >> > > > > > >> > >> > administrative apis, but these will be used
> in a
> >>>> number
> >>>> > >> > of
> >>>> > >> > > > > > >> scenarios
> >>>> > >> > > > > > >> > so
> >>>> > >> > > > > > >> > >> > they should make logical sense even in the
> >>>> absence of
> >>>> > >> the
> >>>> > >> > > > > command
> >>>> > >> > > > > > >> line
> >>>> > >> > > > > > >> > >> > tool. Hence comments like trying to clarify
> the
> >>>> > >> > > relationship
> >>>> > >> > > > > > >> between
> >>>> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these
> kinds
> >>>> of
> >>>> > >> things
> >>>> > >> > > > > really
> >>>> > >> > > > > > >> need
> >>>> > >> > > > > > >> > >> to be
> >>>> > >> > > > > > >> > >> > thought through.
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >> > Hope that makes sense.
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >> > -Jay
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
> >>>> Biletskyi <
> >>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >> > > Jay,
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> > > Thanks for answering. You understood
> >>>> correctly, most
> >>>> > >> of
> >>>> > >> > > my
> >>>> > >> > > > > > >> comments
> >>>> > >> > > > > > >> > >> were
> >>>> > >> > > > > > >> > >> > > related to your point 1) - about "well
> >>>> thought-out"
> >>>> > >> > apis.
> >>>> > >> > > > > Also,
> >>>> > >> > > > > > >> yes,
> >>>> > >> > > > > > >> > >> as I
> >>>> > >> > > > > > >> > >> > > understood we would like to introduce a
> single
> >>>> > >> unified
> >>>> > >> > > CLI
> >>>> > >> > > > > tool
> >>>> > >> > > > > > >> with
> >>>> > >> > > > > > >> > >> > > centralized server-side request handling
> for
> >>>> lots of
> >>>> > >> > > > existing
> >>>> > >> > > > > > >> ones
> >>>> > >> > > > > > >> > >> (incl.
> >>>> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> >>>> > >> ReassignPartitions,
> >>>> > >> > > smth
> >>>> > >> > > > > > else
> >>>> > >> > > > > > >> if
> >>>> > >> > > > > > >> > >> added
> >>>> > >> > > > > > >> > >> > > in future). In our previous discussion (
> >>>> > >> > > > > > >> > >> > >
> >>>> https://issues.apache.org/jira/browse/KAFKA-1694)
> >>>> > >> > people
> >>>> > >> > > > > said
> >>>> > >> > > > > > >> > they'd
> >>>> > >> > > > > > >> > >> > > rather
> >>>> > >> > > > > > >> > >> > > have a separate message for each command,
> so,
> >>>> yes,
> >>>> > >> this
> >>>> > >> > > > way I
> >>>> > >> > > > > > >> came
> >>>> > >> > > > > > >> > to
> >>>> > >> > > > > > >> > >> 1-1
> >>>> > >> > > > > > >> > >> > > mapping between commands in the tool and
> >>>> protocol
> >>>> > >> > > > additions.
> >>>> > >> > > > > > But
> >>>> > >> > > > > > >> I
> >>>> > >> > > > > > >> > >> might
> >>>> > >> > > > > > >> > >> > be
> >>>> > >> > > > > > >> > >> > > wrong.
> >>>> > >> > > > > > >> > >> > > At the end I just try to start discussion
> how
> >>>> at
> >>>> > >> least
> >>>> > >> > > > > > generally
> >>>> > >> > > > > > >> > this
> >>>> > >> > > > > > >> > >> > > protocol should look like.
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> > > Thanks,
> >>>> > >> > > > > > >> > >> > > Andrii
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay
> Kreps <
> >>>> > >> > > > > > jay.kreps@gmail.com
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > >> wrote:
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> > > > Hey Andrii,
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > To answer your earlier question we just
> >>>> really
> >>>> > >> can't
> >>>> > >> > be
> >>>> > >> > > > > > adding
> >>>> > >> > > > > > >> any
> >>>> > >> > > > > > >> > >> more
> >>>> > >> > > > > > >> > >> > > > scala protocol objects. These things are
> >>>> super hard
> >>>> > >> > to
> >>>> > >> > > > > > maintain
> >>>> > >> > > > > > >> > >> because
> >>>> > >> > > > > > >> > >> > > > they hand code the byte parsing and don't
> >>>> have good
> >>>> > >> > > > > > versioning
> >>>> > >> > > > > > >> > >> support.
> >>>> > >> > > > > > >> > >> > > > Since we are already planning on
> converting
> >>>> we
> >>>> > >> > > definitely
> >>>> > >> > > > > > don't
> >>>> > >> > > > > > >> > >> want to
> >>>> > >> > > > > > >> > >> > > add
> >>>> > >> > > > > > >> > >> > > > a ton more of these--they are total tech
> >>>> debt.
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > What does it mean that the changes are
> >>>> isolated
> >>>> > >> from
> >>>> > >> > > the
> >>>> > >> > > > > > >> current
> >>>> > >> > > > > > >> > >> code
> >>>> > >> > > > > > >> > >> > > base?
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > I actually didn't understand the
> remaining
> >>>> > >> comments,
> >>>> > >> > > > which
> >>>> > >> > > > > of
> >>>> > >> > > > > > >> the
> >>>> > >> > > > > > >> > >> > points
> >>>> > >> > > > > > >> > >> > > > are you responding to?
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > Maybe one sticking point here is that it
> >>>> seems like
> >>>> > >> > you
> >>>> > >> > > > > want
> >>>> > >> > > > > > to
> >>>> > >> > > > > > >> > make
> >>>> > >> > > > > > >> > >> > some
> >>>> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1
> mapping
> >>>> > >> between
> >>>> > >> > > > > > commands
> >>>> > >> > > > > > >> you
> >>>> > >> > > > > > >> > >> > > imagine
> >>>> > >> > > > > > >> > >> > > > in the tool and protocol additions. I
> want
> >>>> to make
> >>>> > >> > sure
> >>>> > >> > > > we
> >>>> > >> > > > > > >> don't
> >>>> > >> > > > > > >> > do
> >>>> > >> > > > > > >> > >> > that.
> >>>> > >> > > > > > >> > >> > > > The protocol needs to be really really
> well
> >>>> thought
> >>>> > >> > out
> >>>> > >> > > > > > against
> >>>> > >> > > > > > >> > many
> >>>> > >> > > > > > >> > >> > use
> >>>> > >> > > > > > >> > >> > > > cases so it should make perfect logical
> >>>> sense in
> >>>> > >> the
> >>>> > >> > > > > absence
> >>>> > >> > > > > > of
> >>>> > >> > > > > > >> > >> knowing
> >>>> > >> > > > > > >> > >> > > the
> >>>> > >> > > > > > >> > >> > > > command line tool, right?
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > -Jay
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii
> >>>> Biletskyi
> >>>> > >> <
> >>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > > Hey Jay,
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > I would like to continue this
> discussion
> >>>> as it
> >>>> > >> seem
> >>>> > >> > > > there
> >>>> > >> > > > > > is
> >>>> > >> > > > > > >> no
> >>>> > >> > > > > > >> > >> > > progress
> >>>> > >> > > > > > >> > >> > > > > here.
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > First of all, could you please explain
> >>>> what did
> >>>> > >> you
> >>>> > >> > > > mean
> >>>> > >> > > > > in
> >>>> > >> > > > > > >> 2?
> >>>> > >> > > > > > >> > How
> >>>> > >> > > > > > >> > >> > > > exactly
> >>>> > >> > > > > > >> > >> > > > > are we going to migrate to the new java
> >>>> protocol
> >>>> > >> > > > > > definitions.
> >>>> > >> > > > > > >> > And
> >>>> > >> > > > > > >> > >> why
> >>>> > >> > > > > > >> > >> > > > it's
> >>>> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > I agree with you, this feature includes
> >>>> lots of
> >>>> > >> > > stuff,
> >>>> > >> > > > > but
> >>>> > >> > > > > > >> > >> thankfully
> >>>> > >> > > > > > >> > >> > > > > almost all changes are isolated from
> the
> >>>> current
> >>>> > >> > code
> >>>> > >> > > > > base,
> >>>> > >> > > > > > >> > >> > > > > so the main thing, I think, we need to
> >>>> agree is
> >>>> > >> > RQ/RP
> >>>> > >> > > > > > format.
> >>>> > >> > > > > > >> > >> > > > > So how can we start discussion about
> the
> >>>> concrete
> >>>> > >> > > > > messages
> >>>> > >> > > > > > >> > format?
> >>>> > >> > > > > > >> > >> > > > > Can we take (
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >>
> >>>> > >> > > > > >
> >>>> > >> > > > >
> >>>> > >> > > >
> >>>> > >> > >
> >>>> > >> >
> >>>> > >>
> >>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> >>>> > >> > > > > > >> > >> > > > > )
> >>>> > >> > > > > > >> > >> > > > > as starting point?
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > We had some doubts earlier whether it
> worth
> >>>> > >> > > introducing
> >>>> > >> > > > > one
> >>>> > >> > > > > > >> > >> generic
> >>>> > >> > > > > > >> > >> > > Admin
> >>>> > >> > > > > > >> > >> > > > > Request for all commands (
> >>>> > >> > > > > > >> > >> > > >
> >>>> https://issues.apache.org/jira/browse/KAFKA-1694
> >>>> > >> > > > > > >> > >> > > > > )
> >>>> > >> > > > > > >> > >> > > > > but then everybody agreed it would be
> >>>> better to
> >>>> > >> > have
> >>>> > >> > > > > > separate
> >>>> > >> > > > > > >> > >> message
> >>>> > >> > > > > > >> > >> > > for
> >>>> > >> > > > > > >> > >> > > > > each admin command. The Request part is
> >>>> really
> >>>> > >> > > dictated
> >>>> > >> > > > > > from
> >>>> > >> > > > > > >> the
> >>>> > >> > > > > > >> > >> > > command
> >>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself,
> so
> >>>> the
> >>>> > >> > proposed
> >>>> > >> > > > > > version
> >>>> > >> > > > > > >> > >> should
> >>>> > >> > > > > > >> > >> > be
> >>>> > >> > > > > > >> > >> > > > > fine (let's put aside for now remarks
> about
> >>>> > >> > Optional
> >>>> > >> > > > > type,
> >>>> > >> > > > > > >> > >> batching,
> >>>> > >> > > > > > >> > >> > > > > configs normalization - I agree with
> all of
> >>>> > >> them).
> >>>> > >> > > > > > >> > >> > > > > So the second part is Response. I see
> >>>> there are
> >>>> > >> two
> >>>> > >> > > > cases
> >>>> > >> > > > > > >> here.
> >>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> Create/Alter/... ;
> >>>> b)
> >>>> > >> "Get"
> >>>> > >> > > > > > requests -
> >>>> > >> > > > > > >> > >> > > > > List/Describe...
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > a) should only hold request result
> >>>> (regardless
> >>>> > >> what
> >>>> > >> > > we
> >>>> > >> > > > > > decide
> >>>> > >> > > > > > >> > >> about
> >>>> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
> execution).
> >>>> > >> > > > > > >> > >> > > > > Usually we provide error code in
> response
> >>>> but
> >>>> > >> since
> >>>> > >> > > we
> >>>> > >> > > > > will
> >>>> > >> > > > > > >> use
> >>>> > >> > > > > > >> > >> this
> >>>> > >> > > > > > >> > >> > in
> >>>> > >> > > > > > >> > >> > > > > interactive shell we need some human
> >>>> readable
> >>>> > >> error
> >>>> > >> > > > > > >> description
> >>>> > >> > > > > > >> > -
> >>>> > >> > > > > > >> > >> so
> >>>> > >> > > > > > >> > >> > I
> >>>> > >> > > > > > >> > >> > > > > added errorDesription field where you
> can
> >>>> at
> >>>> > >> least
> >>>> > >> > > > leave
> >>>> > >> > > > > > >> > >> > > > > exception.getMessage.
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > b) in addition to previous item message
> >>>> should
> >>>> > >> hold
> >>>> > >> > > > > command
> >>>> > >> > > > > > >> > >> specific
> >>>> > >> > > > > > >> > >> > > > > response data. We can discuss in detail
> >>>> each of
> >>>> > >> > them
> >>>> > >> > > > but
> >>>> > >> > > > > > >> let's
> >>>> > >> > > > > > >> > for
> >>>> > >> > > > > > >> > >> > now
> >>>> > >> > > > > > >> > >> > > > > agree about the overall pattern.
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > Thanks,
> >>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay
> Kreps
> >>>> <
> >>>> > >> > > > > > >> jay.kreps@gmail.com
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > >> > > wrote:
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > > Hey Joe,
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > This is great. A few comments on
> KIP-4
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 1. This is much needed functionality,
> >>>> but there
> >>>> > >> > > are a
> >>>> > >> > > > > lot
> >>>> > >> > > > > > >> of
> >>>> > >> > > > > > >> > >> the so
> >>>> > >> > > > > > >> > >> > > > let's
> >>>> > >> > > > > > >> > >> > > > > > really think these protocols
> through. We
> >>>> really
> >>>> > >> > > want
> >>>> > >> > > > to
> >>>> > >> > > > > > >> end up
> >>>> > >> > > > > > >> > >> > with a
> >>>> > >> > > > > > >> > >> > > > set
> >>>> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis.
> >>>> For this
> >>>> > >> > > > reason I
> >>>> > >> > > > > > >> think
> >>>> > >> > > > > > >> > >> it is
> >>>> > >> > > > > > >> > >> > > > > really
> >>>> > >> > > > > > >> > >> > > > > > important to think through the end
> state
> >>>> even
> >>>> > >> if
> >>>> > >> > > that
> >>>> > >> > > > > > >> includes
> >>>> > >> > > > > > >> > >> APIs
> >>>> > >> > > > > > >> > >> > > we
> >>>> > >> > > > > > >> > >> > > > > > won't implement in the first phase.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait
> until
> >>>> we
> >>>> > >> have
> >>>> > >> > > > > switched
> >>>> > >> > > > > > >> the
> >>>> > >> > > > > > >> > >> > server
> >>>> > >> > > > > > >> > >> > > > over
> >>>> > >> > > > > > >> > >> > > > > > to the new java protocol
> definitions. If
> >>>> we add
> >>>> > >> > > > upteen
> >>>> > >> > > > > > >> more ad
> >>>> > >> > > > > > >> > >> hoc
> >>>> > >> > > > > > >> > >> > > > scala
> >>>> > >> > > > > > >> > >> > > > > > objects that is just generating more
> >>>> work for
> >>>> > >> the
> >>>> > >> > > > > > >> conversion
> >>>> > >> > > > > > >> > we
> >>>> > >> > > > > > >> > >> > know
> >>>> > >> > > > > > >> > >> > > we
> >>>> > >> > > > > > >> > >> > > > > > have to do.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new
> type of
> >>>> > >> > optional
> >>>> > >> > > > > > >> parameter.
> >>>> > >> > > > > > >> > >> This
> >>>> > >> > > > > > >> > >> > is
> >>>> > >> > > > > > >> > >> > > > > > inconsistent with everything else in
> the
> >>>> > >> protocol
> >>>> > >> > > > where
> >>>> > >> > > > > > we
> >>>> > >> > > > > > >> use
> >>>> > >> > > > > > >> > >> -1
> >>>> > >> > > > > > >> > >> > or
> >>>> > >> > > > > > >> > >> > > > some
> >>>> > >> > > > > > >> > >> > > > > > other marker value. You could argue
> >>>> either way
> >>>> > >> > but
> >>>> > >> > > > > let's
> >>>> > >> > > > > > >> stick
> >>>> > >> > > > > > >> > >> with
> >>>> > >> > > > > > >> > >> > > > that
> >>>> > >> > > > > > >> > >> > > > > > for consistency. For clients that
> >>>> implemented
> >>>> > >> the
> >>>> > >> > > > > > protocol
> >>>> > >> > > > > > >> in
> >>>> > >> > > > > > >> > a
> >>>> > >> > > > > > >> > >> > > better
> >>>> > >> > > > > > >> > >> > > > > way
> >>>> > >> > > > > > >> > >> > > > > > than our scala code these basic
> >>>> primitives are
> >>>> > >> > hard
> >>>> > >> > > > to
> >>>> > >> > > > > > >> change.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
> >>>> duplicate
> >>>> > >> > > > > > >> > TopicMetadataRequest
> >>>> > >> > > > > > >> > >> > > which
> >>>> > >> > > > > > >> > >> > > > > has
> >>>> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I
> think
> >>>> we
> >>>> > >> > should
> >>>> > >> > > > > rename
> >>>> > >> > > > > > >> that
> >>>> > >> > > > > > >> > >> > > request
> >>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> >>>> > >> MetadataRequest)
> >>>> > >> > > and
> >>>> > >> > > > > > >> include
> >>>> > >> > > > > > >> > >> the id
> >>>> > >> > > > > > >> > >> > > of
> >>>> > >> > > > > > >> > >> > > > > the
> >>>> > >> > > > > > >> > >> > > > > > controller. Or are there other
> things we
> >>>> could
> >>>> > >> > add
> >>>> > >> > > > > here?
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to make
> a
> >>>> lot of
> >>>> > >> > > > requests
> >>>> > >> > > > > > that
> >>>> > >> > > > > > >> > can
> >>>> > >> > > > > > >> > >> > only
> >>>> > >> > > > > > >> > >> > > go
> >>>> > >> > > > > > >> > >> > > > > to
> >>>> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of
> >>>> burden for
> >>>> > >> > > > client
> >>>> > >> > > > > > >> > >> > > implementations
> >>>> > >> > > > > > >> > >> > > > > (it
> >>>> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can
> fail
> >>>> in many
> >>>> > >> > > parts
> >>>> > >> > > > > so
> >>>> > >> > > > > > it
> >>>> > >> > > > > > >> > >> ends up
> >>>> > >> > > > > > >> > >> > > > > being a
> >>>> > >> > > > > > >> > >> > > > > > full state machine to do right). I
> think
> >>>> we
> >>>> > >> > should
> >>>> > >> > > > > > consider
> >>>> > >> > > > > > >> > >> making
> >>>> > >> > > > > > >> > >> > > > admin
> >>>> > >> > > > > > >> > >> > > > > > commands and ideally as many of the
> >>>> other apis
> >>>> > >> as
> >>>> > >> > > > > > possible
> >>>> > >> > > > > > >> > >> > available
> >>>> > >> > > > > > >> > >> > > on
> >>>> > >> > > > > > >> > >> > > > > all
> >>>> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
> >>>> controller on
> >>>> > >> > the
> >>>> > >> > > > > broker
> >>>> > >> > > > > > >> > side.
> >>>> > >> > > > > > >> > >> > > Perhaps
> >>>> > >> > > > > > >> > >> > > > > > there would be a general way to
> >>>> encapsulate
> >>>> > >> this
> >>>> > >> > > > > > re-routing
> >>>> > >> > > > > > >> > >> > behavior.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 6. We should probably normalize the
> key
> >>>> value
> >>>> > >> > pairs
> >>>> > >> > > > > used
> >>>> > >> > > > > > >> for
> >>>> > >> > > > > > >> > >> > configs
> >>>> > >> > > > > > >> > >> > > > > rather
> >>>> > >> > > > > > >> > >> > > > > > than embedding a new formatting. So
> two
> >>>> strings
> >>>> > >> > > > rather
> >>>> > >> > > > > > than
> >>>> > >> > > > > > >> > one
> >>>> > >> > > > > > >> > >> > with
> >>>> > >> > > > > > >> > >> > > an
> >>>> > >> > > > > > >> > >> > > > > > internal equals sign.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs
> >>>> that the
> >>>> > >> > > > command
> >>>> > >> > > > > > has
> >>>> > >> > > > > > >> > >> begun or
> >>>> > >> > > > > > >> > >> > > > that
> >>>> > >> > > > > > >> > >> > > > > > the command has been completed? It
> is a
> >>>> lot
> >>>> > >> more
> >>>> > >> > > > usable
> >>>> > >> > > > > > if
> >>>> > >> > > > > > >> the
> >>>> > >> > > > > > >> > >> > > command
> >>>> > >> > > > > > >> > >> > > > > has
> >>>> > >> > > > > > >> > >> > > > > > been completed so you know that if
> you
> >>>> create a
> >>>> > >> > > topic
> >>>> > >> > > > > and
> >>>> > >> > > > > > >> then
> >>>> > >> > > > > > >> > >> > > publish
> >>>> > >> > > > > > >> > >> > > > to
> >>>> > >> > > > > > >> > >> > > > > > it you won't get an exception about
> >>>> there being
> >>>> > >> > no
> >>>> > >> > > > such
> >>>> > >> > > > > > >> topic.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
> >>>> duplicate a
> >>>> > >> lot
> >>>> > >> > > of
> >>>> > >> > > > > > stuff
> >>>> > >> > > > > > >> in
> >>>> > >> > > > > > >> > >> the
> >>>> > >> > > > > > >> > >> > > > > metadata
> >>>> > >> > > > > > >> > >> > > > > > request. Is there a reason to give
> back
> >>>> topics
> >>>> > >> > > marked
> >>>> > >> > > > > for
> >>>> > >> > > > > > >> > >> > deletion? I
> >>>> > >> > > > > > >> > >> > > > > feel
> >>>> > >> > > > > > >> > >> > > > > > like if we just make the
> post-condition
> >>>> of the
> >>>> > >> > > delete
> >>>> > >> > > > > > >> command
> >>>> > >> > > > > > >> > be
> >>>> > >> > > > > > >> > >> > that
> >>>> > >> > > > > > >> > >> > > > the
> >>>> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid of
> >>>> the need
> >>>> > >> > for
> >>>> > >> > > > this
> >>>> > >> > > > > > >> right?
> >>>> > >> > > > > > >> > >> And
> >>>> > >> > > > > > >> > >> > it
> >>>> > >> > > > > > >> > >> > > > > will
> >>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 9. Should we consider batching these
> >>>> requests?
> >>>> > >> We
> >>>> > >> > > > have
> >>>> > >> > > > > > >> > generally
> >>>> > >> > > > > > >> > >> > > tried
> >>>> > >> > > > > > >> > >> > > > to
> >>>> > >> > > > > > >> > >> > > > > > allow multiple operations to be
> batched.
> >>>> My
> >>>> > >> > > suspicion
> >>>> > >> > > > > is
> >>>> > >> > > > > > >> that
> >>>> > >> > > > > > >> > >> > without
> >>>> > >> > > > > > >> > >> > > > > this
> >>>> > >> > > > > > >> > >> > > > > > we will get a lot of code that does
> >>>> something
> >>>> > >> > like
> >>>> > >> > > > > > >> > >> > > > > >    for(topic:
> adminClient.listTopics())
> >>>> > >> > > > > > >> > >> > > > > >
>  adminClient.describeTopic(topic)
> >>>> > >> > > > > > >> > >> > > > > > this code will work great when you
> test
> >>>> on 5
> >>>> > >> > topics
> >>>> > >> > > > but
> >>>> > >> > > > > > >> not do
> >>>> > >> > > > > > >> > >> as
> >>>> > >> > > > > > >> > >> > > well
> >>>> > >> > > > > > >> > >> > > > if
> >>>> > >> > > > > > >> > >> > > > > > you have 50k.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss
> how
> >>>> we want
> >>>> > >> to
> >>>> > >> > > > > expose
> >>>> > >> > > > > > a
> >>>> > >> > > > > > >> > >> > > programmatic
> >>>> > >> > > > > > >> > >> > > > > JVM
> >>>> > >> > > > > > >> > >> > > > > > client api for these operations.
> >>>> Currently
> >>>> > >> people
> >>>> > >> > > > rely
> >>>> > >> > > > > on
> >>>> > >> > > > > > >> > >> > AdminUtils
> >>>> > >> > > > > > >> > >> > > > > which
> >>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we
> probably
> >>>> need
> >>>> > >> > > another
> >>>> > >> > > > > > client
> >>>> > >> > > > > > >> > >> under
> >>>> > >> > > > > > >> > >> > > > > clients/
> >>>> > >> > > > > > >> > >> > > > > > that exposes administrative
> >>>> functionality. We
> >>>> > >> > will
> >>>> > >> > > > need
> >>>> > >> > > > > > >> this
> >>>> > >> > > > > > >> > >> just
> >>>> > >> > > > > > >> > >> > to
> >>>> > >> > > > > > >> > >> > > > > > properly test the new apis, I
> suspect. We
> >>>> > >> should
> >>>> > >> > > > figure
> >>>> > >> > > > > > out
> >>>> > >> > > > > > >> > that
> >>>> > >> > > > > > >> > >> > API.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 11. The other information that would
> be
> >>>> really
> >>>> > >> > > useful
> >>>> > >> > > > > to
> >>>> > >> > > > > > >> get
> >>>> > >> > > > > > >> > >> would
> >>>> > >> > > > > > >> > >> > be
> >>>> > >> > > > > > >> > >> > > > > > information about partitions--how
> much
> >>>> data is
> >>>> > >> in
> >>>> > >> > > the
> >>>> > >> > > > > > >> > partition,
> >>>> > >> > > > > > >> > >> > what
> >>>> > >> > > > > > >> > >> > > > are
> >>>> > >> > > > > > >> > >> > > > > > the segment offsets, what is the
> log-end
> >>>> offset
> >>>> > >> > > (i.e.
> >>>> > >> > > > > > last
> >>>> > >> > > > > > >> > >> offset),
> >>>> > >> > > > > > >> > >> > > > what
> >>>> > >> > > > > > >> > >> > > > > is
> >>>> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think
> that
> >>>> done
> >>>> > >> > right
> >>>> > >> > > > this
> >>>> > >> > > > > > >> would
> >>>> > >> > > > > > >> > be
> >>>> > >> > > > > > >> > >> > the
> >>>> > >> > > > > > >> > >> > > > > > successor to the very awkward
> >>>> OffsetRequest we
> >>>> > >> > have
> >>>> > >> > > > > > today.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > -Jay
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe
> >>>> Stein <
> >>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
> >>>> > >> > > > > > >> > >> > > > > wrote:
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >>
> >>>> > >> > > > > >
> >>>> > >> > > > >
> >>>> > >> > > >
> >>>> > >> > >
> >>>> > >> >
> >>>> > >>
> >>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> > >> > > > > > >> > >> > > > > > > JIRA
> >>>> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> /*******************************************
> >>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
> >>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> >>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
> >>>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> >>>> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> >>>> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> ********************************************/
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >> --
> >>>> > >> > > > > > >> > >> -- Guozhang
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >
> >>>> > >> > > > > > >
> >>>> > >> > > > > >
> >>>> > >> > > > >
> >>>> > >> > > >
> >>>> > >> > >
> >>>> > >> >
> >>>> > >> >
> >>>> > >> >
> >>>> > >> > --
> >>>> > >> > Jeff Holoman
> >>>> > >> > Systems Engineer
> >>>> > >> >
> >>>> > >>
> >>>>
> >>>>
> >>>
> >>>
> >>> --
> >>> -- Guozhang
> >>>
> >>
> >>
> >>
> >> --
> >> -- Guozhang
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Guozhang Wang <wa...@gmail.com>.
Gwen,

My main motivation is not "authenticate via ownership", but rather "query
topics via ownership", and more generally "query topics via patterns",
where a pattern could be a config value, metadata k-v pair, etc. Does it
make sense?

Guozhang

On Thu, Mar 12, 2015 at 12:26 PM, Gwen Shapira <gs...@cloudera.com>
wrote:

> Found KIP-11 (
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface
> )
> It actually specifies changes to the Metadata protocol, so making sure
> both KIPs are consistent in this regard will be good.
>
> On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <gs...@cloudera.com>
> wrote:
> > Specifically for ownership, I think the plan is to add ACL (it sounds
> > like you are describing ACL) via an external system (Argus, Sentry).
> > I remember KIP-11 described this, but I can't find the KIP any longer.
> >
> > Regardless, I think KIP-4 focuses on getting information that already
> > exists from Kafka brokers, not on adding information that perhaps
> > should exist but doesn't yet?
> >
> > Gwen
> >
> >
> >
> >
> >
> > On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >> Folks,
> >>
> >> Just want to elaborate a bit more on the create-topic metadata and
> batching
> >> describe-topic based on config / metadata in my previous email as we
> work
> >> on KAFKA-1694. The main motivation is to have some sort of topic
> management
> >> mechanisms, which I think is quite important in a multi-tenant / cloud
> >> architecture: today anyone can create topics in a shared Kafka cluster,
> but
> >> there is no concept or "ownership" of topics that are created by
> different
> >> users. For example, at LinkedIn we basically distinguish topic owners
> via
> >> some casual topic name prefix, which is a bit awkward and does not fly
> as
> >> we scale our customers. It would be great to use describe-topics such
> as:
> >>
> >> Describe all topics that is created by me.
> >>
> >> Describe all topics whose retention time is overriden to X.
> >>
> >> Describe all topics whose writable group include user Y (this is
> related to
> >> authorization), etc..
> >>
> >> One possible way to achieve this is to add a metadata file in the
> >> create-topic request, whose value will also be written ZK as we create
> the
> >> topic; then describe-topics can choose to batch topics based on 1) name
> >> regex, 2) config K-V matching, 3) metadata regex, etc.
> >>
> >> Thoughts?
> >>
> >> Guozhang
> >>
> >> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >>
> >>> Thanks for the updated wiki. A few comments below:
> >>>
> >>> 1. Error description in response: I think if some errorCode could
> indicate
> >>> several different error cases then we should really change it to
> multiple
> >>> codes. In general the errorCode itself would be precise and sufficient
> for
> >>> describing the server side errors.
> >>>
> >>> 2. Describe topic request: it would be great to go beyond just
> batching on
> >>> topic name regex for this request. For example, a very common use case
> of
> >>> the topic command is to list all topics whose config A's value is B.
> With
> >>> topic name regex then we have to first retrieve __all__ topics's
> >>> description info and then filter at the client end, which will be a
> huge
> >>> burden on ZK.
> >>>
> >>> 3. Config K-Vs in create topic: this is related to the previous point;
> >>> maybe we can add another metadata K-V or just a metadata string along
> side
> >>> with config K-V in create topic like we did for offset commit request.
> This
> >>> field can be quite useful in storing information like "owner" of the
> topic
> >>> who issue the create command, etc, which is quite important for a
> >>> multi-tenant setting. Then in the describe topic request we can also
> batch
> >>> on regex of the metadata field.
> >>>
> >>> 4. Today all the admin operations are async in the sense that command
> will
> >>> return once it is written in ZK, and that is why we need extra
> verification
> >>> like testUtil.waitForTopicCreated() / verify partition reassignment
> >>> request, etc. With admin requests we could add a flag to enable /
> disable
> >>> synchronous requests; when it is turned on, the response will not
> return
> >>> until the request has been completed. And for async requests we can
> add a
> >>> "token" field in the response, and then only need a general "admin
> >>> verification request" with the given token to check if the async
> request
> >>> has been completed.
> >>>
> >>> 5. +1 for extending Metadata request to include controller /
> coordinator
> >>> information, and then we can remove the ConsumerMetadata /
> ClusterMetadata
> >>> requests.
> >>>
> >>> Guozhang
> >>>
> >>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com>
> wrote:
> >>>
> >>>> Thanks for sending that out Joe - I don't think I will be able to make
> >>>> it today, so if notes can be sent out afterward that would be great.
> >>>>
> >>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> >>>> > Thanks for sending this out Joe. Looking forward to chatting with
> >>>> everyone :)
> >>>> >
> >>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly>
> wrote:
> >>>> > > Hey, I just sent out a google hangout invite to all pmc,
> committers
> >>>> and
> >>>> > > everyone I found working on a KIP. If I missed anyone in the
> invite
> >>>> please
> >>>> > > let me know and can update it, np.
> >>>> > >
> >>>> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can
> get
> >>>> INFRA
> >>>> > > help to make a google account so we can manage better?
> >>>> > >
> >>>> > > To discuss
> >>>> > >
> >>>>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> >>>> > > in progress and related JIRA that are interdependent and common
> work.
> >>>> > >
> >>>> > > ~ Joe Stein
> >>>> > >
> >>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com>
> >>>> wrote:
> >>>> > >
> >>>> > >> Let's stay on Google hangouts that will also record and make the
> >>>> sessions
> >>>> > >> available on youtube.
> >>>> > >>
> >>>> > >> -Jay
> >>>> > >>
> >>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
> >>>> jholoman@cloudera.com>
> >>>> > >> wrote:
> >>>> > >>
> >>>> > >> > Jay / Joe
> >>>> > >> >
> >>>> > >> > We're happy to send out a Webex for this purpose. We could
> record
> >>>> the
> >>>> > >> > sessions if there is interest and publish them out.
> >>>> > >> >
> >>>> > >> > Thanks
> >>>> > >> >
> >>>> > >> > Jeff
> >>>> > >> >
> >>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <
> jay.kreps@gmail.com>
> >>>> wrote:
> >>>> > >> >
> >>>> > >> > > Let's try to get the technical hang-ups sorted out, though. I
> >>>> really
> >>>> > >> > think
> >>>> > >> > > there is some benefit to live discussion vs writing. I am
> >>>> hopeful that
> >>>> > >> if
> >>>> > >> > > we post instructions and give ourselves a few attempts we can
> >>>> get it
> >>>> > >> > > working.
> >>>> > >> > >
> >>>> > >> > > Tuesday at that time would work for me...any objections?
> >>>> > >> > >
> >>>> > >> > > -Jay
> >>>> > >> > >
> >>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <
> joe.stein@stealth.ly
> >>>> >
> >>>> > >> wrote:
> >>>> > >> > >
> >>>> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET /
> 10am
> >>>> PT
> >>>> > >> ????
> >>>> > >> > > >
> >>>> > >> > > > I don't mind google hangout but there is always some issue
> or
> >>>> > >> whatever
> >>>> > >> > so
> >>>> > >> > > > we know the apache irc channel works. We can start there
> and
> >>>> see how
> >>>> > >> it
> >>>> > >> > > > goes? We can pull transcripts too and associate to tickets
> if
> >>>> need be
> >>>> > >> > > makes
> >>>> > >> > > > it helpful for things.
> >>>> > >> > > >
> >>>> > >> > > > ~ Joestein
> >>>> > >> > > >
> >>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> >>>> jay.kreps@gmail.com>
> >>>> > >> > wrote:
> >>>> > >> > > >
> >>>> > >> > > > > We'd talked about doing a Google Hangout to chat about
> this.
> >>>> What
> >>>> > >> > about
> >>>> > >> > > > > generalizing that a little further...I actually think it
> >>>> would be
> >>>> > >> > good
> >>>> > >> > > > for
> >>>> > >> > > > > everyone spending a reasonable chunk of their week on
> Kafka
> >>>> stuff
> >>>> > >> to
> >>>> > >> > > > maybe
> >>>> > >> > > > > sync up once a week. I think we could use time to talk
> >>>> through
> >>>> > >> design
> >>>> > >> > > > > stuff, make sure we are on top of code reviews, talk
> through
> >>>> any
> >>>> > >> > tricky
> >>>> > >> > > > > issues, etc.
> >>>> > >> > > > >
> >>>> > >> > > > > We can make it publicly available so that any one can
> follow
> >>>> along
> >>>> > >> > who
> >>>> > >> > > > > likes.
> >>>> > >> > > > >
> >>>> > >> > > > > Any interest in doing this? If so I'll try to set it up
> >>>> starting
> >>>> > >> next
> >>>> > >> > > > week.
> >>>> > >> > > > >
> >>>> > >> > > > > -Jay
> >>>> > >> > > > >
> >>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> >>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > >
> >>>> > >> > > > > > Hi all,
> >>>> > >> > > > > >
> >>>> > >> > > > > > I've updated KIP page, fixed / aligned document
> structure.
> >>>> Also I
> >>>> > >> > > added
> >>>> > >> > > > > > some
> >>>> > >> > > > > > very initial proposal for AdminClient so we have
> something
> >>>> to
> >>>> > >> start
> >>>> > >> > > > from
> >>>> > >> > > > > > while
> >>>> > >> > > > > > discussing the KIP.
> >>>> > >> > > > > >
> >>>> > >> > > > > >
> >>>> > >> > > > > >
> >>>> > >> > > > >
> >>>> > >> > > >
> >>>> > >> > >
> >>>> > >> >
> >>>> > >>
> >>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >>>> > >> > > > > >
> >>>> > >> > > > > > Thanks,
> >>>> > >> > > > > > Andrii Biletskyi
> >>>> > >> > > > > >
> >>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> >>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > > >
> >>>> > >> > > > > > > Jay,
> >>>> > >> > > > > > >
> >>>> > >> > > > > > > Re error messages: you are right, in most cases
> client
> >>>> will
> >>>> > >> have
> >>>> > >> > > > enough
> >>>> > >> > > > > > > context to show descriptive error message. My
> concern is
> >>>> that
> >>>> > >> we
> >>>> > >> > > will
> >>>> > >> > > > > > have
> >>>> > >> > > > > > > to
> >>>> > >> > > > > > > add lots of new error codes for each possible error.
> Of
> >>>> course,
> >>>> > >> > we
> >>>> > >> > > > > could
> >>>> > >> > > > > > > reuse
> >>>> > >> > > > > > > some of existing like UknownTopicOrPartitionCode,
> but we
> >>>> will
> >>>> > >> > also
> >>>> > >> > > > need
> >>>> > >> > > > > > to
> >>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
> >>>> TopicConfigInvalid (both
> >>>> > >> > for
> >>>> > >> > > > > topic
> >>>> > >> > > > > > > name and config, and probably user would like to know
> >>>> what
> >>>> > >> > exactly
> >>>> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
> >>>> > >> InternalError
> >>>> > >> > > > (e.g.
> >>>> > >> > > > > > > zookeeper failure) etc.
> >>>> > >> > > > > > > And this is only for TopicCommand, we will also need
> to
> >>>> add
> >>>> > >> > similar
> >>>> > >> > > > > stuff
> >>>> > >> > > > > > > for
> >>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up
> >>>> with a
> >>>> > >> > large
> >>>> > >> > > > list
> >>>> > >> > > > > > of
> >>>> > >> > > > > > > error codes, used only in Admin protocol.
> >>>> > >> > > > > > > Having said that, I agree my proposal is not
> consistent
> >>>> with
> >>>> > >> > other
> >>>> > >> > > > > cases.
> >>>> > >> > > > > > > Maybe we can find better solution or something
> >>>> in-between.
> >>>> > >> > > > > > >
> >>>> > >> > > > > > > Re Hangout chat: I think it is a great idea. This
> way we
> >>>> can
> >>>> > >> move
> >>>> > >> > > on
> >>>> > >> > > > > > > faster.
> >>>> > >> > > > > > > Let's agree somehow on date/time so people can join.
> >>>> Will work
> >>>> > >> > for
> >>>> > >> > > me
> >>>> > >> > > > > > this
> >>>> > >> > > > > > > and
> >>>> > >> > > > > > > next week almost anytime if agreed in advance.
> >>>> > >> > > > > > >
> >>>> > >> > > > > > > Thanks,
> >>>> > >> > > > > > > Andrii
> >>>> > >> > > > > > >
> >>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> >>>> > >> jay.kreps@gmail.com>
> >>>> > >> > > > > wrote:
> >>>> > >> > > > > > >
> >>>> > >> > > > > > >> Hey Andrii,
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >> Generally we can do good error handling without
> needing
> >>>> custom
> >>>> > >> > > > > > server-side
> >>>> > >> > > > > > >> messages. I.e. generally the client has the context
> to
> >>>> know
> >>>> > >> that
> >>>> > >> > > if
> >>>> > >> > > > it
> >>>> > >> > > > > > got
> >>>> > >> > > > > > >> an error that the topic doesn't exist to say "Topic
> X
> >>>> doesn't
> >>>> > >> > > exist"
> >>>> > >> > > > > > >> rather
> >>>> > >> > > > > > >> than "error code 14" (or whatever). Maybe there are
> >>>> specific
> >>>> > >> > cases
> >>>> > >> > > > > where
> >>>> > >> > > > > > >> this is hard? If we want to add server-side error
> >>>> messages we
> >>>> > >> > > really
> >>>> > >> > > > > do
> >>>> > >> > > > > > >> need to do this in a consistent way across the
> protocol.
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >> I still have a bunch of open questions here from my
> >>>> previous
> >>>> > >> > > list. I
> >>>> > >> > > > > > will
> >>>> > >> > > > > > >> be out for the next few days for Strata though.
> Maybe
> >>>> we could
> >>>> > >> > do
> >>>> > >> > > a
> >>>> > >> > > > > > Google
> >>>> > >> > > > > > >> Hangout chat on any open issues some time towards
> the
> >>>> end of
> >>>> > >> > next
> >>>> > >> > > > week
> >>>> > >> > > > > > for
> >>>> > >> > > > > > >> anyone interested in this ticket? I have a feeling
> that
> >>>> might
> >>>> > >> > > > progress
> >>>> > >> > > > > > >> things a little faster than email--I think we could
> talk
> >>>> > >> through
> >>>> > >> > > > those
> >>>> > >> > > > > > >> issues I brought up fairly quickly...
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >> -Jay
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> >>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >> > Hi all,
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > I'm trying to address some of the issues which
> were
> >>>> > >> mentioned
> >>>> > >> > > > > earlier
> >>>> > >> > > > > > >> about
> >>>> > >> > > > > > >> > Admin RQ/RP format. One of those was about
> batching
> >>>> > >> > operations.
> >>>> > >> > > > What
> >>>> > >> > > > > > if
> >>>> > >> > > > > > >> we
> >>>> > >> > > > > > >> > follow TopicCommand approach and let people
> specify
> >>>> > >> topic-name
> >>>> > >> > > by
> >>>> > >> > > > > > >> regexp -
> >>>> > >> > > > > > >> > would that cover most of the use cases?
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > Secondly, is what information should we generally
> >>>> provide in
> >>>> > >> > > Admin
> >>>> > >> > > > > > >> > responses.
> >>>> > >> > > > > > >> > I realize that Admin commands don't imply they
> will
> >>>> be used
> >>>> > >> > only
> >>>> > >> > > > in
> >>>> > >> > > > > > CLI
> >>>> > >> > > > > > >> > but,
> >>>> > >> > > > > > >> > it seems to me, CLI is a very important client of
> this
> >>>> > >> > feature.
> >>>> > >> > > In
> >>>> > >> > > > > > this
> >>>> > >> > > > > > >> > case,
> >>>> > >> > > > > > >> > seems logical, we would like to provide users with
> >>>> rich
> >>>> > >> > > experience
> >>>> > >> > > > > in
> >>>> > >> > > > > > >> terms
> >>>> > >> > > > > > >> > of
> >>>> > >> > > > > > >> > getting results / errors of the executed commands.
> >>>> Usually
> >>>> > >> we
> >>>> > >> > > > supply
> >>>> > >> > > > > > >> with
> >>>> > >> > > > > > >> > responses only errorCode, which looks very
> limiting,
> >>>> in case
> >>>> > >> > of
> >>>> > >> > > > CLI
> >>>> > >> > > > > we
> >>>> > >> > > > > > >> may
> >>>> > >> > > > > > >> > want to print human readable error description.
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > So, taking into account previous item about
> batching,
> >>>> what
> >>>> > >> do
> >>>> > >> > > you
> >>>> > >> > > > > > think
> >>>> > >> > > > > > >> > about
> >>>> > >> > > > > > >> > having smth like:
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > ('create' doesn't support regexp)
> >>>> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions
> Replicas
> >>>> > >> > > > > ReplicaAssignment
> >>>> > >> > > > > > >> > [Config]
> >>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
> >>>> > >> > > > > > >> >   ErrorCode => int16
> >>>> > >> > > > > > >> >   ErrorDescription => string (empty if successful)
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> >>>> > >> > > ReplicaAssignment
> >>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> >>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
> >>>> ErrorDescription]
> >>>> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
> >>>> > >> > > > > > >> >   CommandErrorCode => int16
> >>>> > >> > > > > > >> >   CommandErrorDescription => string (nonempty in
> case
> >>>> of
> >>>> > >> fatal
> >>>> > >> > > > > error,
> >>>> > >> > > > > > >> e.g.
> >>>> > >> > > > > > >> > we couldn't get topics by regexp)
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> >>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName
> TopicDescription
> >>>> > >> ErrorCode
> >>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> >>>> CommandErrorDescription
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > Also, any thoughts about our discussion regarding
> >>>> re-routing
> >>>> > >> > > > > facility?
> >>>> > >> > > > > > >> In
> >>>> > >> > > > > > >> > my
> >>>> > >> > > > > > >> > understanding, it is like between augmenting
> >>>> > >> > > TopicMetadataRequest
> >>>> > >> > > > > > >> > (to include at least controllerId) and
> implementing
> >>>> new
> >>>> > >> > generic
> >>>> > >> > > > > > >> re-routing
> >>>> > >> > > > > > >> > facility so sending messages to controller will be
> >>>> handled
> >>>> > >> by
> >>>> > >> > > it.
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > Thanks,
> >>>> > >> > > > > > >> > Andrii Biletskyi
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi
> <
> >>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > > @Guozhang:
> >>>> > >> > > > > > >> > > Thanks for your comments, I've answered some of
> >>>> those. The
> >>>> > >> > > main
> >>>> > >> > > > > > thing
> >>>> > >> > > > > > >> is
> >>>> > >> > > > > > >> > > having merged request for
> >>>> create-alter-delete-describe - I
> >>>> > >> > > have
> >>>> > >> > > > > some
> >>>> > >> > > > > > >> > > concerns about this approach.
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > > @*Jay*:
> >>>> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is
> also
> >>>> one of
> >>>> > >> > the
> >>>> > >> > > > > > >> concerns.
> >>>> > >> > > > > > >> > We
> >>>> > >> > > > > > >> > > can solve it if we implement re-routing
> facility.
> >>>> But I
> >>>> > >> > agree
> >>>> > >> > > > with
> >>>> > >> > > > > > >> > > Guozhang - it will make clients' internals a
> little
> >>>> bit
> >>>> > >> > easier
> >>>> > >> > > > but
> >>>> > >> > > > > > >> this
> >>>> > >> > > > > > >> > > seems to be a complex logic to implement and
> >>>> support then.
> >>>> > >> > > > > > Especially
> >>>> > >> > > > > > >> for
> >>>> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing
> later
> >>>> for
> >>>> > >> these
> >>>> > >> > > > > > >> requests).
> >>>> > >> > > > > > >> > > Also people will tend to avoid this re-routing
> >>>> facility
> >>>> > >> and
> >>>> > >> > > hold
> >>>> > >> > > > > > local
> >>>> > >> > > > > > >> > > cluster cache to ensure their high-priority
> requests
> >>>> > >> (which
> >>>> > >> > > some
> >>>> > >> > > > > of
> >>>> > >> > > > > > >> the
> >>>> > >> > > > > > >> > > admin requests are) not sent to some busy broker
> >>>> where
> >>>> > >> they
> >>>> > >> > > wait
> >>>> > >> > > > > to
> >>>> > >> > > > > > be
> >>>> > >> > > > > > >> > > routed to the correct one.
> >>>> > >> > > > > > >> > > As pointed out by Jun here (
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >>
> >>>> > >> > > > > >
> >>>> > >> > > > >
> >>>> > >> > > >
> >>>> > >> > >
> >>>> > >> >
> >>>> > >>
> >>>>
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> >>>> > >> > > > > > >> > )
> >>>> > >> > > > > > >> > > to solve the issue we might introduce a message
> >>>> type to
> >>>> > >> get
> >>>> > >> > > > > cluster
> >>>> > >> > > > > > >> > state.
> >>>> > >> > > > > > >> > > But I agree we can just update
> >>>> TopicMetadataResponse to
> >>>> > >> > > include
> >>>> > >> > > > > > >> > > controllerId (and probably smth else).
> >>>> > >> > > > > > >> > > What are you thougths?
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > > Thanks,
> >>>> > >> > > > > > >> > > Andrii
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> >>>> > >> > > > > wangguoz@gmail.com>
> >>>> > >> > > > > > >> > wrote:
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > >> I think for the topics commands we can actually
> >>>> merge
> >>>> > >> > > > > > >> > >> create/alter/delete/describe as one request
> type
> >>>> since
> >>>> > >> > their
> >>>> > >> > > > > > formats
> >>>> > >> > > > > > >> are
> >>>> > >> > > > > > >> > >> very much similar, and keep list-topics and
> others
> >>>> like
> >>>> > >> > > > > > >> > >> partition-reassignment /
> preferred-leader-election
> >>>> as
> >>>> > >> > > separate
> >>>> > >> > > > > > >> request
> >>>> > >> > > > > > >> > >> types, I also left some other comments on the
> RB (
> >>>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> >>>> > >> > > > jay.kreps@gmail.com>
> >>>> > >> > > > > > >> wrote:
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to
> just
> >>>> have
> >>>> > >> one
> >>>> > >> > > "do
> >>>> > >> > > > > > admin
> >>>> > >> > > > > > >> > >> stuff"
> >>>> > >> > > > > > >> > >> > command that has the union of all parameters.
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >> > What I am saying is that command line tools
> are
> >>>> one
> >>>> > >> > client
> >>>> > >> > > of
> >>>> > >> > > > > the
> >>>> > >> > > > > > >> > >> > administrative apis, but these will be used
> in a
> >>>> number
> >>>> > >> > of
> >>>> > >> > > > > > >> scenarios
> >>>> > >> > > > > > >> > so
> >>>> > >> > > > > > >> > >> > they should make logical sense even in the
> >>>> absence of
> >>>> > >> the
> >>>> > >> > > > > command
> >>>> > >> > > > > > >> line
> >>>> > >> > > > > > >> > >> > tool. Hence comments like trying to clarify
> the
> >>>> > >> > > relationship
> >>>> > >> > > > > > >> between
> >>>> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these
> kinds
> >>>> of
> >>>> > >> things
> >>>> > >> > > > > really
> >>>> > >> > > > > > >> need
> >>>> > >> > > > > > >> > >> to be
> >>>> > >> > > > > > >> > >> > thought through.
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >> > Hope that makes sense.
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >> > -Jay
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
> >>>> Biletskyi <
> >>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >> > > Jay,
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> > > Thanks for answering. You understood
> >>>> correctly, most
> >>>> > >> of
> >>>> > >> > > my
> >>>> > >> > > > > > >> comments
> >>>> > >> > > > > > >> > >> were
> >>>> > >> > > > > > >> > >> > > related to your point 1) - about "well
> >>>> thought-out"
> >>>> > >> > apis.
> >>>> > >> > > > > Also,
> >>>> > >> > > > > > >> yes,
> >>>> > >> > > > > > >> > >> as I
> >>>> > >> > > > > > >> > >> > > understood we would like to introduce a
> single
> >>>> > >> unified
> >>>> > >> > > CLI
> >>>> > >> > > > > tool
> >>>> > >> > > > > > >> with
> >>>> > >> > > > > > >> > >> > > centralized server-side request handling
> for
> >>>> lots of
> >>>> > >> > > > existing
> >>>> > >> > > > > > >> ones
> >>>> > >> > > > > > >> > >> (incl.
> >>>> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> >>>> > >> ReassignPartitions,
> >>>> > >> > > smth
> >>>> > >> > > > > > else
> >>>> > >> > > > > > >> if
> >>>> > >> > > > > > >> > >> added
> >>>> > >> > > > > > >> > >> > > in future). In our previous discussion (
> >>>> > >> > > > > > >> > >> > >
> >>>> https://issues.apache.org/jira/browse/KAFKA-1694)
> >>>> > >> > people
> >>>> > >> > > > > said
> >>>> > >> > > > > > >> > they'd
> >>>> > >> > > > > > >> > >> > > rather
> >>>> > >> > > > > > >> > >> > > have a separate message for each command,
> so,
> >>>> yes,
> >>>> > >> this
> >>>> > >> > > > way I
> >>>> > >> > > > > > >> came
> >>>> > >> > > > > > >> > to
> >>>> > >> > > > > > >> > >> 1-1
> >>>> > >> > > > > > >> > >> > > mapping between commands in the tool and
> >>>> protocol
> >>>> > >> > > > additions.
> >>>> > >> > > > > > But
> >>>> > >> > > > > > >> I
> >>>> > >> > > > > > >> > >> might
> >>>> > >> > > > > > >> > >> > be
> >>>> > >> > > > > > >> > >> > > wrong.
> >>>> > >> > > > > > >> > >> > > At the end I just try to start discussion
> how
> >>>> at
> >>>> > >> least
> >>>> > >> > > > > > generally
> >>>> > >> > > > > > >> > this
> >>>> > >> > > > > > >> > >> > > protocol should look like.
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> > > Thanks,
> >>>> > >> > > > > > >> > >> > > Andrii
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay
> Kreps <
> >>>> > >> > > > > > jay.kreps@gmail.com
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >> > >> wrote:
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> > > > Hey Andrii,
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > To answer your earlier question we just
> >>>> really
> >>>> > >> can't
> >>>> > >> > be
> >>>> > >> > > > > > adding
> >>>> > >> > > > > > >> any
> >>>> > >> > > > > > >> > >> more
> >>>> > >> > > > > > >> > >> > > > scala protocol objects. These things are
> >>>> super hard
> >>>> > >> > to
> >>>> > >> > > > > > maintain
> >>>> > >> > > > > > >> > >> because
> >>>> > >> > > > > > >> > >> > > > they hand code the byte parsing and don't
> >>>> have good
> >>>> > >> > > > > > versioning
> >>>> > >> > > > > > >> > >> support.
> >>>> > >> > > > > > >> > >> > > > Since we are already planning on
> converting
> >>>> we
> >>>> > >> > > definitely
> >>>> > >> > > > > > don't
> >>>> > >> > > > > > >> > >> want to
> >>>> > >> > > > > > >> > >> > > add
> >>>> > >> > > > > > >> > >> > > > a ton more of these--they are total tech
> >>>> debt.
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > What does it mean that the changes are
> >>>> isolated
> >>>> > >> from
> >>>> > >> > > the
> >>>> > >> > > > > > >> current
> >>>> > >> > > > > > >> > >> code
> >>>> > >> > > > > > >> > >> > > base?
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > I actually didn't understand the
> remaining
> >>>> > >> comments,
> >>>> > >> > > > which
> >>>> > >> > > > > of
> >>>> > >> > > > > > >> the
> >>>> > >> > > > > > >> > >> > points
> >>>> > >> > > > > > >> > >> > > > are you responding to?
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > Maybe one sticking point here is that it
> >>>> seems like
> >>>> > >> > you
> >>>> > >> > > > > want
> >>>> > >> > > > > > to
> >>>> > >> > > > > > >> > make
> >>>> > >> > > > > > >> > >> > some
> >>>> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1
> mapping
> >>>> > >> between
> >>>> > >> > > > > > commands
> >>>> > >> > > > > > >> you
> >>>> > >> > > > > > >> > >> > > imagine
> >>>> > >> > > > > > >> > >> > > > in the tool and protocol additions. I
> want
> >>>> to make
> >>>> > >> > sure
> >>>> > >> > > > we
> >>>> > >> > > > > > >> don't
> >>>> > >> > > > > > >> > do
> >>>> > >> > > > > > >> > >> > that.
> >>>> > >> > > > > > >> > >> > > > The protocol needs to be really really
> well
> >>>> thought
> >>>> > >> > out
> >>>> > >> > > > > > against
> >>>> > >> > > > > > >> > many
> >>>> > >> > > > > > >> > >> > use
> >>>> > >> > > > > > >> > >> > > > cases so it should make perfect logical
> >>>> sense in
> >>>> > >> the
> >>>> > >> > > > > absence
> >>>> > >> > > > > > of
> >>>> > >> > > > > > >> > >> knowing
> >>>> > >> > > > > > >> > >> > > the
> >>>> > >> > > > > > >> > >> > > > command line tool, right?
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > -Jay
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii
> >>>> Biletskyi
> >>>> > >> <
> >>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > > Hey Jay,
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > I would like to continue this
> discussion
> >>>> as it
> >>>> > >> seem
> >>>> > >> > > > there
> >>>> > >> > > > > > is
> >>>> > >> > > > > > >> no
> >>>> > >> > > > > > >> > >> > > progress
> >>>> > >> > > > > > >> > >> > > > > here.
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > First of all, could you please explain
> >>>> what did
> >>>> > >> you
> >>>> > >> > > > mean
> >>>> > >> > > > > in
> >>>> > >> > > > > > >> 2?
> >>>> > >> > > > > > >> > How
> >>>> > >> > > > > > >> > >> > > > exactly
> >>>> > >> > > > > > >> > >> > > > > are we going to migrate to the new java
> >>>> protocol
> >>>> > >> > > > > > definitions.
> >>>> > >> > > > > > >> > And
> >>>> > >> > > > > > >> > >> why
> >>>> > >> > > > > > >> > >> > > > it's
> >>>> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > I agree with you, this feature includes
> >>>> lots of
> >>>> > >> > > stuff,
> >>>> > >> > > > > but
> >>>> > >> > > > > > >> > >> thankfully
> >>>> > >> > > > > > >> > >> > > > > almost all changes are isolated from
> the
> >>>> current
> >>>> > >> > code
> >>>> > >> > > > > base,
> >>>> > >> > > > > > >> > >> > > > > so the main thing, I think, we need to
> >>>> agree is
> >>>> > >> > RQ/RP
> >>>> > >> > > > > > format.
> >>>> > >> > > > > > >> > >> > > > > So how can we start discussion about
> the
> >>>> concrete
> >>>> > >> > > > > messages
> >>>> > >> > > > > > >> > format?
> >>>> > >> > > > > > >> > >> > > > > Can we take (
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >>
> >>>> > >> > > > > >
> >>>> > >> > > > >
> >>>> > >> > > >
> >>>> > >> > >
> >>>> > >> >
> >>>> > >>
> >>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> >>>> > >> > > > > > >> > >> > > > > )
> >>>> > >> > > > > > >> > >> > > > > as starting point?
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > We had some doubts earlier whether it
> worth
> >>>> > >> > > introducing
> >>>> > >> > > > > one
> >>>> > >> > > > > > >> > >> generic
> >>>> > >> > > > > > >> > >> > > Admin
> >>>> > >> > > > > > >> > >> > > > > Request for all commands (
> >>>> > >> > > > > > >> > >> > > >
> >>>> https://issues.apache.org/jira/browse/KAFKA-1694
> >>>> > >> > > > > > >> > >> > > > > )
> >>>> > >> > > > > > >> > >> > > > > but then everybody agreed it would be
> >>>> better to
> >>>> > >> > have
> >>>> > >> > > > > > separate
> >>>> > >> > > > > > >> > >> message
> >>>> > >> > > > > > >> > >> > > for
> >>>> > >> > > > > > >> > >> > > > > each admin command. The Request part is
> >>>> really
> >>>> > >> > > dictated
> >>>> > >> > > > > > from
> >>>> > >> > > > > > >> the
> >>>> > >> > > > > > >> > >> > > command
> >>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself,
> so
> >>>> the
> >>>> > >> > proposed
> >>>> > >> > > > > > version
> >>>> > >> > > > > > >> > >> should
> >>>> > >> > > > > > >> > >> > be
> >>>> > >> > > > > > >> > >> > > > > fine (let's put aside for now remarks
> about
> >>>> > >> > Optional
> >>>> > >> > > > > type,
> >>>> > >> > > > > > >> > >> batching,
> >>>> > >> > > > > > >> > >> > > > > configs normalization - I agree with
> all of
> >>>> > >> them).
> >>>> > >> > > > > > >> > >> > > > > So the second part is Response. I see
> >>>> there are
> >>>> > >> two
> >>>> > >> > > > cases
> >>>> > >> > > > > > >> here.
> >>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests -
> Create/Alter/... ;
> >>>> b)
> >>>> > >> "Get"
> >>>> > >> > > > > > requests -
> >>>> > >> > > > > > >> > >> > > > > List/Describe...
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > a) should only hold request result
> >>>> (regardless
> >>>> > >> what
> >>>> > >> > > we
> >>>> > >> > > > > > decide
> >>>> > >> > > > > > >> > >> about
> >>>> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
> execution).
> >>>> > >> > > > > > >> > >> > > > > Usually we provide error code in
> response
> >>>> but
> >>>> > >> since
> >>>> > >> > > we
> >>>> > >> > > > > will
> >>>> > >> > > > > > >> use
> >>>> > >> > > > > > >> > >> this
> >>>> > >> > > > > > >> > >> > in
> >>>> > >> > > > > > >> > >> > > > > interactive shell we need some human
> >>>> readable
> >>>> > >> error
> >>>> > >> > > > > > >> description
> >>>> > >> > > > > > >> > -
> >>>> > >> > > > > > >> > >> so
> >>>> > >> > > > > > >> > >> > I
> >>>> > >> > > > > > >> > >> > > > > added errorDesription field where you
> can
> >>>> at
> >>>> > >> least
> >>>> > >> > > > leave
> >>>> > >> > > > > > >> > >> > > > > exception.getMessage.
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > b) in addition to previous item message
> >>>> should
> >>>> > >> hold
> >>>> > >> > > > > command
> >>>> > >> > > > > > >> > >> specific
> >>>> > >> > > > > > >> > >> > > > > response data. We can discuss in detail
> >>>> each of
> >>>> > >> > them
> >>>> > >> > > > but
> >>>> > >> > > > > > >> let's
> >>>> > >> > > > > > >> > for
> >>>> > >> > > > > > >> > >> > now
> >>>> > >> > > > > > >> > >> > > > > agree about the overall pattern.
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > Thanks,
> >>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay
> Kreps
> >>>> <
> >>>> > >> > > > > > >> jay.kreps@gmail.com
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > >> > > wrote:
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > > > > Hey Joe,
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > This is great. A few comments on
> KIP-4
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 1. This is much needed functionality,
> >>>> but there
> >>>> > >> > > are a
> >>>> > >> > > > > lot
> >>>> > >> > > > > > >> of
> >>>> > >> > > > > > >> > >> the so
> >>>> > >> > > > > > >> > >> > > > let's
> >>>> > >> > > > > > >> > >> > > > > > really think these protocols
> through. We
> >>>> really
> >>>> > >> > > want
> >>>> > >> > > > to
> >>>> > >> > > > > > >> end up
> >>>> > >> > > > > > >> > >> > with a
> >>>> > >> > > > > > >> > >> > > > set
> >>>> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis.
> >>>> For this
> >>>> > >> > > > reason I
> >>>> > >> > > > > > >> think
> >>>> > >> > > > > > >> > >> it is
> >>>> > >> > > > > > >> > >> > > > > really
> >>>> > >> > > > > > >> > >> > > > > > important to think through the end
> state
> >>>> even
> >>>> > >> if
> >>>> > >> > > that
> >>>> > >> > > > > > >> includes
> >>>> > >> > > > > > >> > >> APIs
> >>>> > >> > > > > > >> > >> > > we
> >>>> > >> > > > > > >> > >> > > > > > won't implement in the first phase.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait
> until
> >>>> we
> >>>> > >> have
> >>>> > >> > > > > switched
> >>>> > >> > > > > > >> the
> >>>> > >> > > > > > >> > >> > server
> >>>> > >> > > > > > >> > >> > > > over
> >>>> > >> > > > > > >> > >> > > > > > to the new java protocol
> definitions. If
> >>>> we add
> >>>> > >> > > > upteen
> >>>> > >> > > > > > >> more ad
> >>>> > >> > > > > > >> > >> hoc
> >>>> > >> > > > > > >> > >> > > > scala
> >>>> > >> > > > > > >> > >> > > > > > objects that is just generating more
> >>>> work for
> >>>> > >> the
> >>>> > >> > > > > > >> conversion
> >>>> > >> > > > > > >> > we
> >>>> > >> > > > > > >> > >> > know
> >>>> > >> > > > > > >> > >> > > we
> >>>> > >> > > > > > >> > >> > > > > > have to do.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new
> type of
> >>>> > >> > optional
> >>>> > >> > > > > > >> parameter.
> >>>> > >> > > > > > >> > >> This
> >>>> > >> > > > > > >> > >> > is
> >>>> > >> > > > > > >> > >> > > > > > inconsistent with everything else in
> the
> >>>> > >> protocol
> >>>> > >> > > > where
> >>>> > >> > > > > > we
> >>>> > >> > > > > > >> use
> >>>> > >> > > > > > >> > >> -1
> >>>> > >> > > > > > >> > >> > or
> >>>> > >> > > > > > >> > >> > > > some
> >>>> > >> > > > > > >> > >> > > > > > other marker value. You could argue
> >>>> either way
> >>>> > >> > but
> >>>> > >> > > > > let's
> >>>> > >> > > > > > >> stick
> >>>> > >> > > > > > >> > >> with
> >>>> > >> > > > > > >> > >> > > > that
> >>>> > >> > > > > > >> > >> > > > > > for consistency. For clients that
> >>>> implemented
> >>>> > >> the
> >>>> > >> > > > > > protocol
> >>>> > >> > > > > > >> in
> >>>> > >> > > > > > >> > a
> >>>> > >> > > > > > >> > >> > > better
> >>>> > >> > > > > > >> > >> > > > > way
> >>>> > >> > > > > > >> > >> > > > > > than our scala code these basic
> >>>> primitives are
> >>>> > >> > hard
> >>>> > >> > > > to
> >>>> > >> > > > > > >> change.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
> >>>> duplicate
> >>>> > >> > > > > > >> > TopicMetadataRequest
> >>>> > >> > > > > > >> > >> > > which
> >>>> > >> > > > > > >> > >> > > > > has
> >>>> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I
> think
> >>>> we
> >>>> > >> > should
> >>>> > >> > > > > rename
> >>>> > >> > > > > > >> that
> >>>> > >> > > > > > >> > >> > > request
> >>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> >>>> > >> MetadataRequest)
> >>>> > >> > > and
> >>>> > >> > > > > > >> include
> >>>> > >> > > > > > >> > >> the id
> >>>> > >> > > > > > >> > >> > > of
> >>>> > >> > > > > > >> > >> > > > > the
> >>>> > >> > > > > > >> > >> > > > > > controller. Or are there other
> things we
> >>>> could
> >>>> > >> > add
> >>>> > >> > > > > here?
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to make
> a
> >>>> lot of
> >>>> > >> > > > requests
> >>>> > >> > > > > > that
> >>>> > >> > > > > > >> > can
> >>>> > >> > > > > > >> > >> > only
> >>>> > >> > > > > > >> > >> > > go
> >>>> > >> > > > > > >> > >> > > > > to
> >>>> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of
> >>>> burden for
> >>>> > >> > > > client
> >>>> > >> > > > > > >> > >> > > implementations
> >>>> > >> > > > > > >> > >> > > > > (it
> >>>> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can
> fail
> >>>> in many
> >>>> > >> > > parts
> >>>> > >> > > > > so
> >>>> > >> > > > > > it
> >>>> > >> > > > > > >> > >> ends up
> >>>> > >> > > > > > >> > >> > > > > being a
> >>>> > >> > > > > > >> > >> > > > > > full state machine to do right). I
> think
> >>>> we
> >>>> > >> > should
> >>>> > >> > > > > > consider
> >>>> > >> > > > > > >> > >> making
> >>>> > >> > > > > > >> > >> > > > admin
> >>>> > >> > > > > > >> > >> > > > > > commands and ideally as many of the
> >>>> other apis
> >>>> > >> as
> >>>> > >> > > > > > possible
> >>>> > >> > > > > > >> > >> > available
> >>>> > >> > > > > > >> > >> > > on
> >>>> > >> > > > > > >> > >> > > > > all
> >>>> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
> >>>> controller on
> >>>> > >> > the
> >>>> > >> > > > > broker
> >>>> > >> > > > > > >> > side.
> >>>> > >> > > > > > >> > >> > > Perhaps
> >>>> > >> > > > > > >> > >> > > > > > there would be a general way to
> >>>> encapsulate
> >>>> > >> this
> >>>> > >> > > > > > re-routing
> >>>> > >> > > > > > >> > >> > behavior.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 6. We should probably normalize the
> key
> >>>> value
> >>>> > >> > pairs
> >>>> > >> > > > > used
> >>>> > >> > > > > > >> for
> >>>> > >> > > > > > >> > >> > configs
> >>>> > >> > > > > > >> > >> > > > > rather
> >>>> > >> > > > > > >> > >> > > > > > than embedding a new formatting. So
> two
> >>>> strings
> >>>> > >> > > > rather
> >>>> > >> > > > > > than
> >>>> > >> > > > > > >> > one
> >>>> > >> > > > > > >> > >> > with
> >>>> > >> > > > > > >> > >> > > an
> >>>> > >> > > > > > >> > >> > > > > > internal equals sign.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs
> >>>> that the
> >>>> > >> > > > command
> >>>> > >> > > > > > has
> >>>> > >> > > > > > >> > >> begun or
> >>>> > >> > > > > > >> > >> > > > that
> >>>> > >> > > > > > >> > >> > > > > > the command has been completed? It
> is a
> >>>> lot
> >>>> > >> more
> >>>> > >> > > > usable
> >>>> > >> > > > > > if
> >>>> > >> > > > > > >> the
> >>>> > >> > > > > > >> > >> > > command
> >>>> > >> > > > > > >> > >> > > > > has
> >>>> > >> > > > > > >> > >> > > > > > been completed so you know that if
> you
> >>>> create a
> >>>> > >> > > topic
> >>>> > >> > > > > and
> >>>> > >> > > > > > >> then
> >>>> > >> > > > > > >> > >> > > publish
> >>>> > >> > > > > > >> > >> > > > to
> >>>> > >> > > > > > >> > >> > > > > > it you won't get an exception about
> >>>> there being
> >>>> > >> > no
> >>>> > >> > > > such
> >>>> > >> > > > > > >> topic.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
> >>>> duplicate a
> >>>> > >> lot
> >>>> > >> > > of
> >>>> > >> > > > > > stuff
> >>>> > >> > > > > > >> in
> >>>> > >> > > > > > >> > >> the
> >>>> > >> > > > > > >> > >> > > > > metadata
> >>>> > >> > > > > > >> > >> > > > > > request. Is there a reason to give
> back
> >>>> topics
> >>>> > >> > > marked
> >>>> > >> > > > > for
> >>>> > >> > > > > > >> > >> > deletion? I
> >>>> > >> > > > > > >> > >> > > > > feel
> >>>> > >> > > > > > >> > >> > > > > > like if we just make the
> post-condition
> >>>> of the
> >>>> > >> > > delete
> >>>> > >> > > > > > >> command
> >>>> > >> > > > > > >> > be
> >>>> > >> > > > > > >> > >> > that
> >>>> > >> > > > > > >> > >> > > > the
> >>>> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid of
> >>>> the need
> >>>> > >> > for
> >>>> > >> > > > this
> >>>> > >> > > > > > >> right?
> >>>> > >> > > > > > >> > >> And
> >>>> > >> > > > > > >> > >> > it
> >>>> > >> > > > > > >> > >> > > > > will
> >>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 9. Should we consider batching these
> >>>> requests?
> >>>> > >> We
> >>>> > >> > > > have
> >>>> > >> > > > > > >> > generally
> >>>> > >> > > > > > >> > >> > > tried
> >>>> > >> > > > > > >> > >> > > > to
> >>>> > >> > > > > > >> > >> > > > > > allow multiple operations to be
> batched.
> >>>> My
> >>>> > >> > > suspicion
> >>>> > >> > > > > is
> >>>> > >> > > > > > >> that
> >>>> > >> > > > > > >> > >> > without
> >>>> > >> > > > > > >> > >> > > > > this
> >>>> > >> > > > > > >> > >> > > > > > we will get a lot of code that does
> >>>> something
> >>>> > >> > like
> >>>> > >> > > > > > >> > >> > > > > >    for(topic:
> adminClient.listTopics())
> >>>> > >> > > > > > >> > >> > > > > >
>  adminClient.describeTopic(topic)
> >>>> > >> > > > > > >> > >> > > > > > this code will work great when you
> test
> >>>> on 5
> >>>> > >> > topics
> >>>> > >> > > > but
> >>>> > >> > > > > > >> not do
> >>>> > >> > > > > > >> > >> as
> >>>> > >> > > > > > >> > >> > > well
> >>>> > >> > > > > > >> > >> > > > if
> >>>> > >> > > > > > >> > >> > > > > > you have 50k.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss
> how
> >>>> we want
> >>>> > >> to
> >>>> > >> > > > > expose
> >>>> > >> > > > > > a
> >>>> > >> > > > > > >> > >> > > programmatic
> >>>> > >> > > > > > >> > >> > > > > JVM
> >>>> > >> > > > > > >> > >> > > > > > client api for these operations.
> >>>> Currently
> >>>> > >> people
> >>>> > >> > > > rely
> >>>> > >> > > > > on
> >>>> > >> > > > > > >> > >> > AdminUtils
> >>>> > >> > > > > > >> > >> > > > > which
> >>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we
> probably
> >>>> need
> >>>> > >> > > another
> >>>> > >> > > > > > client
> >>>> > >> > > > > > >> > >> under
> >>>> > >> > > > > > >> > >> > > > > clients/
> >>>> > >> > > > > > >> > >> > > > > > that exposes administrative
> >>>> functionality. We
> >>>> > >> > will
> >>>> > >> > > > need
> >>>> > >> > > > > > >> this
> >>>> > >> > > > > > >> > >> just
> >>>> > >> > > > > > >> > >> > to
> >>>> > >> > > > > > >> > >> > > > > > properly test the new apis, I
> suspect. We
> >>>> > >> should
> >>>> > >> > > > figure
> >>>> > >> > > > > > out
> >>>> > >> > > > > > >> > that
> >>>> > >> > > > > > >> > >> > API.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > 11. The other information that would
> be
> >>>> really
> >>>> > >> > > useful
> >>>> > >> > > > > to
> >>>> > >> > > > > > >> get
> >>>> > >> > > > > > >> > >> would
> >>>> > >> > > > > > >> > >> > be
> >>>> > >> > > > > > >> > >> > > > > > information about partitions--how
> much
> >>>> data is
> >>>> > >> in
> >>>> > >> > > the
> >>>> > >> > > > > > >> > partition,
> >>>> > >> > > > > > >> > >> > what
> >>>> > >> > > > > > >> > >> > > > are
> >>>> > >> > > > > > >> > >> > > > > > the segment offsets, what is the
> log-end
> >>>> offset
> >>>> > >> > > (i.e.
> >>>> > >> > > > > > last
> >>>> > >> > > > > > >> > >> offset),
> >>>> > >> > > > > > >> > >> > > > what
> >>>> > >> > > > > > >> > >> > > > > is
> >>>> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think
> that
> >>>> done
> >>>> > >> > right
> >>>> > >> > > > this
> >>>> > >> > > > > > >> would
> >>>> > >> > > > > > >> > be
> >>>> > >> > > > > > >> > >> > the
> >>>> > >> > > > > > >> > >> > > > > > successor to the very awkward
> >>>> OffsetRequest we
> >>>> > >> > have
> >>>> > >> > > > > > today.
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > -Jay
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe
> >>>> Stein <
> >>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
> >>>> > >> > > > > > >> > >> > > > > wrote:
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >>
> >>>> > >> > > > > >
> >>>> > >> > > > >
> >>>> > >> > > >
> >>>> > >> > >
> >>>> > >> >
> >>>> > >>
> >>>>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> > >> > > > > > >> > >> > > > > > > JIRA
> >>>> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> /*******************************************
> >>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
> >>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> >>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
> >>>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> >>>> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> >>>> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> ********************************************/
> >>>> > >> > > > > > >> > >> > > > > > >
> >>>> > >> > > > > > >> > >> > > > > >
> >>>> > >> > > > > > >> > >> > > > >
> >>>> > >> > > > > > >> > >> > > >
> >>>> > >> > > > > > >> > >> > >
> >>>> > >> > > > > > >> > >> >
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >> --
> >>>> > >> > > > > > >> > >> -- Guozhang
> >>>> > >> > > > > > >> > >>
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> > >
> >>>> > >> > > > > > >> >
> >>>> > >> > > > > > >>
> >>>> > >> > > > > > >
> >>>> > >> > > > > > >
> >>>> > >> > > > > >
> >>>> > >> > > > >
> >>>> > >> > > >
> >>>> > >> > >
> >>>> > >> >
> >>>> > >> >
> >>>> > >> >
> >>>> > >> > --
> >>>> > >> > Jeff Holoman
> >>>> > >> > Systems Engineer
> >>>> > >> >
> >>>> > >>
> >>>>
> >>>>
> >>>
> >>>
> >>> --
> >>> -- Guozhang
> >>>
> >>
> >>
> >>
> >> --
> >> -- Guozhang
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Gwen Shapira <gs...@cloudera.com>.
Found KIP-11 (https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface)
It actually specifies changes to the Metadata protocol, so making sure
both KIPs are consistent in this regard will be good.

On Thu, Mar 12, 2015 at 12:21 PM, Gwen Shapira <gs...@cloudera.com> wrote:
> Specifically for ownership, I think the plan is to add ACL (it sounds
> like you are describing ACL) via an external system (Argus, Sentry).
> I remember KIP-11 described this, but I can't find the KIP any longer.
>
> Regardless, I think KIP-4 focuses on getting information that already
> exists from Kafka brokers, not on adding information that perhaps
> should exist but doesn't yet?
>
> Gwen
>
>
>
>
>
> On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <wa...@gmail.com> wrote:
>> Folks,
>>
>> Just want to elaborate a bit more on the create-topic metadata and batching
>> describe-topic based on config / metadata in my previous email as we work
>> on KAFKA-1694. The main motivation is to have some sort of topic management
>> mechanisms, which I think is quite important in a multi-tenant / cloud
>> architecture: today anyone can create topics in a shared Kafka cluster, but
>> there is no concept or "ownership" of topics that are created by different
>> users. For example, at LinkedIn we basically distinguish topic owners via
>> some casual topic name prefix, which is a bit awkward and does not fly as
>> we scale our customers. It would be great to use describe-topics such as:
>>
>> Describe all topics that is created by me.
>>
>> Describe all topics whose retention time is overriden to X.
>>
>> Describe all topics whose writable group include user Y (this is related to
>> authorization), etc..
>>
>> One possible way to achieve this is to add a metadata file in the
>> create-topic request, whose value will also be written ZK as we create the
>> topic; then describe-topics can choose to batch topics based on 1) name
>> regex, 2) config K-V matching, 3) metadata regex, etc.
>>
>> Thoughts?
>>
>> Guozhang
>>
>> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com> wrote:
>>
>>> Thanks for the updated wiki. A few comments below:
>>>
>>> 1. Error description in response: I think if some errorCode could indicate
>>> several different error cases then we should really change it to multiple
>>> codes. In general the errorCode itself would be precise and sufficient for
>>> describing the server side errors.
>>>
>>> 2. Describe topic request: it would be great to go beyond just batching on
>>> topic name regex for this request. For example, a very common use case of
>>> the topic command is to list all topics whose config A's value is B. With
>>> topic name regex then we have to first retrieve __all__ topics's
>>> description info and then filter at the client end, which will be a huge
>>> burden on ZK.
>>>
>>> 3. Config K-Vs in create topic: this is related to the previous point;
>>> maybe we can add another metadata K-V or just a metadata string along side
>>> with config K-V in create topic like we did for offset commit request. This
>>> field can be quite useful in storing information like "owner" of the topic
>>> who issue the create command, etc, which is quite important for a
>>> multi-tenant setting. Then in the describe topic request we can also batch
>>> on regex of the metadata field.
>>>
>>> 4. Today all the admin operations are async in the sense that command will
>>> return once it is written in ZK, and that is why we need extra verification
>>> like testUtil.waitForTopicCreated() / verify partition reassignment
>>> request, etc. With admin requests we could add a flag to enable / disable
>>> synchronous requests; when it is turned on, the response will not return
>>> until the request has been completed. And for async requests we can add a
>>> "token" field in the response, and then only need a general "admin
>>> verification request" with the given token to check if the async request
>>> has been completed.
>>>
>>> 5. +1 for extending Metadata request to include controller / coordinator
>>> information, and then we can remove the ConsumerMetadata / ClusterMetadata
>>> requests.
>>>
>>> Guozhang
>>>
>>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com> wrote:
>>>
>>>> Thanks for sending that out Joe - I don't think I will be able to make
>>>> it today, so if notes can be sent out afterward that would be great.
>>>>
>>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
>>>> > Thanks for sending this out Joe. Looking forward to chatting with
>>>> everyone :)
>>>> >
>>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly> wrote:
>>>> > > Hey, I just sent out a google hangout invite to all pmc, committers
>>>> and
>>>> > > everyone I found working on a KIP. If I missed anyone in the invite
>>>> please
>>>> > > let me know and can update it, np.
>>>> > >
>>>> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can get
>>>> INFRA
>>>> > > help to make a google account so we can manage better?
>>>> > >
>>>> > > To discuss
>>>> > >
>>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>>>> > > in progress and related JIRA that are interdependent and common work.
>>>> > >
>>>> > > ~ Joe Stein
>>>> > >
>>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com>
>>>> wrote:
>>>> > >
>>>> > >> Let's stay on Google hangouts that will also record and make the
>>>> sessions
>>>> > >> available on youtube.
>>>> > >>
>>>> > >> -Jay
>>>> > >>
>>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
>>>> jholoman@cloudera.com>
>>>> > >> wrote:
>>>> > >>
>>>> > >> > Jay / Joe
>>>> > >> >
>>>> > >> > We're happy to send out a Webex for this purpose. We could record
>>>> the
>>>> > >> > sessions if there is interest and publish them out.
>>>> > >> >
>>>> > >> > Thanks
>>>> > >> >
>>>> > >> > Jeff
>>>> > >> >
>>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <ja...@gmail.com>
>>>> wrote:
>>>> > >> >
>>>> > >> > > Let's try to get the technical hang-ups sorted out, though. I
>>>> really
>>>> > >> > think
>>>> > >> > > there is some benefit to live discussion vs writing. I am
>>>> hopeful that
>>>> > >> if
>>>> > >> > > we post instructions and give ourselves a few attempts we can
>>>> get it
>>>> > >> > > working.
>>>> > >> > >
>>>> > >> > > Tuesday at that time would work for me...any objections?
>>>> > >> > >
>>>> > >> > > -Jay
>>>> > >> > >
>>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <joe.stein@stealth.ly
>>>> >
>>>> > >> wrote:
>>>> > >> > >
>>>> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am
>>>> PT
>>>> > >> ????
>>>> > >> > > >
>>>> > >> > > > I don't mind google hangout but there is always some issue or
>>>> > >> whatever
>>>> > >> > so
>>>> > >> > > > we know the apache irc channel works. We can start there and
>>>> see how
>>>> > >> it
>>>> > >> > > > goes? We can pull transcripts too and associate to tickets if
>>>> need be
>>>> > >> > > makes
>>>> > >> > > > it helpful for things.
>>>> > >> > > >
>>>> > >> > > > ~ Joestein
>>>> > >> > > >
>>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
>>>> jay.kreps@gmail.com>
>>>> > >> > wrote:
>>>> > >> > > >
>>>> > >> > > > > We'd talked about doing a Google Hangout to chat about this.
>>>> What
>>>> > >> > about
>>>> > >> > > > > generalizing that a little further...I actually think it
>>>> would be
>>>> > >> > good
>>>> > >> > > > for
>>>> > >> > > > > everyone spending a reasonable chunk of their week on Kafka
>>>> stuff
>>>> > >> to
>>>> > >> > > > maybe
>>>> > >> > > > > sync up once a week. I think we could use time to talk
>>>> through
>>>> > >> design
>>>> > >> > > > > stuff, make sure we are on top of code reviews, talk through
>>>> any
>>>> > >> > tricky
>>>> > >> > > > > issues, etc.
>>>> > >> > > > >
>>>> > >> > > > > We can make it publicly available so that any one can follow
>>>> along
>>>> > >> > who
>>>> > >> > > > > likes.
>>>> > >> > > > >
>>>> > >> > > > > Any interest in doing this? If so I'll try to set it up
>>>> starting
>>>> > >> next
>>>> > >> > > > week.
>>>> > >> > > > >
>>>> > >> > > > > -Jay
>>>> > >> > > > >
>>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
>>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
>>>> > >> > > > >
>>>> > >> > > > > > Hi all,
>>>> > >> > > > > >
>>>> > >> > > > > > I've updated KIP page, fixed / aligned document structure.
>>>> Also I
>>>> > >> > > added
>>>> > >> > > > > > some
>>>> > >> > > > > > very initial proposal for AdminClient so we have something
>>>> to
>>>> > >> start
>>>> > >> > > > from
>>>> > >> > > > > > while
>>>> > >> > > > > > discussing the KIP.
>>>> > >> > > > > >
>>>> > >> > > > > >
>>>> > >> > > > > >
>>>> > >> > > > >
>>>> > >> > > >
>>>> > >> > >
>>>> > >> >
>>>> > >>
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>>>> > >> > > > > >
>>>> > >> > > > > > Thanks,
>>>> > >> > > > > > Andrii Biletskyi
>>>> > >> > > > > >
>>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
>>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
>>>> > >> > > > > >
>>>> > >> > > > > > > Jay,
>>>> > >> > > > > > >
>>>> > >> > > > > > > Re error messages: you are right, in most cases client
>>>> will
>>>> > >> have
>>>> > >> > > > enough
>>>> > >> > > > > > > context to show descriptive error message. My concern is
>>>> that
>>>> > >> we
>>>> > >> > > will
>>>> > >> > > > > > have
>>>> > >> > > > > > > to
>>>> > >> > > > > > > add lots of new error codes for each possible error. Of
>>>> course,
>>>> > >> > we
>>>> > >> > > > > could
>>>> > >> > > > > > > reuse
>>>> > >> > > > > > > some of existing like UknownTopicOrPartitionCode, but we
>>>> will
>>>> > >> > also
>>>> > >> > > > need
>>>> > >> > > > > > to
>>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
>>>> TopicConfigInvalid (both
>>>> > >> > for
>>>> > >> > > > > topic
>>>> > >> > > > > > > name and config, and probably user would like to know
>>>> what
>>>> > >> > exactly
>>>> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
>>>> > >> InternalError
>>>> > >> > > > (e.g.
>>>> > >> > > > > > > zookeeper failure) etc.
>>>> > >> > > > > > > And this is only for TopicCommand, we will also need to
>>>> add
>>>> > >> > similar
>>>> > >> > > > > stuff
>>>> > >> > > > > > > for
>>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up
>>>> with a
>>>> > >> > large
>>>> > >> > > > list
>>>> > >> > > > > > of
>>>> > >> > > > > > > error codes, used only in Admin protocol.
>>>> > >> > > > > > > Having said that, I agree my proposal is not consistent
>>>> with
>>>> > >> > other
>>>> > >> > > > > cases.
>>>> > >> > > > > > > Maybe we can find better solution or something
>>>> in-between.
>>>> > >> > > > > > >
>>>> > >> > > > > > > Re Hangout chat: I think it is a great idea. This way we
>>>> can
>>>> > >> move
>>>> > >> > > on
>>>> > >> > > > > > > faster.
>>>> > >> > > > > > > Let's agree somehow on date/time so people can join.
>>>> Will work
>>>> > >> > for
>>>> > >> > > me
>>>> > >> > > > > > this
>>>> > >> > > > > > > and
>>>> > >> > > > > > > next week almost anytime if agreed in advance.
>>>> > >> > > > > > >
>>>> > >> > > > > > > Thanks,
>>>> > >> > > > > > > Andrii
>>>> > >> > > > > > >
>>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
>>>> > >> jay.kreps@gmail.com>
>>>> > >> > > > > wrote:
>>>> > >> > > > > > >
>>>> > >> > > > > > >> Hey Andrii,
>>>> > >> > > > > > >>
>>>> > >> > > > > > >> Generally we can do good error handling without needing
>>>> custom
>>>> > >> > > > > > server-side
>>>> > >> > > > > > >> messages. I.e. generally the client has the context to
>>>> know
>>>> > >> that
>>>> > >> > > if
>>>> > >> > > > it
>>>> > >> > > > > > got
>>>> > >> > > > > > >> an error that the topic doesn't exist to say "Topic X
>>>> doesn't
>>>> > >> > > exist"
>>>> > >> > > > > > >> rather
>>>> > >> > > > > > >> than "error code 14" (or whatever). Maybe there are
>>>> specific
>>>> > >> > cases
>>>> > >> > > > > where
>>>> > >> > > > > > >> this is hard? If we want to add server-side error
>>>> messages we
>>>> > >> > > really
>>>> > >> > > > > do
>>>> > >> > > > > > >> need to do this in a consistent way across the protocol.
>>>> > >> > > > > > >>
>>>> > >> > > > > > >> I still have a bunch of open questions here from my
>>>> previous
>>>> > >> > > list. I
>>>> > >> > > > > > will
>>>> > >> > > > > > >> be out for the next few days for Strata though. Maybe
>>>> we could
>>>> > >> > do
>>>> > >> > > a
>>>> > >> > > > > > Google
>>>> > >> > > > > > >> Hangout chat on any open issues some time towards the
>>>> end of
>>>> > >> > next
>>>> > >> > > > week
>>>> > >> > > > > > for
>>>> > >> > > > > > >> anyone interested in this ticket? I have a feeling that
>>>> might
>>>> > >> > > > progress
>>>> > >> > > > > > >> things a little faster than email--I think we could talk
>>>> > >> through
>>>> > >> > > > those
>>>> > >> > > > > > >> issues I brought up fairly quickly...
>>>> > >> > > > > > >>
>>>> > >> > > > > > >> -Jay
>>>> > >> > > > > > >>
>>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
>>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
>>>> > >> > > > > > >>
>>>> > >> > > > > > >> > Hi all,
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >> > I'm trying to address some of the issues which were
>>>> > >> mentioned
>>>> > >> > > > > earlier
>>>> > >> > > > > > >> about
>>>> > >> > > > > > >> > Admin RQ/RP format. One of those was about batching
>>>> > >> > operations.
>>>> > >> > > > What
>>>> > >> > > > > > if
>>>> > >> > > > > > >> we
>>>> > >> > > > > > >> > follow TopicCommand approach and let people specify
>>>> > >> topic-name
>>>> > >> > > by
>>>> > >> > > > > > >> regexp -
>>>> > >> > > > > > >> > would that cover most of the use cases?
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >> > Secondly, is what information should we generally
>>>> provide in
>>>> > >> > > Admin
>>>> > >> > > > > > >> > responses.
>>>> > >> > > > > > >> > I realize that Admin commands don't imply they will
>>>> be used
>>>> > >> > only
>>>> > >> > > > in
>>>> > >> > > > > > CLI
>>>> > >> > > > > > >> > but,
>>>> > >> > > > > > >> > it seems to me, CLI is a very important client of this
>>>> > >> > feature.
>>>> > >> > > In
>>>> > >> > > > > > this
>>>> > >> > > > > > >> > case,
>>>> > >> > > > > > >> > seems logical, we would like to provide users with
>>>> rich
>>>> > >> > > experience
>>>> > >> > > > > in
>>>> > >> > > > > > >> terms
>>>> > >> > > > > > >> > of
>>>> > >> > > > > > >> > getting results / errors of the executed commands.
>>>> Usually
>>>> > >> we
>>>> > >> > > > supply
>>>> > >> > > > > > >> with
>>>> > >> > > > > > >> > responses only errorCode, which looks very limiting,
>>>> in case
>>>> > >> > of
>>>> > >> > > > CLI
>>>> > >> > > > > we
>>>> > >> > > > > > >> may
>>>> > >> > > > > > >> > want to print human readable error description.
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >> > So, taking into account previous item about batching,
>>>> what
>>>> > >> do
>>>> > >> > > you
>>>> > >> > > > > > think
>>>> > >> > > > > > >> > about
>>>> > >> > > > > > >> > having smth like:
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >> > ('create' doesn't support regexp)
>>>> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions Replicas
>>>> > >> > > > > ReplicaAssignment
>>>> > >> > > > > > >> > [Config]
>>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
>>>> > >> > > > > > >> >   ErrorCode => int16
>>>> > >> > > > > > >> >   ErrorDescription => string (empty if successful)
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
>>>> > >> > > ReplicaAssignment
>>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
>>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
>>>> ErrorDescription]
>>>> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
>>>> > >> > > > > > >> >   CommandErrorCode => int16
>>>> > >> > > > > > >> >   CommandErrorDescription => string (nonempty in case
>>>> of
>>>> > >> fatal
>>>> > >> > > > > error,
>>>> > >> > > > > > >> e.g.
>>>> > >> > > > > > >> > we couldn't get topics by regexp)
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
>>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName TopicDescription
>>>> > >> ErrorCode
>>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
>>>> CommandErrorDescription
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >> > Also, any thoughts about our discussion regarding
>>>> re-routing
>>>> > >> > > > > facility?
>>>> > >> > > > > > >> In
>>>> > >> > > > > > >> > my
>>>> > >> > > > > > >> > understanding, it is like between augmenting
>>>> > >> > > TopicMetadataRequest
>>>> > >> > > > > > >> > (to include at least controllerId) and implementing
>>>> new
>>>> > >> > generic
>>>> > >> > > > > > >> re-routing
>>>> > >> > > > > > >> > facility so sending messages to controller will be
>>>> handled
>>>> > >> by
>>>> > >> > > it.
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >> > Thanks,
>>>> > >> > > > > > >> > Andrii Biletskyi
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
>>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >> > > @Guozhang:
>>>> > >> > > > > > >> > > Thanks for your comments, I've answered some of
>>>> those. The
>>>> > >> > > main
>>>> > >> > > > > > thing
>>>> > >> > > > > > >> is
>>>> > >> > > > > > >> > > having merged request for
>>>> create-alter-delete-describe - I
>>>> > >> > > have
>>>> > >> > > > > some
>>>> > >> > > > > > >> > > concerns about this approach.
>>>> > >> > > > > > >> > >
>>>> > >> > > > > > >> > > @*Jay*:
>>>> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is also
>>>> one of
>>>> > >> > the
>>>> > >> > > > > > >> concerns.
>>>> > >> > > > > > >> > We
>>>> > >> > > > > > >> > > can solve it if we implement re-routing facility.
>>>> But I
>>>> > >> > agree
>>>> > >> > > > with
>>>> > >> > > > > > >> > > Guozhang - it will make clients' internals a little
>>>> bit
>>>> > >> > easier
>>>> > >> > > > but
>>>> > >> > > > > > >> this
>>>> > >> > > > > > >> > > seems to be a complex logic to implement and
>>>> support then.
>>>> > >> > > > > > Especially
>>>> > >> > > > > > >> for
>>>> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing later
>>>> for
>>>> > >> these
>>>> > >> > > > > > >> requests).
>>>> > >> > > > > > >> > > Also people will tend to avoid this re-routing
>>>> facility
>>>> > >> and
>>>> > >> > > hold
>>>> > >> > > > > > local
>>>> > >> > > > > > >> > > cluster cache to ensure their high-priority requests
>>>> > >> (which
>>>> > >> > > some
>>>> > >> > > > > of
>>>> > >> > > > > > >> the
>>>> > >> > > > > > >> > > admin requests are) not sent to some busy broker
>>>> where
>>>> > >> they
>>>> > >> > > wait
>>>> > >> > > > > to
>>>> > >> > > > > > be
>>>> > >> > > > > > >> > > routed to the correct one.
>>>> > >> > > > > > >> > > As pointed out by Jun here (
>>>> > >> > > > > > >> > >
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >>
>>>> > >> > > > > >
>>>> > >> > > > >
>>>> > >> > > >
>>>> > >> > >
>>>> > >> >
>>>> > >>
>>>> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
>>>> > >> > > > > > >> > )
>>>> > >> > > > > > >> > > to solve the issue we might introduce a message
>>>> type to
>>>> > >> get
>>>> > >> > > > > cluster
>>>> > >> > > > > > >> > state.
>>>> > >> > > > > > >> > > But I agree we can just update
>>>> TopicMetadataResponse to
>>>> > >> > > include
>>>> > >> > > > > > >> > > controllerId (and probably smth else).
>>>> > >> > > > > > >> > > What are you thougths?
>>>> > >> > > > > > >> > >
>>>> > >> > > > > > >> > > Thanks,
>>>> > >> > > > > > >> > > Andrii
>>>> > >> > > > > > >> > >
>>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
>>>> > >> > > > > wangguoz@gmail.com>
>>>> > >> > > > > > >> > wrote:
>>>> > >> > > > > > >> > >
>>>> > >> > > > > > >> > >> I think for the topics commands we can actually
>>>> merge
>>>> > >> > > > > > >> > >> create/alter/delete/describe as one request type
>>>> since
>>>> > >> > their
>>>> > >> > > > > > formats
>>>> > >> > > > > > >> are
>>>> > >> > > > > > >> > >> very much similar, and keep list-topics and others
>>>> like
>>>> > >> > > > > > >> > >> partition-reassignment / preferred-leader-election
>>>> as
>>>> > >> > > separate
>>>> > >> > > > > > >> request
>>>> > >> > > > > > >> > >> types, I also left some other comments on the RB (
>>>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
>>>> > >> > > > > > >> > >>
>>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
>>>> > >> > > > jay.kreps@gmail.com>
>>>> > >> > > > > > >> wrote:
>>>> > >> > > > > > >> > >>
>>>> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to just
>>>> have
>>>> > >> one
>>>> > >> > > "do
>>>> > >> > > > > > admin
>>>> > >> > > > > > >> > >> stuff"
>>>> > >> > > > > > >> > >> > command that has the union of all parameters.
>>>> > >> > > > > > >> > >> >
>>>> > >> > > > > > >> > >> > What I am saying is that command line tools are
>>>> one
>>>> > >> > client
>>>> > >> > > of
>>>> > >> > > > > the
>>>> > >> > > > > > >> > >> > administrative apis, but these will be used in a
>>>> number
>>>> > >> > of
>>>> > >> > > > > > >> scenarios
>>>> > >> > > > > > >> > so
>>>> > >> > > > > > >> > >> > they should make logical sense even in the
>>>> absence of
>>>> > >> the
>>>> > >> > > > > command
>>>> > >> > > > > > >> line
>>>> > >> > > > > > >> > >> > tool. Hence comments like trying to clarify the
>>>> > >> > > relationship
>>>> > >> > > > > > >> between
>>>> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds
>>>> of
>>>> > >> things
>>>> > >> > > > > really
>>>> > >> > > > > > >> need
>>>> > >> > > > > > >> > >> to be
>>>> > >> > > > > > >> > >> > thought through.
>>>> > >> > > > > > >> > >> >
>>>> > >> > > > > > >> > >> > Hope that makes sense.
>>>> > >> > > > > > >> > >> >
>>>> > >> > > > > > >> > >> > -Jay
>>>> > >> > > > > > >> > >> >
>>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
>>>> Biletskyi <
>>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
>>>> > >> > > > > > >> > >> >
>>>> > >> > > > > > >> > >> > > Jay,
>>>> > >> > > > > > >> > >> > >
>>>> > >> > > > > > >> > >> > > Thanks for answering. You understood
>>>> correctly, most
>>>> > >> of
>>>> > >> > > my
>>>> > >> > > > > > >> comments
>>>> > >> > > > > > >> > >> were
>>>> > >> > > > > > >> > >> > > related to your point 1) - about "well
>>>> thought-out"
>>>> > >> > apis.
>>>> > >> > > > > Also,
>>>> > >> > > > > > >> yes,
>>>> > >> > > > > > >> > >> as I
>>>> > >> > > > > > >> > >> > > understood we would like to introduce a single
>>>> > >> unified
>>>> > >> > > CLI
>>>> > >> > > > > tool
>>>> > >> > > > > > >> with
>>>> > >> > > > > > >> > >> > > centralized server-side request handling for
>>>> lots of
>>>> > >> > > > existing
>>>> > >> > > > > > >> ones
>>>> > >> > > > > > >> > >> (incl.
>>>> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
>>>> > >> ReassignPartitions,
>>>> > >> > > smth
>>>> > >> > > > > > else
>>>> > >> > > > > > >> if
>>>> > >> > > > > > >> > >> added
>>>> > >> > > > > > >> > >> > > in future). In our previous discussion (
>>>> > >> > > > > > >> > >> > >
>>>> https://issues.apache.org/jira/browse/KAFKA-1694)
>>>> > >> > people
>>>> > >> > > > > said
>>>> > >> > > > > > >> > they'd
>>>> > >> > > > > > >> > >> > > rather
>>>> > >> > > > > > >> > >> > > have a separate message for each command, so,
>>>> yes,
>>>> > >> this
>>>> > >> > > > way I
>>>> > >> > > > > > >> came
>>>> > >> > > > > > >> > to
>>>> > >> > > > > > >> > >> 1-1
>>>> > >> > > > > > >> > >> > > mapping between commands in the tool and
>>>> protocol
>>>> > >> > > > additions.
>>>> > >> > > > > > But
>>>> > >> > > > > > >> I
>>>> > >> > > > > > >> > >> might
>>>> > >> > > > > > >> > >> > be
>>>> > >> > > > > > >> > >> > > wrong.
>>>> > >> > > > > > >> > >> > > At the end I just try to start discussion how
>>>> at
>>>> > >> least
>>>> > >> > > > > > generally
>>>> > >> > > > > > >> > this
>>>> > >> > > > > > >> > >> > > protocol should look like.
>>>> > >> > > > > > >> > >> > >
>>>> > >> > > > > > >> > >> > > Thanks,
>>>> > >> > > > > > >> > >> > > Andrii
>>>> > >> > > > > > >> > >> > >
>>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
>>>> > >> > > > > > jay.kreps@gmail.com
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >> > >> wrote:
>>>> > >> > > > > > >> > >> > >
>>>> > >> > > > > > >> > >> > > > Hey Andrii,
>>>> > >> > > > > > >> > >> > > >
>>>> > >> > > > > > >> > >> > > > To answer your earlier question we just
>>>> really
>>>> > >> can't
>>>> > >> > be
>>>> > >> > > > > > adding
>>>> > >> > > > > > >> any
>>>> > >> > > > > > >> > >> more
>>>> > >> > > > > > >> > >> > > > scala protocol objects. These things are
>>>> super hard
>>>> > >> > to
>>>> > >> > > > > > maintain
>>>> > >> > > > > > >> > >> because
>>>> > >> > > > > > >> > >> > > > they hand code the byte parsing and don't
>>>> have good
>>>> > >> > > > > > versioning
>>>> > >> > > > > > >> > >> support.
>>>> > >> > > > > > >> > >> > > > Since we are already planning on converting
>>>> we
>>>> > >> > > definitely
>>>> > >> > > > > > don't
>>>> > >> > > > > > >> > >> want to
>>>> > >> > > > > > >> > >> > > add
>>>> > >> > > > > > >> > >> > > > a ton more of these--they are total tech
>>>> debt.
>>>> > >> > > > > > >> > >> > > >
>>>> > >> > > > > > >> > >> > > > What does it mean that the changes are
>>>> isolated
>>>> > >> from
>>>> > >> > > the
>>>> > >> > > > > > >> current
>>>> > >> > > > > > >> > >> code
>>>> > >> > > > > > >> > >> > > base?
>>>> > >> > > > > > >> > >> > > >
>>>> > >> > > > > > >> > >> > > > I actually didn't understand the remaining
>>>> > >> comments,
>>>> > >> > > > which
>>>> > >> > > > > of
>>>> > >> > > > > > >> the
>>>> > >> > > > > > >> > >> > points
>>>> > >> > > > > > >> > >> > > > are you responding to?
>>>> > >> > > > > > >> > >> > > >
>>>> > >> > > > > > >> > >> > > > Maybe one sticking point here is that it
>>>> seems like
>>>> > >> > you
>>>> > >> > > > > want
>>>> > >> > > > > > to
>>>> > >> > > > > > >> > make
>>>> > >> > > > > > >> > >> > some
>>>> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1 mapping
>>>> > >> between
>>>> > >> > > > > > commands
>>>> > >> > > > > > >> you
>>>> > >> > > > > > >> > >> > > imagine
>>>> > >> > > > > > >> > >> > > > in the tool and protocol additions. I want
>>>> to make
>>>> > >> > sure
>>>> > >> > > > we
>>>> > >> > > > > > >> don't
>>>> > >> > > > > > >> > do
>>>> > >> > > > > > >> > >> > that.
>>>> > >> > > > > > >> > >> > > > The protocol needs to be really really well
>>>> thought
>>>> > >> > out
>>>> > >> > > > > > against
>>>> > >> > > > > > >> > many
>>>> > >> > > > > > >> > >> > use
>>>> > >> > > > > > >> > >> > > > cases so it should make perfect logical
>>>> sense in
>>>> > >> the
>>>> > >> > > > > absence
>>>> > >> > > > > > of
>>>> > >> > > > > > >> > >> knowing
>>>> > >> > > > > > >> > >> > > the
>>>> > >> > > > > > >> > >> > > > command line tool, right?
>>>> > >> > > > > > >> > >> > > >
>>>> > >> > > > > > >> > >> > > > -Jay
>>>> > >> > > > > > >> > >> > > >
>>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii
>>>> Biletskyi
>>>> > >> <
>>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
>>>> > >> > > > > > >> > >> > > >
>>>> > >> > > > > > >> > >> > > > > Hey Jay,
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > > > I would like to continue this discussion
>>>> as it
>>>> > >> seem
>>>> > >> > > > there
>>>> > >> > > > > > is
>>>> > >> > > > > > >> no
>>>> > >> > > > > > >> > >> > > progress
>>>> > >> > > > > > >> > >> > > > > here.
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > > > First of all, could you please explain
>>>> what did
>>>> > >> you
>>>> > >> > > > mean
>>>> > >> > > > > in
>>>> > >> > > > > > >> 2?
>>>> > >> > > > > > >> > How
>>>> > >> > > > > > >> > >> > > > exactly
>>>> > >> > > > > > >> > >> > > > > are we going to migrate to the new java
>>>> protocol
>>>> > >> > > > > > definitions.
>>>> > >> > > > > > >> > And
>>>> > >> > > > > > >> > >> why
>>>> > >> > > > > > >> > >> > > > it's
>>>> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > > > I agree with you, this feature includes
>>>> lots of
>>>> > >> > > stuff,
>>>> > >> > > > > but
>>>> > >> > > > > > >> > >> thankfully
>>>> > >> > > > > > >> > >> > > > > almost all changes are isolated from the
>>>> current
>>>> > >> > code
>>>> > >> > > > > base,
>>>> > >> > > > > > >> > >> > > > > so the main thing, I think, we need to
>>>> agree is
>>>> > >> > RQ/RP
>>>> > >> > > > > > format.
>>>> > >> > > > > > >> > >> > > > > So how can we start discussion about the
>>>> concrete
>>>> > >> > > > > messages
>>>> > >> > > > > > >> > format?
>>>> > >> > > > > > >> > >> > > > > Can we take (
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > >
>>>> > >> > > > > > >> > >> > >
>>>> > >> > > > > > >> > >> >
>>>> > >> > > > > > >> > >>
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >>
>>>> > >> > > > > >
>>>> > >> > > > >
>>>> > >> > > >
>>>> > >> > >
>>>> > >> >
>>>> > >>
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
>>>> > >> > > > > > >> > >> > > > > )
>>>> > >> > > > > > >> > >> > > > > as starting point?
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > > > We had some doubts earlier whether it worth
>>>> > >> > > introducing
>>>> > >> > > > > one
>>>> > >> > > > > > >> > >> generic
>>>> > >> > > > > > >> > >> > > Admin
>>>> > >> > > > > > >> > >> > > > > Request for all commands (
>>>> > >> > > > > > >> > >> > > >
>>>> https://issues.apache.org/jira/browse/KAFKA-1694
>>>> > >> > > > > > >> > >> > > > > )
>>>> > >> > > > > > >> > >> > > > > but then everybody agreed it would be
>>>> better to
>>>> > >> > have
>>>> > >> > > > > > separate
>>>> > >> > > > > > >> > >> message
>>>> > >> > > > > > >> > >> > > for
>>>> > >> > > > > > >> > >> > > > > each admin command. The Request part is
>>>> really
>>>> > >> > > dictated
>>>> > >> > > > > > from
>>>> > >> > > > > > >> the
>>>> > >> > > > > > >> > >> > > command
>>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so
>>>> the
>>>> > >> > proposed
>>>> > >> > > > > > version
>>>> > >> > > > > > >> > >> should
>>>> > >> > > > > > >> > >> > be
>>>> > >> > > > > > >> > >> > > > > fine (let's put aside for now remarks about
>>>> > >> > Optional
>>>> > >> > > > > type,
>>>> > >> > > > > > >> > >> batching,
>>>> > >> > > > > > >> > >> > > > > configs normalization - I agree with all of
>>>> > >> them).
>>>> > >> > > > > > >> > >> > > > > So the second part is Response. I see
>>>> there are
>>>> > >> two
>>>> > >> > > > cases
>>>> > >> > > > > > >> here.
>>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ;
>>>> b)
>>>> > >> "Get"
>>>> > >> > > > > > requests -
>>>> > >> > > > > > >> > >> > > > > List/Describe...
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > > > a) should only hold request result
>>>> (regardless
>>>> > >> what
>>>> > >> > > we
>>>> > >> > > > > > decide
>>>> > >> > > > > > >> > >> about
>>>> > >> > > > > > >> > >> > > > > blocking/non-blocking commands execution).
>>>> > >> > > > > > >> > >> > > > > Usually we provide error code in response
>>>> but
>>>> > >> since
>>>> > >> > > we
>>>> > >> > > > > will
>>>> > >> > > > > > >> use
>>>> > >> > > > > > >> > >> this
>>>> > >> > > > > > >> > >> > in
>>>> > >> > > > > > >> > >> > > > > interactive shell we need some human
>>>> readable
>>>> > >> error
>>>> > >> > > > > > >> description
>>>> > >> > > > > > >> > -
>>>> > >> > > > > > >> > >> so
>>>> > >> > > > > > >> > >> > I
>>>> > >> > > > > > >> > >> > > > > added errorDesription field where you can
>>>> at
>>>> > >> least
>>>> > >> > > > leave
>>>> > >> > > > > > >> > >> > > > > exception.getMessage.
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > > > b) in addition to previous item message
>>>> should
>>>> > >> hold
>>>> > >> > > > > command
>>>> > >> > > > > > >> > >> specific
>>>> > >> > > > > > >> > >> > > > > response data. We can discuss in detail
>>>> each of
>>>> > >> > them
>>>> > >> > > > but
>>>> > >> > > > > > >> let's
>>>> > >> > > > > > >> > for
>>>> > >> > > > > > >> > >> > now
>>>> > >> > > > > > >> > >> > > > > agree about the overall pattern.
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > > > Thanks,
>>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps
>>>> <
>>>> > >> > > > > > >> jay.kreps@gmail.com
>>>> > >> > > > > > >> > >
>>>> > >> > > > > > >> > >> > > wrote:
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > > > > Hey Joe,
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > This is great. A few comments on KIP-4
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > 1. This is much needed functionality,
>>>> but there
>>>> > >> > > are a
>>>> > >> > > > > lot
>>>> > >> > > > > > >> of
>>>> > >> > > > > > >> > >> the so
>>>> > >> > > > > > >> > >> > > > let's
>>>> > >> > > > > > >> > >> > > > > > really think these protocols through. We
>>>> really
>>>> > >> > > want
>>>> > >> > > > to
>>>> > >> > > > > > >> end up
>>>> > >> > > > > > >> > >> > with a
>>>> > >> > > > > > >> > >> > > > set
>>>> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis.
>>>> For this
>>>> > >> > > > reason I
>>>> > >> > > > > > >> think
>>>> > >> > > > > > >> > >> it is
>>>> > >> > > > > > >> > >> > > > > really
>>>> > >> > > > > > >> > >> > > > > > important to think through the end state
>>>> even
>>>> > >> if
>>>> > >> > > that
>>>> > >> > > > > > >> includes
>>>> > >> > > > > > >> > >> APIs
>>>> > >> > > > > > >> > >> > > we
>>>> > >> > > > > > >> > >> > > > > > won't implement in the first phase.
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait until
>>>> we
>>>> > >> have
>>>> > >> > > > > switched
>>>> > >> > > > > > >> the
>>>> > >> > > > > > >> > >> > server
>>>> > >> > > > > > >> > >> > > > over
>>>> > >> > > > > > >> > >> > > > > > to the new java protocol definitions. If
>>>> we add
>>>> > >> > > > upteen
>>>> > >> > > > > > >> more ad
>>>> > >> > > > > > >> > >> hoc
>>>> > >> > > > > > >> > >> > > > scala
>>>> > >> > > > > > >> > >> > > > > > objects that is just generating more
>>>> work for
>>>> > >> the
>>>> > >> > > > > > >> conversion
>>>> > >> > > > > > >> > we
>>>> > >> > > > > > >> > >> > know
>>>> > >> > > > > > >> > >> > > we
>>>> > >> > > > > > >> > >> > > > > > have to do.
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new type of
>>>> > >> > optional
>>>> > >> > > > > > >> parameter.
>>>> > >> > > > > > >> > >> This
>>>> > >> > > > > > >> > >> > is
>>>> > >> > > > > > >> > >> > > > > > inconsistent with everything else in the
>>>> > >> protocol
>>>> > >> > > > where
>>>> > >> > > > > > we
>>>> > >> > > > > > >> use
>>>> > >> > > > > > >> > >> -1
>>>> > >> > > > > > >> > >> > or
>>>> > >> > > > > > >> > >> > > > some
>>>> > >> > > > > > >> > >> > > > > > other marker value. You could argue
>>>> either way
>>>> > >> > but
>>>> > >> > > > > let's
>>>> > >> > > > > > >> stick
>>>> > >> > > > > > >> > >> with
>>>> > >> > > > > > >> > >> > > > that
>>>> > >> > > > > > >> > >> > > > > > for consistency. For clients that
>>>> implemented
>>>> > >> the
>>>> > >> > > > > > protocol
>>>> > >> > > > > > >> in
>>>> > >> > > > > > >> > a
>>>> > >> > > > > > >> > >> > > better
>>>> > >> > > > > > >> > >> > > > > way
>>>> > >> > > > > > >> > >> > > > > > than our scala code these basic
>>>> primitives are
>>>> > >> > hard
>>>> > >> > > > to
>>>> > >> > > > > > >> change.
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
>>>> duplicate
>>>> > >> > > > > > >> > TopicMetadataRequest
>>>> > >> > > > > > >> > >> > > which
>>>> > >> > > > > > >> > >> > > > > has
>>>> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I think
>>>> we
>>>> > >> > should
>>>> > >> > > > > rename
>>>> > >> > > > > > >> that
>>>> > >> > > > > > >> > >> > > request
>>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
>>>> > >> MetadataRequest)
>>>> > >> > > and
>>>> > >> > > > > > >> include
>>>> > >> > > > > > >> > >> the id
>>>> > >> > > > > > >> > >> > > of
>>>> > >> > > > > > >> > >> > > > > the
>>>> > >> > > > > > >> > >> > > > > > controller. Or are there other things we
>>>> could
>>>> > >> > add
>>>> > >> > > > > here?
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to make a
>>>> lot of
>>>> > >> > > > requests
>>>> > >> > > > > > that
>>>> > >> > > > > > >> > can
>>>> > >> > > > > > >> > >> > only
>>>> > >> > > > > > >> > >> > > go
>>>> > >> > > > > > >> > >> > > > > to
>>>> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of
>>>> burden for
>>>> > >> > > > client
>>>> > >> > > > > > >> > >> > > implementations
>>>> > >> > > > > > >> > >> > > > > (it
>>>> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can fail
>>>> in many
>>>> > >> > > parts
>>>> > >> > > > > so
>>>> > >> > > > > > it
>>>> > >> > > > > > >> > >> ends up
>>>> > >> > > > > > >> > >> > > > > being a
>>>> > >> > > > > > >> > >> > > > > > full state machine to do right). I think
>>>> we
>>>> > >> > should
>>>> > >> > > > > > consider
>>>> > >> > > > > > >> > >> making
>>>> > >> > > > > > >> > >> > > > admin
>>>> > >> > > > > > >> > >> > > > > > commands and ideally as many of the
>>>> other apis
>>>> > >> as
>>>> > >> > > > > > possible
>>>> > >> > > > > > >> > >> > available
>>>> > >> > > > > > >> > >> > > on
>>>> > >> > > > > > >> > >> > > > > all
>>>> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
>>>> controller on
>>>> > >> > the
>>>> > >> > > > > broker
>>>> > >> > > > > > >> > side.
>>>> > >> > > > > > >> > >> > > Perhaps
>>>> > >> > > > > > >> > >> > > > > > there would be a general way to
>>>> encapsulate
>>>> > >> this
>>>> > >> > > > > > re-routing
>>>> > >> > > > > > >> > >> > behavior.
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > 6. We should probably normalize the key
>>>> value
>>>> > >> > pairs
>>>> > >> > > > > used
>>>> > >> > > > > > >> for
>>>> > >> > > > > > >> > >> > configs
>>>> > >> > > > > > >> > >> > > > > rather
>>>> > >> > > > > > >> > >> > > > > > than embedding a new formatting. So two
>>>> strings
>>>> > >> > > > rather
>>>> > >> > > > > > than
>>>> > >> > > > > > >> > one
>>>> > >> > > > > > >> > >> > with
>>>> > >> > > > > > >> > >> > > an
>>>> > >> > > > > > >> > >> > > > > > internal equals sign.
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs
>>>> that the
>>>> > >> > > > command
>>>> > >> > > > > > has
>>>> > >> > > > > > >> > >> begun or
>>>> > >> > > > > > >> > >> > > > that
>>>> > >> > > > > > >> > >> > > > > > the command has been completed? It is a
>>>> lot
>>>> > >> more
>>>> > >> > > > usable
>>>> > >> > > > > > if
>>>> > >> > > > > > >> the
>>>> > >> > > > > > >> > >> > > command
>>>> > >> > > > > > >> > >> > > > > has
>>>> > >> > > > > > >> > >> > > > > > been completed so you know that if you
>>>> create a
>>>> > >> > > topic
>>>> > >> > > > > and
>>>> > >> > > > > > >> then
>>>> > >> > > > > > >> > >> > > publish
>>>> > >> > > > > > >> > >> > > > to
>>>> > >> > > > > > >> > >> > > > > > it you won't get an exception about
>>>> there being
>>>> > >> > no
>>>> > >> > > > such
>>>> > >> > > > > > >> topic.
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
>>>> duplicate a
>>>> > >> lot
>>>> > >> > > of
>>>> > >> > > > > > stuff
>>>> > >> > > > > > >> in
>>>> > >> > > > > > >> > >> the
>>>> > >> > > > > > >> > >> > > > > metadata
>>>> > >> > > > > > >> > >> > > > > > request. Is there a reason to give back
>>>> topics
>>>> > >> > > marked
>>>> > >> > > > > for
>>>> > >> > > > > > >> > >> > deletion? I
>>>> > >> > > > > > >> > >> > > > > feel
>>>> > >> > > > > > >> > >> > > > > > like if we just make the post-condition
>>>> of the
>>>> > >> > > delete
>>>> > >> > > > > > >> command
>>>> > >> > > > > > >> > be
>>>> > >> > > > > > >> > >> > that
>>>> > >> > > > > > >> > >> > > > the
>>>> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid of
>>>> the need
>>>> > >> > for
>>>> > >> > > > this
>>>> > >> > > > > > >> right?
>>>> > >> > > > > > >> > >> And
>>>> > >> > > > > > >> > >> > it
>>>> > >> > > > > > >> > >> > > > > will
>>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > 9. Should we consider batching these
>>>> requests?
>>>> > >> We
>>>> > >> > > > have
>>>> > >> > > > > > >> > generally
>>>> > >> > > > > > >> > >> > > tried
>>>> > >> > > > > > >> > >> > > > to
>>>> > >> > > > > > >> > >> > > > > > allow multiple operations to be batched.
>>>> My
>>>> > >> > > suspicion
>>>> > >> > > > > is
>>>> > >> > > > > > >> that
>>>> > >> > > > > > >> > >> > without
>>>> > >> > > > > > >> > >> > > > > this
>>>> > >> > > > > > >> > >> > > > > > we will get a lot of code that does
>>>> something
>>>> > >> > like
>>>> > >> > > > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
>>>> > >> > > > > > >> > >> > > > > >       adminClient.describeTopic(topic)
>>>> > >> > > > > > >> > >> > > > > > this code will work great when you test
>>>> on 5
>>>> > >> > topics
>>>> > >> > > > but
>>>> > >> > > > > > >> not do
>>>> > >> > > > > > >> > >> as
>>>> > >> > > > > > >> > >> > > well
>>>> > >> > > > > > >> > >> > > > if
>>>> > >> > > > > > >> > >> > > > > > you have 50k.
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss how
>>>> we want
>>>> > >> to
>>>> > >> > > > > expose
>>>> > >> > > > > > a
>>>> > >> > > > > > >> > >> > > programmatic
>>>> > >> > > > > > >> > >> > > > > JVM
>>>> > >> > > > > > >> > >> > > > > > client api for these operations.
>>>> Currently
>>>> > >> people
>>>> > >> > > > rely
>>>> > >> > > > > on
>>>> > >> > > > > > >> > >> > AdminUtils
>>>> > >> > > > > > >> > >> > > > > which
>>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we probably
>>>> need
>>>> > >> > > another
>>>> > >> > > > > > client
>>>> > >> > > > > > >> > >> under
>>>> > >> > > > > > >> > >> > > > > clients/
>>>> > >> > > > > > >> > >> > > > > > that exposes administrative
>>>> functionality. We
>>>> > >> > will
>>>> > >> > > > need
>>>> > >> > > > > > >> this
>>>> > >> > > > > > >> > >> just
>>>> > >> > > > > > >> > >> > to
>>>> > >> > > > > > >> > >> > > > > > properly test the new apis, I suspect. We
>>>> > >> should
>>>> > >> > > > figure
>>>> > >> > > > > > out
>>>> > >> > > > > > >> > that
>>>> > >> > > > > > >> > >> > API.
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > 11. The other information that would be
>>>> really
>>>> > >> > > useful
>>>> > >> > > > > to
>>>> > >> > > > > > >> get
>>>> > >> > > > > > >> > >> would
>>>> > >> > > > > > >> > >> > be
>>>> > >> > > > > > >> > >> > > > > > information about partitions--how much
>>>> data is
>>>> > >> in
>>>> > >> > > the
>>>> > >> > > > > > >> > partition,
>>>> > >> > > > > > >> > >> > what
>>>> > >> > > > > > >> > >> > > > are
>>>> > >> > > > > > >> > >> > > > > > the segment offsets, what is the log-end
>>>> offset
>>>> > >> > > (i.e.
>>>> > >> > > > > > last
>>>> > >> > > > > > >> > >> offset),
>>>> > >> > > > > > >> > >> > > > what
>>>> > >> > > > > > >> > >> > > > > is
>>>> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think that
>>>> done
>>>> > >> > right
>>>> > >> > > > this
>>>> > >> > > > > > >> would
>>>> > >> > > > > > >> > be
>>>> > >> > > > > > >> > >> > the
>>>> > >> > > > > > >> > >> > > > > > successor to the very awkward
>>>> OffsetRequest we
>>>> > >> > have
>>>> > >> > > > > > today.
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > -Jay
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe
>>>> Stein <
>>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
>>>> > >> > > > > > >> > >> > > > > wrote:
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
>>>> > >> > > > > > >> > >> > > > > > >
>>>> > >> > > > > > >> > >> > > > > > >
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > >
>>>> > >> > > > > > >> > >> > >
>>>> > >> > > > > > >> > >> >
>>>> > >> > > > > > >> > >>
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >>
>>>> > >> > > > > >
>>>> > >> > > > >
>>>> > >> > > >
>>>> > >> > >
>>>> > >> >
>>>> > >>
>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>>>> > >> > > > > > >> > >> > > > > > >
>>>> > >> > > > > > >> > >> > > > > > > JIRA
>>>> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
>>>> > >> > > > > > >> > >> > > > > > >
>>>> > >> > > > > > >> > >> > > > > > >
>>>> /*******************************************
>>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
>>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
>>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
>>>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
>>>> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
>>>> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
>>>> > >> > > > > > >> > >> > > >
>>>> > >> > > > > > >> > >> > > > > > >
>>>> ********************************************/
>>>> > >> > > > > > >> > >> > > > > > >
>>>> > >> > > > > > >> > >> > > > > >
>>>> > >> > > > > > >> > >> > > > >
>>>> > >> > > > > > >> > >> > > >
>>>> > >> > > > > > >> > >> > >
>>>> > >> > > > > > >> > >> >
>>>> > >> > > > > > >> > >>
>>>> > >> > > > > > >> > >>
>>>> > >> > > > > > >> > >>
>>>> > >> > > > > > >> > >> --
>>>> > >> > > > > > >> > >> -- Guozhang
>>>> > >> > > > > > >> > >>
>>>> > >> > > > > > >> > >
>>>> > >> > > > > > >> > >
>>>> > >> > > > > > >> >
>>>> > >> > > > > > >>
>>>> > >> > > > > > >
>>>> > >> > > > > > >
>>>> > >> > > > > >
>>>> > >> > > > >
>>>> > >> > > >
>>>> > >> > >
>>>> > >> >
>>>> > >> >
>>>> > >> >
>>>> > >> > --
>>>> > >> > Jeff Holoman
>>>> > >> > Systems Engineer
>>>> > >> >
>>>> > >>
>>>>
>>>>
>>>
>>>
>>> --
>>> -- Guozhang
>>>
>>
>>
>>
>> --
>> -- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Gwen Shapira <gs...@cloudera.com>.
Specifically for ownership, I think the plan is to add ACL (it sounds
like you are describing ACL) via an external system (Argus, Sentry).
I remember KIP-11 described this, but I can't find the KIP any longer.

Regardless, I think KIP-4 focuses on getting information that already
exists from Kafka brokers, not on adding information that perhaps
should exist but doesn't yet?

Gwen





On Thu, Mar 12, 2015 at 6:37 AM, Guozhang Wang <wa...@gmail.com> wrote:
> Folks,
>
> Just want to elaborate a bit more on the create-topic metadata and batching
> describe-topic based on config / metadata in my previous email as we work
> on KAFKA-1694. The main motivation is to have some sort of topic management
> mechanisms, which I think is quite important in a multi-tenant / cloud
> architecture: today anyone can create topics in a shared Kafka cluster, but
> there is no concept or "ownership" of topics that are created by different
> users. For example, at LinkedIn we basically distinguish topic owners via
> some casual topic name prefix, which is a bit awkward and does not fly as
> we scale our customers. It would be great to use describe-topics such as:
>
> Describe all topics that is created by me.
>
> Describe all topics whose retention time is overriden to X.
>
> Describe all topics whose writable group include user Y (this is related to
> authorization), etc..
>
> One possible way to achieve this is to add a metadata file in the
> create-topic request, whose value will also be written ZK as we create the
> topic; then describe-topics can choose to batch topics based on 1) name
> regex, 2) config K-V matching, 3) metadata regex, etc.
>
> Thoughts?
>
> Guozhang
>
> On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com> wrote:
>
>> Thanks for the updated wiki. A few comments below:
>>
>> 1. Error description in response: I think if some errorCode could indicate
>> several different error cases then we should really change it to multiple
>> codes. In general the errorCode itself would be precise and sufficient for
>> describing the server side errors.
>>
>> 2. Describe topic request: it would be great to go beyond just batching on
>> topic name regex for this request. For example, a very common use case of
>> the topic command is to list all topics whose config A's value is B. With
>> topic name regex then we have to first retrieve __all__ topics's
>> description info and then filter at the client end, which will be a huge
>> burden on ZK.
>>
>> 3. Config K-Vs in create topic: this is related to the previous point;
>> maybe we can add another metadata K-V or just a metadata string along side
>> with config K-V in create topic like we did for offset commit request. This
>> field can be quite useful in storing information like "owner" of the topic
>> who issue the create command, etc, which is quite important for a
>> multi-tenant setting. Then in the describe topic request we can also batch
>> on regex of the metadata field.
>>
>> 4. Today all the admin operations are async in the sense that command will
>> return once it is written in ZK, and that is why we need extra verification
>> like testUtil.waitForTopicCreated() / verify partition reassignment
>> request, etc. With admin requests we could add a flag to enable / disable
>> synchronous requests; when it is turned on, the response will not return
>> until the request has been completed. And for async requests we can add a
>> "token" field in the response, and then only need a general "admin
>> verification request" with the given token to check if the async request
>> has been completed.
>>
>> 5. +1 for extending Metadata request to include controller / coordinator
>> information, and then we can remove the ConsumerMetadata / ClusterMetadata
>> requests.
>>
>> Guozhang
>>
>> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com> wrote:
>>
>>> Thanks for sending that out Joe - I don't think I will be able to make
>>> it today, so if notes can be sent out afterward that would be great.
>>>
>>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
>>> > Thanks for sending this out Joe. Looking forward to chatting with
>>> everyone :)
>>> >
>>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly> wrote:
>>> > > Hey, I just sent out a google hangout invite to all pmc, committers
>>> and
>>> > > everyone I found working on a KIP. If I missed anyone in the invite
>>> please
>>> > > let me know and can update it, np.
>>> > >
>>> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can get
>>> INFRA
>>> > > help to make a google account so we can manage better?
>>> > >
>>> > > To discuss
>>> > >
>>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>>> > > in progress and related JIRA that are interdependent and common work.
>>> > >
>>> > > ~ Joe Stein
>>> > >
>>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com>
>>> wrote:
>>> > >
>>> > >> Let's stay on Google hangouts that will also record and make the
>>> sessions
>>> > >> available on youtube.
>>> > >>
>>> > >> -Jay
>>> > >>
>>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
>>> jholoman@cloudera.com>
>>> > >> wrote:
>>> > >>
>>> > >> > Jay / Joe
>>> > >> >
>>> > >> > We're happy to send out a Webex for this purpose. We could record
>>> the
>>> > >> > sessions if there is interest and publish them out.
>>> > >> >
>>> > >> > Thanks
>>> > >> >
>>> > >> > Jeff
>>> > >> >
>>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <ja...@gmail.com>
>>> wrote:
>>> > >> >
>>> > >> > > Let's try to get the technical hang-ups sorted out, though. I
>>> really
>>> > >> > think
>>> > >> > > there is some benefit to live discussion vs writing. I am
>>> hopeful that
>>> > >> if
>>> > >> > > we post instructions and give ourselves a few attempts we can
>>> get it
>>> > >> > > working.
>>> > >> > >
>>> > >> > > Tuesday at that time would work for me...any objections?
>>> > >> > >
>>> > >> > > -Jay
>>> > >> > >
>>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <joe.stein@stealth.ly
>>> >
>>> > >> wrote:
>>> > >> > >
>>> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am
>>> PT
>>> > >> ????
>>> > >> > > >
>>> > >> > > > I don't mind google hangout but there is always some issue or
>>> > >> whatever
>>> > >> > so
>>> > >> > > > we know the apache irc channel works. We can start there and
>>> see how
>>> > >> it
>>> > >> > > > goes? We can pull transcripts too and associate to tickets if
>>> need be
>>> > >> > > makes
>>> > >> > > > it helpful for things.
>>> > >> > > >
>>> > >> > > > ~ Joestein
>>> > >> > > >
>>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
>>> jay.kreps@gmail.com>
>>> > >> > wrote:
>>> > >> > > >
>>> > >> > > > > We'd talked about doing a Google Hangout to chat about this.
>>> What
>>> > >> > about
>>> > >> > > > > generalizing that a little further...I actually think it
>>> would be
>>> > >> > good
>>> > >> > > > for
>>> > >> > > > > everyone spending a reasonable chunk of their week on Kafka
>>> stuff
>>> > >> to
>>> > >> > > > maybe
>>> > >> > > > > sync up once a week. I think we could use time to talk
>>> through
>>> > >> design
>>> > >> > > > > stuff, make sure we are on top of code reviews, talk through
>>> any
>>> > >> > tricky
>>> > >> > > > > issues, etc.
>>> > >> > > > >
>>> > >> > > > > We can make it publicly available so that any one can follow
>>> along
>>> > >> > who
>>> > >> > > > > likes.
>>> > >> > > > >
>>> > >> > > > > Any interest in doing this? If so I'll try to set it up
>>> starting
>>> > >> next
>>> > >> > > > week.
>>> > >> > > > >
>>> > >> > > > > -Jay
>>> > >> > > > >
>>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
>>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
>>> > >> > > > >
>>> > >> > > > > > Hi all,
>>> > >> > > > > >
>>> > >> > > > > > I've updated KIP page, fixed / aligned document structure.
>>> Also I
>>> > >> > > added
>>> > >> > > > > > some
>>> > >> > > > > > very initial proposal for AdminClient so we have something
>>> to
>>> > >> start
>>> > >> > > > from
>>> > >> > > > > > while
>>> > >> > > > > > discussing the KIP.
>>> > >> > > > > >
>>> > >> > > > > >
>>> > >> > > > > >
>>> > >> > > > >
>>> > >> > > >
>>> > >> > >
>>> > >> >
>>> > >>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>>> > >> > > > > >
>>> > >> > > > > > Thanks,
>>> > >> > > > > > Andrii Biletskyi
>>> > >> > > > > >
>>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
>>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
>>> > >> > > > > >
>>> > >> > > > > > > Jay,
>>> > >> > > > > > >
>>> > >> > > > > > > Re error messages: you are right, in most cases client
>>> will
>>> > >> have
>>> > >> > > > enough
>>> > >> > > > > > > context to show descriptive error message. My concern is
>>> that
>>> > >> we
>>> > >> > > will
>>> > >> > > > > > have
>>> > >> > > > > > > to
>>> > >> > > > > > > add lots of new error codes for each possible error. Of
>>> course,
>>> > >> > we
>>> > >> > > > > could
>>> > >> > > > > > > reuse
>>> > >> > > > > > > some of existing like UknownTopicOrPartitionCode, but we
>>> will
>>> > >> > also
>>> > >> > > > need
>>> > >> > > > > > to
>>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
>>> TopicConfigInvalid (both
>>> > >> > for
>>> > >> > > > > topic
>>> > >> > > > > > > name and config, and probably user would like to know
>>> what
>>> > >> > exactly
>>> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
>>> > >> InternalError
>>> > >> > > > (e.g.
>>> > >> > > > > > > zookeeper failure) etc.
>>> > >> > > > > > > And this is only for TopicCommand, we will also need to
>>> add
>>> > >> > similar
>>> > >> > > > > stuff
>>> > >> > > > > > > for
>>> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up
>>> with a
>>> > >> > large
>>> > >> > > > list
>>> > >> > > > > > of
>>> > >> > > > > > > error codes, used only in Admin protocol.
>>> > >> > > > > > > Having said that, I agree my proposal is not consistent
>>> with
>>> > >> > other
>>> > >> > > > > cases.
>>> > >> > > > > > > Maybe we can find better solution or something
>>> in-between.
>>> > >> > > > > > >
>>> > >> > > > > > > Re Hangout chat: I think it is a great idea. This way we
>>> can
>>> > >> move
>>> > >> > > on
>>> > >> > > > > > > faster.
>>> > >> > > > > > > Let's agree somehow on date/time so people can join.
>>> Will work
>>> > >> > for
>>> > >> > > me
>>> > >> > > > > > this
>>> > >> > > > > > > and
>>> > >> > > > > > > next week almost anytime if agreed in advance.
>>> > >> > > > > > >
>>> > >> > > > > > > Thanks,
>>> > >> > > > > > > Andrii
>>> > >> > > > > > >
>>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
>>> > >> jay.kreps@gmail.com>
>>> > >> > > > > wrote:
>>> > >> > > > > > >
>>> > >> > > > > > >> Hey Andrii,
>>> > >> > > > > > >>
>>> > >> > > > > > >> Generally we can do good error handling without needing
>>> custom
>>> > >> > > > > > server-side
>>> > >> > > > > > >> messages. I.e. generally the client has the context to
>>> know
>>> > >> that
>>> > >> > > if
>>> > >> > > > it
>>> > >> > > > > > got
>>> > >> > > > > > >> an error that the topic doesn't exist to say "Topic X
>>> doesn't
>>> > >> > > exist"
>>> > >> > > > > > >> rather
>>> > >> > > > > > >> than "error code 14" (or whatever). Maybe there are
>>> specific
>>> > >> > cases
>>> > >> > > > > where
>>> > >> > > > > > >> this is hard? If we want to add server-side error
>>> messages we
>>> > >> > > really
>>> > >> > > > > do
>>> > >> > > > > > >> need to do this in a consistent way across the protocol.
>>> > >> > > > > > >>
>>> > >> > > > > > >> I still have a bunch of open questions here from my
>>> previous
>>> > >> > > list. I
>>> > >> > > > > > will
>>> > >> > > > > > >> be out for the next few days for Strata though. Maybe
>>> we could
>>> > >> > do
>>> > >> > > a
>>> > >> > > > > > Google
>>> > >> > > > > > >> Hangout chat on any open issues some time towards the
>>> end of
>>> > >> > next
>>> > >> > > > week
>>> > >> > > > > > for
>>> > >> > > > > > >> anyone interested in this ticket? I have a feeling that
>>> might
>>> > >> > > > progress
>>> > >> > > > > > >> things a little faster than email--I think we could talk
>>> > >> through
>>> > >> > > > those
>>> > >> > > > > > >> issues I brought up fairly quickly...
>>> > >> > > > > > >>
>>> > >> > > > > > >> -Jay
>>> > >> > > > > > >>
>>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
>>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
>>> > >> > > > > > >>
>>> > >> > > > > > >> > Hi all,
>>> > >> > > > > > >> >
>>> > >> > > > > > >> > I'm trying to address some of the issues which were
>>> > >> mentioned
>>> > >> > > > > earlier
>>> > >> > > > > > >> about
>>> > >> > > > > > >> > Admin RQ/RP format. One of those was about batching
>>> > >> > operations.
>>> > >> > > > What
>>> > >> > > > > > if
>>> > >> > > > > > >> we
>>> > >> > > > > > >> > follow TopicCommand approach and let people specify
>>> > >> topic-name
>>> > >> > > by
>>> > >> > > > > > >> regexp -
>>> > >> > > > > > >> > would that cover most of the use cases?
>>> > >> > > > > > >> >
>>> > >> > > > > > >> > Secondly, is what information should we generally
>>> provide in
>>> > >> > > Admin
>>> > >> > > > > > >> > responses.
>>> > >> > > > > > >> > I realize that Admin commands don't imply they will
>>> be used
>>> > >> > only
>>> > >> > > > in
>>> > >> > > > > > CLI
>>> > >> > > > > > >> > but,
>>> > >> > > > > > >> > it seems to me, CLI is a very important client of this
>>> > >> > feature.
>>> > >> > > In
>>> > >> > > > > > this
>>> > >> > > > > > >> > case,
>>> > >> > > > > > >> > seems logical, we would like to provide users with
>>> rich
>>> > >> > > experience
>>> > >> > > > > in
>>> > >> > > > > > >> terms
>>> > >> > > > > > >> > of
>>> > >> > > > > > >> > getting results / errors of the executed commands.
>>> Usually
>>> > >> we
>>> > >> > > > supply
>>> > >> > > > > > >> with
>>> > >> > > > > > >> > responses only errorCode, which looks very limiting,
>>> in case
>>> > >> > of
>>> > >> > > > CLI
>>> > >> > > > > we
>>> > >> > > > > > >> may
>>> > >> > > > > > >> > want to print human readable error description.
>>> > >> > > > > > >> >
>>> > >> > > > > > >> > So, taking into account previous item about batching,
>>> what
>>> > >> do
>>> > >> > > you
>>> > >> > > > > > think
>>> > >> > > > > > >> > about
>>> > >> > > > > > >> > having smth like:
>>> > >> > > > > > >> >
>>> > >> > > > > > >> > ('create' doesn't support regexp)
>>> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions Replicas
>>> > >> > > > > ReplicaAssignment
>>> > >> > > > > > >> > [Config]
>>> > >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
>>> > >> > > > > > >> >   ErrorCode => int16
>>> > >> > > > > > >> >   ErrorDescription => string (empty if successful)
>>> > >> > > > > > >> >
>>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
>>> > >> > > ReplicaAssignment
>>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
>>> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
>>> ErrorDescription]
>>> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
>>> > >> > > > > > >> >   CommandErrorCode => int16
>>> > >> > > > > > >> >   CommandErrorDescription => string (nonempty in case
>>> of
>>> > >> fatal
>>> > >> > > > > error,
>>> > >> > > > > > >> e.g.
>>> > >> > > > > > >> > we couldn't get topics by regexp)
>>> > >> > > > > > >> >
>>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
>>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName TopicDescription
>>> > >> ErrorCode
>>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
>>> CommandErrorDescription
>>> > >> > > > > > >> >
>>> > >> > > > > > >> > Also, any thoughts about our discussion regarding
>>> re-routing
>>> > >> > > > > facility?
>>> > >> > > > > > >> In
>>> > >> > > > > > >> > my
>>> > >> > > > > > >> > understanding, it is like between augmenting
>>> > >> > > TopicMetadataRequest
>>> > >> > > > > > >> > (to include at least controllerId) and implementing
>>> new
>>> > >> > generic
>>> > >> > > > > > >> re-routing
>>> > >> > > > > > >> > facility so sending messages to controller will be
>>> handled
>>> > >> by
>>> > >> > > it.
>>> > >> > > > > > >> >
>>> > >> > > > > > >> > Thanks,
>>> > >> > > > > > >> > Andrii Biletskyi
>>> > >> > > > > > >> >
>>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
>>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
>>> > >> > > > > > >> >
>>> > >> > > > > > >> > > @Guozhang:
>>> > >> > > > > > >> > > Thanks for your comments, I've answered some of
>>> those. The
>>> > >> > > main
>>> > >> > > > > > thing
>>> > >> > > > > > >> is
>>> > >> > > > > > >> > > having merged request for
>>> create-alter-delete-describe - I
>>> > >> > > have
>>> > >> > > > > some
>>> > >> > > > > > >> > > concerns about this approach.
>>> > >> > > > > > >> > >
>>> > >> > > > > > >> > > @*Jay*:
>>> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is also
>>> one of
>>> > >> > the
>>> > >> > > > > > >> concerns.
>>> > >> > > > > > >> > We
>>> > >> > > > > > >> > > can solve it if we implement re-routing facility.
>>> But I
>>> > >> > agree
>>> > >> > > > with
>>> > >> > > > > > >> > > Guozhang - it will make clients' internals a little
>>> bit
>>> > >> > easier
>>> > >> > > > but
>>> > >> > > > > > >> this
>>> > >> > > > > > >> > > seems to be a complex logic to implement and
>>> support then.
>>> > >> > > > > > Especially
>>> > >> > > > > > >> for
>>> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing later
>>> for
>>> > >> these
>>> > >> > > > > > >> requests).
>>> > >> > > > > > >> > > Also people will tend to avoid this re-routing
>>> facility
>>> > >> and
>>> > >> > > hold
>>> > >> > > > > > local
>>> > >> > > > > > >> > > cluster cache to ensure their high-priority requests
>>> > >> (which
>>> > >> > > some
>>> > >> > > > > of
>>> > >> > > > > > >> the
>>> > >> > > > > > >> > > admin requests are) not sent to some busy broker
>>> where
>>> > >> they
>>> > >> > > wait
>>> > >> > > > > to
>>> > >> > > > > > be
>>> > >> > > > > > >> > > routed to the correct one.
>>> > >> > > > > > >> > > As pointed out by Jun here (
>>> > >> > > > > > >> > >
>>> > >> > > > > > >> >
>>> > >> > > > > > >>
>>> > >> > > > > >
>>> > >> > > > >
>>> > >> > > >
>>> > >> > >
>>> > >> >
>>> > >>
>>> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
>>> > >> > > > > > >> > )
>>> > >> > > > > > >> > > to solve the issue we might introduce a message
>>> type to
>>> > >> get
>>> > >> > > > > cluster
>>> > >> > > > > > >> > state.
>>> > >> > > > > > >> > > But I agree we can just update
>>> TopicMetadataResponse to
>>> > >> > > include
>>> > >> > > > > > >> > > controllerId (and probably smth else).
>>> > >> > > > > > >> > > What are you thougths?
>>> > >> > > > > > >> > >
>>> > >> > > > > > >> > > Thanks,
>>> > >> > > > > > >> > > Andrii
>>> > >> > > > > > >> > >
>>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
>>> > >> > > > > wangguoz@gmail.com>
>>> > >> > > > > > >> > wrote:
>>> > >> > > > > > >> > >
>>> > >> > > > > > >> > >> I think for the topics commands we can actually
>>> merge
>>> > >> > > > > > >> > >> create/alter/delete/describe as one request type
>>> since
>>> > >> > their
>>> > >> > > > > > formats
>>> > >> > > > > > >> are
>>> > >> > > > > > >> > >> very much similar, and keep list-topics and others
>>> like
>>> > >> > > > > > >> > >> partition-reassignment / preferred-leader-election
>>> as
>>> > >> > > separate
>>> > >> > > > > > >> request
>>> > >> > > > > > >> > >> types, I also left some other comments on the RB (
>>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
>>> > >> > > > > > >> > >>
>>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
>>> > >> > > > jay.kreps@gmail.com>
>>> > >> > > > > > >> wrote:
>>> > >> > > > > > >> > >>
>>> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to just
>>> have
>>> > >> one
>>> > >> > > "do
>>> > >> > > > > > admin
>>> > >> > > > > > >> > >> stuff"
>>> > >> > > > > > >> > >> > command that has the union of all parameters.
>>> > >> > > > > > >> > >> >
>>> > >> > > > > > >> > >> > What I am saying is that command line tools are
>>> one
>>> > >> > client
>>> > >> > > of
>>> > >> > > > > the
>>> > >> > > > > > >> > >> > administrative apis, but these will be used in a
>>> number
>>> > >> > of
>>> > >> > > > > > >> scenarios
>>> > >> > > > > > >> > so
>>> > >> > > > > > >> > >> > they should make logical sense even in the
>>> absence of
>>> > >> the
>>> > >> > > > > command
>>> > >> > > > > > >> line
>>> > >> > > > > > >> > >> > tool. Hence comments like trying to clarify the
>>> > >> > > relationship
>>> > >> > > > > > >> between
>>> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds
>>> of
>>> > >> things
>>> > >> > > > > really
>>> > >> > > > > > >> need
>>> > >> > > > > > >> > >> to be
>>> > >> > > > > > >> > >> > thought through.
>>> > >> > > > > > >> > >> >
>>> > >> > > > > > >> > >> > Hope that makes sense.
>>> > >> > > > > > >> > >> >
>>> > >> > > > > > >> > >> > -Jay
>>> > >> > > > > > >> > >> >
>>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
>>> Biletskyi <
>>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
>>> > >> > > > > > >> > >> >
>>> > >> > > > > > >> > >> > > Jay,
>>> > >> > > > > > >> > >> > >
>>> > >> > > > > > >> > >> > > Thanks for answering. You understood
>>> correctly, most
>>> > >> of
>>> > >> > > my
>>> > >> > > > > > >> comments
>>> > >> > > > > > >> > >> were
>>> > >> > > > > > >> > >> > > related to your point 1) - about "well
>>> thought-out"
>>> > >> > apis.
>>> > >> > > > > Also,
>>> > >> > > > > > >> yes,
>>> > >> > > > > > >> > >> as I
>>> > >> > > > > > >> > >> > > understood we would like to introduce a single
>>> > >> unified
>>> > >> > > CLI
>>> > >> > > > > tool
>>> > >> > > > > > >> with
>>> > >> > > > > > >> > >> > > centralized server-side request handling for
>>> lots of
>>> > >> > > > existing
>>> > >> > > > > > >> ones
>>> > >> > > > > > >> > >> (incl.
>>> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
>>> > >> ReassignPartitions,
>>> > >> > > smth
>>> > >> > > > > > else
>>> > >> > > > > > >> if
>>> > >> > > > > > >> > >> added
>>> > >> > > > > > >> > >> > > in future). In our previous discussion (
>>> > >> > > > > > >> > >> > >
>>> https://issues.apache.org/jira/browse/KAFKA-1694)
>>> > >> > people
>>> > >> > > > > said
>>> > >> > > > > > >> > they'd
>>> > >> > > > > > >> > >> > > rather
>>> > >> > > > > > >> > >> > > have a separate message for each command, so,
>>> yes,
>>> > >> this
>>> > >> > > > way I
>>> > >> > > > > > >> came
>>> > >> > > > > > >> > to
>>> > >> > > > > > >> > >> 1-1
>>> > >> > > > > > >> > >> > > mapping between commands in the tool and
>>> protocol
>>> > >> > > > additions.
>>> > >> > > > > > But
>>> > >> > > > > > >> I
>>> > >> > > > > > >> > >> might
>>> > >> > > > > > >> > >> > be
>>> > >> > > > > > >> > >> > > wrong.
>>> > >> > > > > > >> > >> > > At the end I just try to start discussion how
>>> at
>>> > >> least
>>> > >> > > > > > generally
>>> > >> > > > > > >> > this
>>> > >> > > > > > >> > >> > > protocol should look like.
>>> > >> > > > > > >> > >> > >
>>> > >> > > > > > >> > >> > > Thanks,
>>> > >> > > > > > >> > >> > > Andrii
>>> > >> > > > > > >> > >> > >
>>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
>>> > >> > > > > > jay.kreps@gmail.com
>>> > >> > > > > > >> >
>>> > >> > > > > > >> > >> wrote:
>>> > >> > > > > > >> > >> > >
>>> > >> > > > > > >> > >> > > > Hey Andrii,
>>> > >> > > > > > >> > >> > > >
>>> > >> > > > > > >> > >> > > > To answer your earlier question we just
>>> really
>>> > >> can't
>>> > >> > be
>>> > >> > > > > > adding
>>> > >> > > > > > >> any
>>> > >> > > > > > >> > >> more
>>> > >> > > > > > >> > >> > > > scala protocol objects. These things are
>>> super hard
>>> > >> > to
>>> > >> > > > > > maintain
>>> > >> > > > > > >> > >> because
>>> > >> > > > > > >> > >> > > > they hand code the byte parsing and don't
>>> have good
>>> > >> > > > > > versioning
>>> > >> > > > > > >> > >> support.
>>> > >> > > > > > >> > >> > > > Since we are already planning on converting
>>> we
>>> > >> > > definitely
>>> > >> > > > > > don't
>>> > >> > > > > > >> > >> want to
>>> > >> > > > > > >> > >> > > add
>>> > >> > > > > > >> > >> > > > a ton more of these--they are total tech
>>> debt.
>>> > >> > > > > > >> > >> > > >
>>> > >> > > > > > >> > >> > > > What does it mean that the changes are
>>> isolated
>>> > >> from
>>> > >> > > the
>>> > >> > > > > > >> current
>>> > >> > > > > > >> > >> code
>>> > >> > > > > > >> > >> > > base?
>>> > >> > > > > > >> > >> > > >
>>> > >> > > > > > >> > >> > > > I actually didn't understand the remaining
>>> > >> comments,
>>> > >> > > > which
>>> > >> > > > > of
>>> > >> > > > > > >> the
>>> > >> > > > > > >> > >> > points
>>> > >> > > > > > >> > >> > > > are you responding to?
>>> > >> > > > > > >> > >> > > >
>>> > >> > > > > > >> > >> > > > Maybe one sticking point here is that it
>>> seems like
>>> > >> > you
>>> > >> > > > > want
>>> > >> > > > > > to
>>> > >> > > > > > >> > make
>>> > >> > > > > > >> > >> > some
>>> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1 mapping
>>> > >> between
>>> > >> > > > > > commands
>>> > >> > > > > > >> you
>>> > >> > > > > > >> > >> > > imagine
>>> > >> > > > > > >> > >> > > > in the tool and protocol additions. I want
>>> to make
>>> > >> > sure
>>> > >> > > > we
>>> > >> > > > > > >> don't
>>> > >> > > > > > >> > do
>>> > >> > > > > > >> > >> > that.
>>> > >> > > > > > >> > >> > > > The protocol needs to be really really well
>>> thought
>>> > >> > out
>>> > >> > > > > > against
>>> > >> > > > > > >> > many
>>> > >> > > > > > >> > >> > use
>>> > >> > > > > > >> > >> > > > cases so it should make perfect logical
>>> sense in
>>> > >> the
>>> > >> > > > > absence
>>> > >> > > > > > of
>>> > >> > > > > > >> > >> knowing
>>> > >> > > > > > >> > >> > > the
>>> > >> > > > > > >> > >> > > > command line tool, right?
>>> > >> > > > > > >> > >> > > >
>>> > >> > > > > > >> > >> > > > -Jay
>>> > >> > > > > > >> > >> > > >
>>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii
>>> Biletskyi
>>> > >> <
>>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
>>> > >> > > > > > >> > >> > > >
>>> > >> > > > > > >> > >> > > > > Hey Jay,
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > > > I would like to continue this discussion
>>> as it
>>> > >> seem
>>> > >> > > > there
>>> > >> > > > > > is
>>> > >> > > > > > >> no
>>> > >> > > > > > >> > >> > > progress
>>> > >> > > > > > >> > >> > > > > here.
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > > > First of all, could you please explain
>>> what did
>>> > >> you
>>> > >> > > > mean
>>> > >> > > > > in
>>> > >> > > > > > >> 2?
>>> > >> > > > > > >> > How
>>> > >> > > > > > >> > >> > > > exactly
>>> > >> > > > > > >> > >> > > > > are we going to migrate to the new java
>>> protocol
>>> > >> > > > > > definitions.
>>> > >> > > > > > >> > And
>>> > >> > > > > > >> > >> why
>>> > >> > > > > > >> > >> > > > it's
>>> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > > > I agree with you, this feature includes
>>> lots of
>>> > >> > > stuff,
>>> > >> > > > > but
>>> > >> > > > > > >> > >> thankfully
>>> > >> > > > > > >> > >> > > > > almost all changes are isolated from the
>>> current
>>> > >> > code
>>> > >> > > > > base,
>>> > >> > > > > > >> > >> > > > > so the main thing, I think, we need to
>>> agree is
>>> > >> > RQ/RP
>>> > >> > > > > > format.
>>> > >> > > > > > >> > >> > > > > So how can we start discussion about the
>>> concrete
>>> > >> > > > > messages
>>> > >> > > > > > >> > format?
>>> > >> > > > > > >> > >> > > > > Can we take (
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > >
>>> > >> > > > > > >> > >> > >
>>> > >> > > > > > >> > >> >
>>> > >> > > > > > >> > >>
>>> > >> > > > > > >> >
>>> > >> > > > > > >>
>>> > >> > > > > >
>>> > >> > > > >
>>> > >> > > >
>>> > >> > >
>>> > >> >
>>> > >>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
>>> > >> > > > > > >> > >> > > > > )
>>> > >> > > > > > >> > >> > > > > as starting point?
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > > > We had some doubts earlier whether it worth
>>> > >> > > introducing
>>> > >> > > > > one
>>> > >> > > > > > >> > >> generic
>>> > >> > > > > > >> > >> > > Admin
>>> > >> > > > > > >> > >> > > > > Request for all commands (
>>> > >> > > > > > >> > >> > > >
>>> https://issues.apache.org/jira/browse/KAFKA-1694
>>> > >> > > > > > >> > >> > > > > )
>>> > >> > > > > > >> > >> > > > > but then everybody agreed it would be
>>> better to
>>> > >> > have
>>> > >> > > > > > separate
>>> > >> > > > > > >> > >> message
>>> > >> > > > > > >> > >> > > for
>>> > >> > > > > > >> > >> > > > > each admin command. The Request part is
>>> really
>>> > >> > > dictated
>>> > >> > > > > > from
>>> > >> > > > > > >> the
>>> > >> > > > > > >> > >> > > command
>>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so
>>> the
>>> > >> > proposed
>>> > >> > > > > > version
>>> > >> > > > > > >> > >> should
>>> > >> > > > > > >> > >> > be
>>> > >> > > > > > >> > >> > > > > fine (let's put aside for now remarks about
>>> > >> > Optional
>>> > >> > > > > type,
>>> > >> > > > > > >> > >> batching,
>>> > >> > > > > > >> > >> > > > > configs normalization - I agree with all of
>>> > >> them).
>>> > >> > > > > > >> > >> > > > > So the second part is Response. I see
>>> there are
>>> > >> two
>>> > >> > > > cases
>>> > >> > > > > > >> here.
>>> > >> > > > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ;
>>> b)
>>> > >> "Get"
>>> > >> > > > > > requests -
>>> > >> > > > > > >> > >> > > > > List/Describe...
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > > > a) should only hold request result
>>> (regardless
>>> > >> what
>>> > >> > > we
>>> > >> > > > > > decide
>>> > >> > > > > > >> > >> about
>>> > >> > > > > > >> > >> > > > > blocking/non-blocking commands execution).
>>> > >> > > > > > >> > >> > > > > Usually we provide error code in response
>>> but
>>> > >> since
>>> > >> > > we
>>> > >> > > > > will
>>> > >> > > > > > >> use
>>> > >> > > > > > >> > >> this
>>> > >> > > > > > >> > >> > in
>>> > >> > > > > > >> > >> > > > > interactive shell we need some human
>>> readable
>>> > >> error
>>> > >> > > > > > >> description
>>> > >> > > > > > >> > -
>>> > >> > > > > > >> > >> so
>>> > >> > > > > > >> > >> > I
>>> > >> > > > > > >> > >> > > > > added errorDesription field where you can
>>> at
>>> > >> least
>>> > >> > > > leave
>>> > >> > > > > > >> > >> > > > > exception.getMessage.
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > > > b) in addition to previous item message
>>> should
>>> > >> hold
>>> > >> > > > > command
>>> > >> > > > > > >> > >> specific
>>> > >> > > > > > >> > >> > > > > response data. We can discuss in detail
>>> each of
>>> > >> > them
>>> > >> > > > but
>>> > >> > > > > > >> let's
>>> > >> > > > > > >> > for
>>> > >> > > > > > >> > >> > now
>>> > >> > > > > > >> > >> > > > > agree about the overall pattern.
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > > > Thanks,
>>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps
>>> <
>>> > >> > > > > > >> jay.kreps@gmail.com
>>> > >> > > > > > >> > >
>>> > >> > > > > > >> > >> > > wrote:
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > > > > Hey Joe,
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > This is great. A few comments on KIP-4
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > 1. This is much needed functionality,
>>> but there
>>> > >> > > are a
>>> > >> > > > > lot
>>> > >> > > > > > >> of
>>> > >> > > > > > >> > >> the so
>>> > >> > > > > > >> > >> > > > let's
>>> > >> > > > > > >> > >> > > > > > really think these protocols through. We
>>> really
>>> > >> > > want
>>> > >> > > > to
>>> > >> > > > > > >> end up
>>> > >> > > > > > >> > >> > with a
>>> > >> > > > > > >> > >> > > > set
>>> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis.
>>> For this
>>> > >> > > > reason I
>>> > >> > > > > > >> think
>>> > >> > > > > > >> > >> it is
>>> > >> > > > > > >> > >> > > > > really
>>> > >> > > > > > >> > >> > > > > > important to think through the end state
>>> even
>>> > >> if
>>> > >> > > that
>>> > >> > > > > > >> includes
>>> > >> > > > > > >> > >> APIs
>>> > >> > > > > > >> > >> > > we
>>> > >> > > > > > >> > >> > > > > > won't implement in the first phase.
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait until
>>> we
>>> > >> have
>>> > >> > > > > switched
>>> > >> > > > > > >> the
>>> > >> > > > > > >> > >> > server
>>> > >> > > > > > >> > >> > > > over
>>> > >> > > > > > >> > >> > > > > > to the new java protocol definitions. If
>>> we add
>>> > >> > > > upteen
>>> > >> > > > > > >> more ad
>>> > >> > > > > > >> > >> hoc
>>> > >> > > > > > >> > >> > > > scala
>>> > >> > > > > > >> > >> > > > > > objects that is just generating more
>>> work for
>>> > >> the
>>> > >> > > > > > >> conversion
>>> > >> > > > > > >> > we
>>> > >> > > > > > >> > >> > know
>>> > >> > > > > > >> > >> > > we
>>> > >> > > > > > >> > >> > > > > > have to do.
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new type of
>>> > >> > optional
>>> > >> > > > > > >> parameter.
>>> > >> > > > > > >> > >> This
>>> > >> > > > > > >> > >> > is
>>> > >> > > > > > >> > >> > > > > > inconsistent with everything else in the
>>> > >> protocol
>>> > >> > > > where
>>> > >> > > > > > we
>>> > >> > > > > > >> use
>>> > >> > > > > > >> > >> -1
>>> > >> > > > > > >> > >> > or
>>> > >> > > > > > >> > >> > > > some
>>> > >> > > > > > >> > >> > > > > > other marker value. You could argue
>>> either way
>>> > >> > but
>>> > >> > > > > let's
>>> > >> > > > > > >> stick
>>> > >> > > > > > >> > >> with
>>> > >> > > > > > >> > >> > > > that
>>> > >> > > > > > >> > >> > > > > > for consistency. For clients that
>>> implemented
>>> > >> the
>>> > >> > > > > > protocol
>>> > >> > > > > > >> in
>>> > >> > > > > > >> > a
>>> > >> > > > > > >> > >> > > better
>>> > >> > > > > > >> > >> > > > > way
>>> > >> > > > > > >> > >> > > > > > than our scala code these basic
>>> primitives are
>>> > >> > hard
>>> > >> > > > to
>>> > >> > > > > > >> change.
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
>>> duplicate
>>> > >> > > > > > >> > TopicMetadataRequest
>>> > >> > > > > > >> > >> > > which
>>> > >> > > > > > >> > >> > > > > has
>>> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I think
>>> we
>>> > >> > should
>>> > >> > > > > rename
>>> > >> > > > > > >> that
>>> > >> > > > > > >> > >> > > request
>>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
>>> > >> MetadataRequest)
>>> > >> > > and
>>> > >> > > > > > >> include
>>> > >> > > > > > >> > >> the id
>>> > >> > > > > > >> > >> > > of
>>> > >> > > > > > >> > >> > > > > the
>>> > >> > > > > > >> > >> > > > > > controller. Or are there other things we
>>> could
>>> > >> > add
>>> > >> > > > > here?
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to make a
>>> lot of
>>> > >> > > > requests
>>> > >> > > > > > that
>>> > >> > > > > > >> > can
>>> > >> > > > > > >> > >> > only
>>> > >> > > > > > >> > >> > > go
>>> > >> > > > > > >> > >> > > > > to
>>> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of
>>> burden for
>>> > >> > > > client
>>> > >> > > > > > >> > >> > > implementations
>>> > >> > > > > > >> > >> > > > > (it
>>> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can fail
>>> in many
>>> > >> > > parts
>>> > >> > > > > so
>>> > >> > > > > > it
>>> > >> > > > > > >> > >> ends up
>>> > >> > > > > > >> > >> > > > > being a
>>> > >> > > > > > >> > >> > > > > > full state machine to do right). I think
>>> we
>>> > >> > should
>>> > >> > > > > > consider
>>> > >> > > > > > >> > >> making
>>> > >> > > > > > >> > >> > > > admin
>>> > >> > > > > > >> > >> > > > > > commands and ideally as many of the
>>> other apis
>>> > >> as
>>> > >> > > > > > possible
>>> > >> > > > > > >> > >> > available
>>> > >> > > > > > >> > >> > > on
>>> > >> > > > > > >> > >> > > > > all
>>> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
>>> controller on
>>> > >> > the
>>> > >> > > > > broker
>>> > >> > > > > > >> > side.
>>> > >> > > > > > >> > >> > > Perhaps
>>> > >> > > > > > >> > >> > > > > > there would be a general way to
>>> encapsulate
>>> > >> this
>>> > >> > > > > > re-routing
>>> > >> > > > > > >> > >> > behavior.
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > 6. We should probably normalize the key
>>> value
>>> > >> > pairs
>>> > >> > > > > used
>>> > >> > > > > > >> for
>>> > >> > > > > > >> > >> > configs
>>> > >> > > > > > >> > >> > > > > rather
>>> > >> > > > > > >> > >> > > > > > than embedding a new formatting. So two
>>> strings
>>> > >> > > > rather
>>> > >> > > > > > than
>>> > >> > > > > > >> > one
>>> > >> > > > > > >> > >> > with
>>> > >> > > > > > >> > >> > > an
>>> > >> > > > > > >> > >> > > > > > internal equals sign.
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs
>>> that the
>>> > >> > > > command
>>> > >> > > > > > has
>>> > >> > > > > > >> > >> begun or
>>> > >> > > > > > >> > >> > > > that
>>> > >> > > > > > >> > >> > > > > > the command has been completed? It is a
>>> lot
>>> > >> more
>>> > >> > > > usable
>>> > >> > > > > > if
>>> > >> > > > > > >> the
>>> > >> > > > > > >> > >> > > command
>>> > >> > > > > > >> > >> > > > > has
>>> > >> > > > > > >> > >> > > > > > been completed so you know that if you
>>> create a
>>> > >> > > topic
>>> > >> > > > > and
>>> > >> > > > > > >> then
>>> > >> > > > > > >> > >> > > publish
>>> > >> > > > > > >> > >> > > > to
>>> > >> > > > > > >> > >> > > > > > it you won't get an exception about
>>> there being
>>> > >> > no
>>> > >> > > > such
>>> > >> > > > > > >> topic.
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
>>> duplicate a
>>> > >> lot
>>> > >> > > of
>>> > >> > > > > > stuff
>>> > >> > > > > > >> in
>>> > >> > > > > > >> > >> the
>>> > >> > > > > > >> > >> > > > > metadata
>>> > >> > > > > > >> > >> > > > > > request. Is there a reason to give back
>>> topics
>>> > >> > > marked
>>> > >> > > > > for
>>> > >> > > > > > >> > >> > deletion? I
>>> > >> > > > > > >> > >> > > > > feel
>>> > >> > > > > > >> > >> > > > > > like if we just make the post-condition
>>> of the
>>> > >> > > delete
>>> > >> > > > > > >> command
>>> > >> > > > > > >> > be
>>> > >> > > > > > >> > >> > that
>>> > >> > > > > > >> > >> > > > the
>>> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid of
>>> the need
>>> > >> > for
>>> > >> > > > this
>>> > >> > > > > > >> right?
>>> > >> > > > > > >> > >> And
>>> > >> > > > > > >> > >> > it
>>> > >> > > > > > >> > >> > > > > will
>>> > >> > > > > > >> > >> > > > > > be much more intuitive.
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > 9. Should we consider batching these
>>> requests?
>>> > >> We
>>> > >> > > > have
>>> > >> > > > > > >> > generally
>>> > >> > > > > > >> > >> > > tried
>>> > >> > > > > > >> > >> > > > to
>>> > >> > > > > > >> > >> > > > > > allow multiple operations to be batched.
>>> My
>>> > >> > > suspicion
>>> > >> > > > > is
>>> > >> > > > > > >> that
>>> > >> > > > > > >> > >> > without
>>> > >> > > > > > >> > >> > > > > this
>>> > >> > > > > > >> > >> > > > > > we will get a lot of code that does
>>> something
>>> > >> > like
>>> > >> > > > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
>>> > >> > > > > > >> > >> > > > > >       adminClient.describeTopic(topic)
>>> > >> > > > > > >> > >> > > > > > this code will work great when you test
>>> on 5
>>> > >> > topics
>>> > >> > > > but
>>> > >> > > > > > >> not do
>>> > >> > > > > > >> > >> as
>>> > >> > > > > > >> > >> > > well
>>> > >> > > > > > >> > >> > > > if
>>> > >> > > > > > >> > >> > > > > > you have 50k.
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss how
>>> we want
>>> > >> to
>>> > >> > > > > expose
>>> > >> > > > > > a
>>> > >> > > > > > >> > >> > > programmatic
>>> > >> > > > > > >> > >> > > > > JVM
>>> > >> > > > > > >> > >> > > > > > client api for these operations.
>>> Currently
>>> > >> people
>>> > >> > > > rely
>>> > >> > > > > on
>>> > >> > > > > > >> > >> > AdminUtils
>>> > >> > > > > > >> > >> > > > > which
>>> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we probably
>>> need
>>> > >> > > another
>>> > >> > > > > > client
>>> > >> > > > > > >> > >> under
>>> > >> > > > > > >> > >> > > > > clients/
>>> > >> > > > > > >> > >> > > > > > that exposes administrative
>>> functionality. We
>>> > >> > will
>>> > >> > > > need
>>> > >> > > > > > >> this
>>> > >> > > > > > >> > >> just
>>> > >> > > > > > >> > >> > to
>>> > >> > > > > > >> > >> > > > > > properly test the new apis, I suspect. We
>>> > >> should
>>> > >> > > > figure
>>> > >> > > > > > out
>>> > >> > > > > > >> > that
>>> > >> > > > > > >> > >> > API.
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > 11. The other information that would be
>>> really
>>> > >> > > useful
>>> > >> > > > > to
>>> > >> > > > > > >> get
>>> > >> > > > > > >> > >> would
>>> > >> > > > > > >> > >> > be
>>> > >> > > > > > >> > >> > > > > > information about partitions--how much
>>> data is
>>> > >> in
>>> > >> > > the
>>> > >> > > > > > >> > partition,
>>> > >> > > > > > >> > >> > what
>>> > >> > > > > > >> > >> > > > are
>>> > >> > > > > > >> > >> > > > > > the segment offsets, what is the log-end
>>> offset
>>> > >> > > (i.e.
>>> > >> > > > > > last
>>> > >> > > > > > >> > >> offset),
>>> > >> > > > > > >> > >> > > > what
>>> > >> > > > > > >> > >> > > > > is
>>> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think that
>>> done
>>> > >> > right
>>> > >> > > > this
>>> > >> > > > > > >> would
>>> > >> > > > > > >> > be
>>> > >> > > > > > >> > >> > the
>>> > >> > > > > > >> > >> > > > > > successor to the very awkward
>>> OffsetRequest we
>>> > >> > have
>>> > >> > > > > > today.
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > -Jay
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe
>>> Stein <
>>> > >> > > > > > >> > >> joe.stein@stealth.ly>
>>> > >> > > > > > >> > >> > > > > wrote:
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
>>> > >> > > > > > >> > >> > > > > > >
>>> > >> > > > > > >> > >> > > > > > >
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > >
>>> > >> > > > > > >> > >> > >
>>> > >> > > > > > >> > >> >
>>> > >> > > > > > >> > >>
>>> > >> > > > > > >> >
>>> > >> > > > > > >>
>>> > >> > > > > >
>>> > >> > > > >
>>> > >> > > >
>>> > >> > >
>>> > >> >
>>> > >>
>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>>> > >> > > > > > >> > >> > > > > > >
>>> > >> > > > > > >> > >> > > > > > > JIRA
>>> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
>>> > >> > > > > > >> > >> > > > > > >
>>> > >> > > > > > >> > >> > > > > > >
>>> /*******************************************
>>> > >> > > > > > >> > >> > > > > > >  Joe Stein
>>> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
>>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
>>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
>>> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
>>> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
>>> > >> > > > > > >> > >> > > >
>>> > >> > > > > > >> > >> > > > > > >
>>> ********************************************/
>>> > >> > > > > > >> > >> > > > > > >
>>> > >> > > > > > >> > >> > > > > >
>>> > >> > > > > > >> > >> > > > >
>>> > >> > > > > > >> > >> > > >
>>> > >> > > > > > >> > >> > >
>>> > >> > > > > > >> > >> >
>>> > >> > > > > > >> > >>
>>> > >> > > > > > >> > >>
>>> > >> > > > > > >> > >>
>>> > >> > > > > > >> > >> --
>>> > >> > > > > > >> > >> -- Guozhang
>>> > >> > > > > > >> > >>
>>> > >> > > > > > >> > >
>>> > >> > > > > > >> > >
>>> > >> > > > > > >> >
>>> > >> > > > > > >>
>>> > >> > > > > > >
>>> > >> > > > > > >
>>> > >> > > > > >
>>> > >> > > > >
>>> > >> > > >
>>> > >> > >
>>> > >> >
>>> > >> >
>>> > >> >
>>> > >> > --
>>> > >> > Jeff Holoman
>>> > >> > Systems Engineer
>>> > >> >
>>> > >>
>>>
>>>
>>
>>
>> --
>> -- Guozhang
>>
>
>
>
> --
> -- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Tong Li <li...@us.ibm.com>.
Guozhang,
     augmenting topic is fine, but as soon as we start doing that, other
issues follow, for example, access control, who can access the topic, who
can grant permissions. how the information (metadata) itself gets secured.
Should the information be saved in ZK or a datastore? Will using a metadata
file causing long term problems such as file updates/synchronization, once
we have this metadata file, more people will want to put more stuff in it.
how can we control the format? K-V pair not good for large data set.
    Clearly there is a need for it, I wonder if we can make this thing
plugable and provide a default implementation which allows us try different
solutions and also allow people to completely ignore it if they do not want
to deal with any of these.

Thanks.

Tong Li
OpenStack & Kafka Community Development
Building 501/B205
litong01@us.ibm.com



From:	Guozhang Wang <wa...@gmail.com>
To:	"dev@kafka.apache.org" <de...@kafka.apache.org>
Date:	03/12/2015 09:39 AM
Subject:	Re: [DISCUSS] KIP-4 - Command line and centralized
            administrative operations



Folks,

Just want to elaborate a bit more on the create-topic metadata and batching
describe-topic based on config / metadata in my previous email as we work
on KAFKA-1694. The main motivation is to have some sort of topic management
mechanisms, which I think is quite important in a multi-tenant / cloud
architecture: today anyone can create topics in a shared Kafka cluster, but
there is no concept or "ownership" of topics that are created by different
users. For example, at LinkedIn we basically distinguish topic owners via
some casual topic name prefix, which is a bit awkward and does not fly as
we scale our customers. It would be great to use describe-topics such as:

Describe all topics that is created by me.

Describe all topics whose retention time is overriden to X.

Describe all topics whose writable group include user Y (this is related to
authorization), etc..

One possible way to achieve this is to add a metadata file in the
create-topic request, whose value will also be written ZK as we create the
topic; then describe-topics can choose to batch topics based on 1) name
regex, 2) config K-V matching, 3) metadata regex, etc.

Thoughts?

Guozhang

On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Thanks for the updated wiki. A few comments below:
>
> 1. Error description in response: I think if some errorCode could
indicate
> several different error cases then we should really change it to multiple
> codes. In general the errorCode itself would be precise and sufficient
for
> describing the server side errors.
>
> 2. Describe topic request: it would be great to go beyond just batching
on
> topic name regex for this request. For example, a very common use case of
> the topic command is to list all topics whose config A's value is B. With
> topic name regex then we have to first retrieve __all__ topics's
> description info and then filter at the client end, which will be a huge
> burden on ZK.
>
> 3. Config K-Vs in create topic: this is related to the previous point;
> maybe we can add another metadata K-V or just a metadata string along
side
> with config K-V in create topic like we did for offset commit request.
This
> field can be quite useful in storing information like "owner" of the
topic
> who issue the create command, etc, which is quite important for a
> multi-tenant setting. Then in the describe topic request we can also
batch
> on regex of the metadata field.
>
> 4. Today all the admin operations are async in the sense that command
will
> return once it is written in ZK, and that is why we need extra
verification
> like testUtil.waitForTopicCreated() / verify partition reassignment
> request, etc. With admin requests we could add a flag to enable / disable
> synchronous requests; when it is turned on, the response will not return
> until the request has been completed. And for async requests we can add a
> "token" field in the response, and then only need a general "admin
> verification request" with the given token to check if the async request
> has been completed.
>
> 5. +1 for extending Metadata request to include controller / coordinator
> information, and then we can remove the ConsumerMetadata /
ClusterMetadata
> requests.
>
> Guozhang
>
> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com> wrote:
>
>> Thanks for sending that out Joe - I don't think I will be able to make
>> it today, so if notes can be sent out afterward that would be great.
>>
>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
>> > Thanks for sending this out Joe. Looking forward to chatting with
>> everyone :)
>> >
>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly>
wrote:
>> > > Hey, I just sent out a google hangout invite to all pmc, committers
>> and
>> > > everyone I found working on a KIP. If I missed anyone in the invite
>> please
>> > > let me know and can update it, np.
>> > >
>> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can get
>> INFRA
>> > > help to make a google account so we can manage better?
>> > >
>> > > To discuss
>> > >
>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement
+Proposals
>> > > in progress and related JIRA that are interdependent and common
work.
>> > >
>> > > ~ Joe Stein
>> > >
>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com>
>> wrote:
>> > >
>> > >> Let's stay on Google hangouts that will also record and make the
>> sessions
>> > >> available on youtube.
>> > >>
>> > >> -Jay
>> > >>
>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
>> jholoman@cloudera.com>
>> > >> wrote:
>> > >>
>> > >> > Jay / Joe
>> > >> >
>> > >> > We're happy to send out a Webex for this purpose. We could record
>> the
>> > >> > sessions if there is interest and publish them out.
>> > >> >
>> > >> > Thanks
>> > >> >
>> > >> > Jeff
>> > >> >
>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <ja...@gmail.com>
>> wrote:
>> > >> >
>> > >> > > Let's try to get the technical hang-ups sorted out, though. I
>> really
>> > >> > think
>> > >> > > there is some benefit to live discussion vs writing. I am
>> hopeful that
>> > >> if
>> > >> > > we post instructions and give ourselves a few attempts we can
>> get it
>> > >> > > working.
>> > >> > >
>> > >> > > Tuesday at that time would work for me...any objections?
>> > >> > >
>> > >> > > -Jay
>> > >> > >
>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein
<joe.stein@stealth.ly
>> >
>> > >> wrote:
>> > >> > >
>> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET /
10am
>> PT
>> > >> ????
>> > >> > > >
>> > >> > > > I don't mind google hangout but there is always some issue or
>> > >> whatever
>> > >> > so
>> > >> > > > we know the apache irc channel works. We can start there and
>> see how
>> > >> it
>> > >> > > > goes? We can pull transcripts too and associate to tickets if
>> need be
>> > >> > > makes
>> > >> > > > it helpful for things.
>> > >> > > >
>> > >> > > > ~ Joestein
>> > >> > > >
>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
>> jay.kreps@gmail.com>
>> > >> > wrote:
>> > >> > > >
>> > >> > > > > We'd talked about doing a Google Hangout to chat about
this.
>> What
>> > >> > about
>> > >> > > > > generalizing that a little further...I actually think it
>> would be
>> > >> > good
>> > >> > > > for
>> > >> > > > > everyone spending a reasonable chunk of their week on Kafka
>> stuff
>> > >> to
>> > >> > > > maybe
>> > >> > > > > sync up once a week. I think we could use time to talk
>> through
>> > >> design
>> > >> > > > > stuff, make sure we are on top of code reviews, talk
through
>> any
>> > >> > tricky
>> > >> > > > > issues, etc.
>> > >> > > > >
>> > >> > > > > We can make it publicly available so that any one can
follow
>> along
>> > >> > who
>> > >> > > > > likes.
>> > >> > > > >
>> > >> > > > > Any interest in doing this? If so I'll try to set it up
>> starting
>> > >> next
>> > >> > > > week.
>> > >> > > > >
>> > >> > > > > -Jay
>> > >> > > > >
>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > >
>> > >> > > > > > Hi all,
>> > >> > > > > >
>> > >> > > > > > I've updated KIP page, fixed / aligned document
structure.
>> Also I
>> > >> > > added
>> > >> > > > > > some
>> > >> > > > > > very initial proposal for AdminClient so we have
something
>> to
>> > >> start
>> > >> > > > from
>> > >> > > > > > while
>> > >> > > > > > discussing the KIP.
>> > >> > > > > >
>> > >> > > > > >
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line
+and+centralized+administrative+operations
>> > >> > > > > >
>> > >> > > > > > Thanks,
>> > >> > > > > > Andrii Biletskyi
>> > >> > > > > >
>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > > >
>> > >> > > > > > > Jay,
>> > >> > > > > > >
>> > >> > > > > > > Re error messages: you are right, in most cases client
>> will
>> > >> have
>> > >> > > > enough
>> > >> > > > > > > context to show descriptive error message. My concern
is
>> that
>> > >> we
>> > >> > > will
>> > >> > > > > > have
>> > >> > > > > > > to
>> > >> > > > > > > add lots of new error codes for each possible error. Of
>> course,
>> > >> > we
>> > >> > > > > could
>> > >> > > > > > > reuse
>> > >> > > > > > > some of existing like UknownTopicOrPartitionCode, but
we
>> will
>> > >> > also
>> > >> > > > need
>> > >> > > > > > to
>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
>> TopicConfigInvalid (both
>> > >> > for
>> > >> > > > > topic
>> > >> > > > > > > name and config, and probably user would like to know
>> what
>> > >> > exactly
>> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
>> > >> InternalError
>> > >> > > > (e.g.
>> > >> > > > > > > zookeeper failure) etc.
>> > >> > > > > > > And this is only for TopicCommand, we will also need to
>> add
>> > >> > similar
>> > >> > > > > stuff
>> > >> > > > > > > for
>> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up
>> with a
>> > >> > large
>> > >> > > > list
>> > >> > > > > > of
>> > >> > > > > > > error codes, used only in Admin protocol.
>> > >> > > > > > > Having said that, I agree my proposal is not consistent
>> with
>> > >> > other
>> > >> > > > > cases.
>> > >> > > > > > > Maybe we can find better solution or something
>> in-between.
>> > >> > > > > > >
>> > >> > > > > > > Re Hangout chat: I think it is a great idea. This way
we
>> can
>> > >> move
>> > >> > > on
>> > >> > > > > > > faster.
>> > >> > > > > > > Let's agree somehow on date/time so people can join.
>> Will work
>> > >> > for
>> > >> > > me
>> > >> > > > > > this
>> > >> > > > > > > and
>> > >> > > > > > > next week almost anytime if agreed in advance.
>> > >> > > > > > >
>> > >> > > > > > > Thanks,
>> > >> > > > > > > Andrii
>> > >> > > > > > >
>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
>> > >> jay.kreps@gmail.com>
>> > >> > > > > wrote:
>> > >> > > > > > >
>> > >> > > > > > >> Hey Andrii,
>> > >> > > > > > >>
>> > >> > > > > > >> Generally we can do good error handling without
needing
>> custom
>> > >> > > > > > server-side
>> > >> > > > > > >> messages. I.e. generally the client has the context to
>> know
>> > >> that
>> > >> > > if
>> > >> > > > it
>> > >> > > > > > got
>> > >> > > > > > >> an error that the topic doesn't exist to say "Topic X
>> doesn't
>> > >> > > exist"
>> > >> > > > > > >> rather
>> > >> > > > > > >> than "error code 14" (or whatever). Maybe there are
>> specific
>> > >> > cases
>> > >> > > > > where
>> > >> > > > > > >> this is hard? If we want to add server-side error
>> messages we
>> > >> > > really
>> > >> > > > > do
>> > >> > > > > > >> need to do this in a consistent way across the
protocol.
>> > >> > > > > > >>
>> > >> > > > > > >> I still have a bunch of open questions here from my
>> previous
>> > >> > > list. I
>> > >> > > > > > will
>> > >> > > > > > >> be out for the next few days for Strata though. Maybe
>> we could
>> > >> > do
>> > >> > > a
>> > >> > > > > > Google
>> > >> > > > > > >> Hangout chat on any open issues some time towards the
>> end of
>> > >> > next
>> > >> > > > week
>> > >> > > > > > for
>> > >> > > > > > >> anyone interested in this ticket? I have a feeling
that
>> might
>> > >> > > > progress
>> > >> > > > > > >> things a little faster than email--I think we could
talk
>> > >> through
>> > >> > > > those
>> > >> > > > > > >> issues I brought up fairly quickly...
>> > >> > > > > > >>
>> > >> > > > > > >> -Jay
>> > >> > > > > > >>
>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > > > >>
>> > >> > > > > > >> > Hi all,
>> > >> > > > > > >> >
>> > >> > > > > > >> > I'm trying to address some of the issues which were
>> > >> mentioned
>> > >> > > > > earlier
>> > >> > > > > > >> about
>> > >> > > > > > >> > Admin RQ/RP format. One of those was about batching
>> > >> > operations.
>> > >> > > > What
>> > >> > > > > > if
>> > >> > > > > > >> we
>> > >> > > > > > >> > follow TopicCommand approach and let people specify
>> > >> topic-name
>> > >> > > by
>> > >> > > > > > >> regexp -
>> > >> > > > > > >> > would that cover most of the use cases?
>> > >> > > > > > >> >
>> > >> > > > > > >> > Secondly, is what information should we generally
>> provide in
>> > >> > > Admin
>> > >> > > > > > >> > responses.
>> > >> > > > > > >> > I realize that Admin commands don't imply they will
>> be used
>> > >> > only
>> > >> > > > in
>> > >> > > > > > CLI
>> > >> > > > > > >> > but,
>> > >> > > > > > >> > it seems to me, CLI is a very important client of
this
>> > >> > feature.
>> > >> > > In
>> > >> > > > > > this
>> > >> > > > > > >> > case,
>> > >> > > > > > >> > seems logical, we would like to provide users with
>> rich
>> > >> > > experience
>> > >> > > > > in
>> > >> > > > > > >> terms
>> > >> > > > > > >> > of
>> > >> > > > > > >> > getting results / errors of the executed commands.
>> Usually
>> > >> we
>> > >> > > > supply
>> > >> > > > > > >> with
>> > >> > > > > > >> > responses only errorCode, which looks very limiting,
>> in case
>> > >> > of
>> > >> > > > CLI
>> > >> > > > > we
>> > >> > > > > > >> may
>> > >> > > > > > >> > want to print human readable error description.
>> > >> > > > > > >> >
>> > >> > > > > > >> > So, taking into account previous item about
batching,
>> what
>> > >> do
>> > >> > > you
>> > >> > > > > > think
>> > >> > > > > > >> > about
>> > >> > > > > > >> > having smth like:
>> > >> > > > > > >> >
>> > >> > > > > > >> > ('create' doesn't support regexp)
>> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions Replicas
>> > >> > > > > ReplicaAssignment
>> > >> > > > > > >> > [Config]
>> > >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
>> > >> > > > > > >> >   ErrorCode => int16
>> > >> > > > > > >> >   ErrorDescription => string (empty if successful)
>> > >> > > > > > >> >
>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
>> > >> > > ReplicaAssignment
>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
>> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
>> ErrorDescription]
>> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
>> > >> > > > > > >> >   CommandErrorCode => int16
>> > >> > > > > > >> >   CommandErrorDescription => string (nonempty in
case
>> of
>> > >> fatal
>> > >> > > > > error,
>> > >> > > > > > >> e.g.
>> > >> > > > > > >> > we couldn't get topics by regexp)
>> > >> > > > > > >> >
>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName TopicDescription
>> > >> ErrorCode
>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
>> CommandErrorDescription
>> > >> > > > > > >> >
>> > >> > > > > > >> > Also, any thoughts about our discussion regarding
>> re-routing
>> > >> > > > > facility?
>> > >> > > > > > >> In
>> > >> > > > > > >> > my
>> > >> > > > > > >> > understanding, it is like between augmenting
>> > >> > > TopicMetadataRequest
>> > >> > > > > > >> > (to include at least controllerId) and implementing
>> new
>> > >> > generic
>> > >> > > > > > >> re-routing
>> > >> > > > > > >> > facility so sending messages to controller will be
>> handled
>> > >> by
>> > >> > > it.
>> > >> > > > > > >> >
>> > >> > > > > > >> > Thanks,
>> > >> > > > > > >> > Andrii Biletskyi
>> > >> > > > > > >> >
>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > > > >> >
>> > >> > > > > > >> > > @Guozhang:
>> > >> > > > > > >> > > Thanks for your comments, I've answered some of
>> those. The
>> > >> > > main
>> > >> > > > > > thing
>> > >> > > > > > >> is
>> > >> > > > > > >> > > having merged request for
>> create-alter-delete-describe - I
>> > >> > > have
>> > >> > > > > some
>> > >> > > > > > >> > > concerns about this approach.
>> > >> > > > > > >> > >
>> > >> > > > > > >> > > @*Jay*:
>> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is also
>> one of
>> > >> > the
>> > >> > > > > > >> concerns.
>> > >> > > > > > >> > We
>> > >> > > > > > >> > > can solve it if we implement re-routing facility.
>> But I
>> > >> > agree
>> > >> > > > with
>> > >> > > > > > >> > > Guozhang - it will make clients' internals a
little
>> bit
>> > >> > easier
>> > >> > > > but
>> > >> > > > > > >> this
>> > >> > > > > > >> > > seems to be a complex logic to implement and
>> support then.
>> > >> > > > > > Especially
>> > >> > > > > > >> for
>> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing later
>> for
>> > >> these
>> > >> > > > > > >> requests).
>> > >> > > > > > >> > > Also people will tend to avoid this re-routing
>> facility
>> > >> and
>> > >> > > hold
>> > >> > > > > > local
>> > >> > > > > > >> > > cluster cache to ensure their high-priority
requests
>> > >> (which
>> > >> > > some
>> > >> > > > > of
>> > >> > > > > > >> the
>> > >> > > > > > >> > > admin requests are) not sent to some busy broker
>> where
>> > >> they
>> > >> > > wait
>> > >> > > > > to
>> > >> > > > > > be
>> > >> > > > > > >> > > routed to the correct one.
>> > >> > > > > > >> > > As pointed out by Jun here (
>> > >> > > > > > >> > >
>> > >> > > > > > >> >
>> > >> > > > > > >>
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>>
https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530

>> > >> > > > > > >> > )
>> > >> > > > > > >> > > to solve the issue we might introduce a message
>> type to
>> > >> get
>> > >> > > > > cluster
>> > >> > > > > > >> > state.
>> > >> > > > > > >> > > But I agree we can just update
>> TopicMetadataResponse to
>> > >> > > include
>> > >> > > > > > >> > > controllerId (and probably smth else).
>> > >> > > > > > >> > > What are you thougths?
>> > >> > > > > > >> > >
>> > >> > > > > > >> > > Thanks,
>> > >> > > > > > >> > > Andrii
>> > >> > > > > > >> > >
>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
>> > >> > > > > wangguoz@gmail.com>
>> > >> > > > > > >> > wrote:
>> > >> > > > > > >> > >
>> > >> > > > > > >> > >> I think for the topics commands we can actually
>> merge
>> > >> > > > > > >> > >> create/alter/delete/describe as one request type
>> since
>> > >> > their
>> > >> > > > > > formats
>> > >> > > > > > >> are
>> > >> > > > > > >> > >> very much similar, and keep list-topics and
others
>> like
>> > >> > > > > > >> > >> partition-reassignment /
preferred-leader-election
>> as
>> > >> > > separate
>> > >> > > > > > >> request
>> > >> > > > > > >> > >> types, I also left some other comments on the RB
(
>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
>> > >> > > > jay.kreps@gmail.com>
>> > >> > > > > > >> wrote:
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to just
>> have
>> > >> one
>> > >> > > "do
>> > >> > > > > > admin
>> > >> > > > > > >> > >> stuff"
>> > >> > > > > > >> > >> > command that has the union of all parameters.
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >> > What I am saying is that command line tools are
>> one
>> > >> > client
>> > >> > > of
>> > >> > > > > the
>> > >> > > > > > >> > >> > administrative apis, but these will be used in
a
>> number
>> > >> > of
>> > >> > > > > > >> scenarios
>> > >> > > > > > >> > so
>> > >> > > > > > >> > >> > they should make logical sense even in the
>> absence of
>> > >> the
>> > >> > > > > command
>> > >> > > > > > >> line
>> > >> > > > > > >> > >> > tool. Hence comments like trying to clarify the
>> > >> > > relationship
>> > >> > > > > > >> between
>> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds
>> of
>> > >> things
>> > >> > > > > really
>> > >> > > > > > >> need
>> > >> > > > > > >> > >> to be
>> > >> > > > > > >> > >> > thought through.
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >> > Hope that makes sense.
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >> > -Jay
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
>> Biletskyi <
>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >> > > Jay,
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> > > Thanks for answering. You understood
>> correctly, most
>> > >> of
>> > >> > > my
>> > >> > > > > > >> comments
>> > >> > > > > > >> > >> were
>> > >> > > > > > >> > >> > > related to your point 1) - about "well
>> thought-out"
>> > >> > apis.
>> > >> > > > > Also,
>> > >> > > > > > >> yes,
>> > >> > > > > > >> > >> as I
>> > >> > > > > > >> > >> > > understood we would like to introduce a
single
>> > >> unified
>> > >> > > CLI
>> > >> > > > > tool
>> > >> > > > > > >> with
>> > >> > > > > > >> > >> > > centralized server-side request handling for
>> lots of
>> > >> > > > existing
>> > >> > > > > > >> ones
>> > >> > > > > > >> > >> (incl.
>> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
>> > >> ReassignPartitions,
>> > >> > > smth
>> > >> > > > > > else
>> > >> > > > > > >> if
>> > >> > > > > > >> > >> added
>> > >> > > > > > >> > >> > > in future). In our previous discussion (
>> > >> > > > > > >> > >> > >
>> https://issues.apache.org/jira/browse/KAFKA-1694)
>> > >> > people
>> > >> > > > > said
>> > >> > > > > > >> > they'd
>> > >> > > > > > >> > >> > > rather
>> > >> > > > > > >> > >> > > have a separate message for each command, so,
>> yes,
>> > >> this
>> > >> > > > way I
>> > >> > > > > > >> came
>> > >> > > > > > >> > to
>> > >> > > > > > >> > >> 1-1
>> > >> > > > > > >> > >> > > mapping between commands in the tool and
>> protocol
>> > >> > > > additions.
>> > >> > > > > > But
>> > >> > > > > > >> I
>> > >> > > > > > >> > >> might
>> > >> > > > > > >> > >> > be
>> > >> > > > > > >> > >> > > wrong.
>> > >> > > > > > >> > >> > > At the end I just try to start discussion how
>> at
>> > >> least
>> > >> > > > > > generally
>> > >> > > > > > >> > this
>> > >> > > > > > >> > >> > > protocol should look like.
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> > > Thanks,
>> > >> > > > > > >> > >> > > Andrii
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
>> > >> > > > > > jay.kreps@gmail.com
>> > >> > > > > > >> >
>> > >> > > > > > >> > >> wrote:
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> > > > Hey Andrii,
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > To answer your earlier question we just
>> really
>> > >> can't
>> > >> > be
>> > >> > > > > > adding
>> > >> > > > > > >> any
>> > >> > > > > > >> > >> more
>> > >> > > > > > >> > >> > > > scala protocol objects. These things are
>> super hard
>> > >> > to
>> > >> > > > > > maintain
>> > >> > > > > > >> > >> because
>> > >> > > > > > >> > >> > > > they hand code the byte parsing and don't
>> have good
>> > >> > > > > > versioning
>> > >> > > > > > >> > >> support.
>> > >> > > > > > >> > >> > > > Since we are already planning on converting
>> we
>> > >> > > definitely
>> > >> > > > > > don't
>> > >> > > > > > >> > >> want to
>> > >> > > > > > >> > >> > > add
>> > >> > > > > > >> > >> > > > a ton more of these--they are total tech
>> debt.
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > What does it mean that the changes are
>> isolated
>> > >> from
>> > >> > > the
>> > >> > > > > > >> current
>> > >> > > > > > >> > >> code
>> > >> > > > > > >> > >> > > base?
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > I actually didn't understand the remaining
>> > >> comments,
>> > >> > > > which
>> > >> > > > > of
>> > >> > > > > > >> the
>> > >> > > > > > >> > >> > points
>> > >> > > > > > >> > >> > > > are you responding to?
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > Maybe one sticking point here is that it
>> seems like
>> > >> > you
>> > >> > > > > want
>> > >> > > > > > to
>> > >> > > > > > >> > make
>> > >> > > > > > >> > >> > some
>> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1
mapping
>> > >> between
>> > >> > > > > > commands
>> > >> > > > > > >> you
>> > >> > > > > > >> > >> > > imagine
>> > >> > > > > > >> > >> > > > in the tool and protocol additions. I want
>> to make
>> > >> > sure
>> > >> > > > we
>> > >> > > > > > >> don't
>> > >> > > > > > >> > do
>> > >> > > > > > >> > >> > that.
>> > >> > > > > > >> > >> > > > The protocol needs to be really really well
>> thought
>> > >> > out
>> > >> > > > > > against
>> > >> > > > > > >> > many
>> > >> > > > > > >> > >> > use
>> > >> > > > > > >> > >> > > > cases so it should make perfect logical
>> sense in
>> > >> the
>> > >> > > > > absence
>> > >> > > > > > of
>> > >> > > > > > >> > >> knowing
>> > >> > > > > > >> > >> > > the
>> > >> > > > > > >> > >> > > > command line tool, right?
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > -Jay
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii
>> Biletskyi
>> > >> <
>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > > Hey Jay,
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > I would like to continue this discussion
>> as it
>> > >> seem
>> > >> > > > there
>> > >> > > > > > is
>> > >> > > > > > >> no
>> > >> > > > > > >> > >> > > progress
>> > >> > > > > > >> > >> > > > > here.
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > First of all, could you please explain
>> what did
>> > >> you
>> > >> > > > mean
>> > >> > > > > in
>> > >> > > > > > >> 2?
>> > >> > > > > > >> > How
>> > >> > > > > > >> > >> > > > exactly
>> > >> > > > > > >> > >> > > > > are we going to migrate to the new java
>> protocol
>> > >> > > > > > definitions.
>> > >> > > > > > >> > And
>> > >> > > > > > >> > >> why
>> > >> > > > > > >> > >> > > > it's
>> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > I agree with you, this feature includes
>> lots of
>> > >> > > stuff,
>> > >> > > > > but
>> > >> > > > > > >> > >> thankfully
>> > >> > > > > > >> > >> > > > > almost all changes are isolated from the
>> current
>> > >> > code
>> > >> > > > > base,
>> > >> > > > > > >> > >> > > > > so the main thing, I think, we need to
>> agree is
>> > >> > RQ/RP
>> > >> > > > > > format.
>> > >> > > > > > >> > >> > > > > So how can we start discussion about the
>> concrete
>> > >> > > > > messages
>> > >> > > > > > >> > format?
>> > >> > > > > > >> > >> > > > > Can we take (
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >>
>> > >> > > > > > >> >
>> > >> > > > > > >>
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line
+and+centralized+administrative
+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat

>> > >> > > > > > >> > >> > > > > )
>> > >> > > > > > >> > >> > > > > as starting point?
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > We had some doubts earlier whether it
worth
>> > >> > > introducing
>> > >> > > > > one
>> > >> > > > > > >> > >> generic
>> > >> > > > > > >> > >> > > Admin
>> > >> > > > > > >> > >> > > > > Request for all commands (
>> > >> > > > > > >> > >> > > >
>> https://issues.apache.org/jira/browse/KAFKA-1694
>> > >> > > > > > >> > >> > > > > )
>> > >> > > > > > >> > >> > > > > but then everybody agreed it would be
>> better to
>> > >> > have
>> > >> > > > > > separate
>> > >> > > > > > >> > >> message
>> > >> > > > > > >> > >> > > for
>> > >> > > > > > >> > >> > > > > each admin command. The Request part is
>> really
>> > >> > > dictated
>> > >> > > > > > from
>> > >> > > > > > >> the
>> > >> > > > > > >> > >> > > command
>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so
>> the
>> > >> > proposed
>> > >> > > > > > version
>> > >> > > > > > >> > >> should
>> > >> > > > > > >> > >> > be
>> > >> > > > > > >> > >> > > > > fine (let's put aside for now remarks
about
>> > >> > Optional
>> > >> > > > > type,
>> > >> > > > > > >> > >> batching,
>> > >> > > > > > >> > >> > > > > configs normalization - I agree with all
of
>> > >> them).
>> > >> > > > > > >> > >> > > > > So the second part is Response. I see
>> there are
>> > >> two
>> > >> > > > cases
>> > >> > > > > > >> here.
>> > >> > > > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ;
>> b)
>> > >> "Get"
>> > >> > > > > > requests -
>> > >> > > > > > >> > >> > > > > List/Describe...
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > a) should only hold request result
>> (regardless
>> > >> what
>> > >> > > we
>> > >> > > > > > decide
>> > >> > > > > > >> > >> about
>> > >> > > > > > >> > >> > > > > blocking/non-blocking commands
execution).
>> > >> > > > > > >> > >> > > > > Usually we provide error code in response
>> but
>> > >> since
>> > >> > > we
>> > >> > > > > will
>> > >> > > > > > >> use
>> > >> > > > > > >> > >> this
>> > >> > > > > > >> > >> > in
>> > >> > > > > > >> > >> > > > > interactive shell we need some human
>> readable
>> > >> error
>> > >> > > > > > >> description
>> > >> > > > > > >> > -
>> > >> > > > > > >> > >> so
>> > >> > > > > > >> > >> > I
>> > >> > > > > > >> > >> > > > > added errorDesription field where you can
>> at
>> > >> least
>> > >> > > > leave
>> > >> > > > > > >> > >> > > > > exception.getMessage.
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > b) in addition to previous item message
>> should
>> > >> hold
>> > >> > > > > command
>> > >> > > > > > >> > >> specific
>> > >> > > > > > >> > >> > > > > response data. We can discuss in detail
>> each of
>> > >> > them
>> > >> > > > but
>> > >> > > > > > >> let's
>> > >> > > > > > >> > for
>> > >> > > > > > >> > >> > now
>> > >> > > > > > >> > >> > > > > agree about the overall pattern.
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > Thanks,
>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay
Kreps
>> <
>> > >> > > > > > >> jay.kreps@gmail.com
>> > >> > > > > > >> > >
>> > >> > > > > > >> > >> > > wrote:
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > > Hey Joe,
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > This is great. A few comments on KIP-4
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 1. This is much needed functionality,
>> but there
>> > >> > > are a
>> > >> > > > > lot
>> > >> > > > > > >> of
>> > >> > > > > > >> > >> the so
>> > >> > > > > > >> > >> > > > let's
>> > >> > > > > > >> > >> > > > > > really think these protocols through.
We
>> really
>> > >> > > want
>> > >> > > > to
>> > >> > > > > > >> end up
>> > >> > > > > > >> > >> > with a
>> > >> > > > > > >> > >> > > > set
>> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis.
>> For this
>> > >> > > > reason I
>> > >> > > > > > >> think
>> > >> > > > > > >> > >> it is
>> > >> > > > > > >> > >> > > > > really
>> > >> > > > > > >> > >> > > > > > important to think through the end
state
>> even
>> > >> if
>> > >> > > that
>> > >> > > > > > >> includes
>> > >> > > > > > >> > >> APIs
>> > >> > > > > > >> > >> > > we
>> > >> > > > > > >> > >> > > > > > won't implement in the first phase.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait
until
>> we
>> > >> have
>> > >> > > > > switched
>> > >> > > > > > >> the
>> > >> > > > > > >> > >> > server
>> > >> > > > > > >> > >> > > > over
>> > >> > > > > > >> > >> > > > > > to the new java protocol definitions.
If
>> we add
>> > >> > > > upteen
>> > >> > > > > > >> more ad
>> > >> > > > > > >> > >> hoc
>> > >> > > > > > >> > >> > > > scala
>> > >> > > > > > >> > >> > > > > > objects that is just generating more
>> work for
>> > >> the
>> > >> > > > > > >> conversion
>> > >> > > > > > >> > we
>> > >> > > > > > >> > >> > know
>> > >> > > > > > >> > >> > > we
>> > >> > > > > > >> > >> > > > > > have to do.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new type
of
>> > >> > optional
>> > >> > > > > > >> parameter.
>> > >> > > > > > >> > >> This
>> > >> > > > > > >> > >> > is
>> > >> > > > > > >> > >> > > > > > inconsistent with everything else in
the
>> > >> protocol
>> > >> > > > where
>> > >> > > > > > we
>> > >> > > > > > >> use
>> > >> > > > > > >> > >> -1
>> > >> > > > > > >> > >> > or
>> > >> > > > > > >> > >> > > > some
>> > >> > > > > > >> > >> > > > > > other marker value. You could argue
>> either way
>> > >> > but
>> > >> > > > > let's
>> > >> > > > > > >> stick
>> > >> > > > > > >> > >> with
>> > >> > > > > > >> > >> > > > that
>> > >> > > > > > >> > >> > > > > > for consistency. For clients that
>> implemented
>> > >> the
>> > >> > > > > > protocol
>> > >> > > > > > >> in
>> > >> > > > > > >> > a
>> > >> > > > > > >> > >> > > better
>> > >> > > > > > >> > >> > > > > way
>> > >> > > > > > >> > >> > > > > > than our scala code these basic
>> primitives are
>> > >> > hard
>> > >> > > > to
>> > >> > > > > > >> change.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
>> duplicate
>> > >> > > > > > >> > TopicMetadataRequest
>> > >> > > > > > >> > >> > > which
>> > >> > > > > > >> > >> > > > > has
>> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I
think
>> we
>> > >> > should
>> > >> > > > > rename
>> > >> > > > > > >> that
>> > >> > > > > > >> > >> > > request
>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
>> > >> MetadataRequest)
>> > >> > > and
>> > >> > > > > > >> include
>> > >> > > > > > >> > >> the id
>> > >> > > > > > >> > >> > > of
>> > >> > > > > > >> > >> > > > > the
>> > >> > > > > > >> > >> > > > > > controller. Or are there other things
we
>> could
>> > >> > add
>> > >> > > > > here?
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to make a
>> lot of
>> > >> > > > requests
>> > >> > > > > > that
>> > >> > > > > > >> > can
>> > >> > > > > > >> > >> > only
>> > >> > > > > > >> > >> > > go
>> > >> > > > > > >> > >> > > > > to
>> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of
>> burden for
>> > >> > > > client
>> > >> > > > > > >> > >> > > implementations
>> > >> > > > > > >> > >> > > > > (it
>> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can fail
>> in many
>> > >> > > parts
>> > >> > > > > so
>> > >> > > > > > it
>> > >> > > > > > >> > >> ends up
>> > >> > > > > > >> > >> > > > > being a
>> > >> > > > > > >> > >> > > > > > full state machine to do right). I
think
>> we
>> > >> > should
>> > >> > > > > > consider
>> > >> > > > > > >> > >> making
>> > >> > > > > > >> > >> > > > admin
>> > >> > > > > > >> > >> > > > > > commands and ideally as many of the
>> other apis
>> > >> as
>> > >> > > > > > possible
>> > >> > > > > > >> > >> > available
>> > >> > > > > > >> > >> > > on
>> > >> > > > > > >> > >> > > > > all
>> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
>> controller on
>> > >> > the
>> > >> > > > > broker
>> > >> > > > > > >> > side.
>> > >> > > > > > >> > >> > > Perhaps
>> > >> > > > > > >> > >> > > > > > there would be a general way to
>> encapsulate
>> > >> this
>> > >> > > > > > re-routing
>> > >> > > > > > >> > >> > behavior.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 6. We should probably normalize the key
>> value
>> > >> > pairs
>> > >> > > > > used
>> > >> > > > > > >> for
>> > >> > > > > > >> > >> > configs
>> > >> > > > > > >> > >> > > > > rather
>> > >> > > > > > >> > >> > > > > > than embedding a new formatting. So two
>> strings
>> > >> > > > rather
>> > >> > > > > > than
>> > >> > > > > > >> > one
>> > >> > > > > > >> > >> > with
>> > >> > > > > > >> > >> > > an
>> > >> > > > > > >> > >> > > > > > internal equals sign.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs
>> that the
>> > >> > > > command
>> > >> > > > > > has
>> > >> > > > > > >> > >> begun or
>> > >> > > > > > >> > >> > > > that
>> > >> > > > > > >> > >> > > > > > the command has been completed? It is a
>> lot
>> > >> more
>> > >> > > > usable
>> > >> > > > > > if
>> > >> > > > > > >> the
>> > >> > > > > > >> > >> > > command
>> > >> > > > > > >> > >> > > > > has
>> > >> > > > > > >> > >> > > > > > been completed so you know that if you
>> create a
>> > >> > > topic
>> > >> > > > > and
>> > >> > > > > > >> then
>> > >> > > > > > >> > >> > > publish
>> > >> > > > > > >> > >> > > > to
>> > >> > > > > > >> > >> > > > > > it you won't get an exception about
>> there being
>> > >> > no
>> > >> > > > such
>> > >> > > > > > >> topic.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
>> duplicate a
>> > >> lot
>> > >> > > of
>> > >> > > > > > stuff
>> > >> > > > > > >> in
>> > >> > > > > > >> > >> the
>> > >> > > > > > >> > >> > > > > metadata
>> > >> > > > > > >> > >> > > > > > request. Is there a reason to give back
>> topics
>> > >> > > marked
>> > >> > > > > for
>> > >> > > > > > >> > >> > deletion? I
>> > >> > > > > > >> > >> > > > > feel
>> > >> > > > > > >> > >> > > > > > like if we just make the post-condition
>> of the
>> > >> > > delete
>> > >> > > > > > >> command
>> > >> > > > > > >> > be
>> > >> > > > > > >> > >> > that
>> > >> > > > > > >> > >> > > > the
>> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid of
>> the need
>> > >> > for
>> > >> > > > this
>> > >> > > > > > >> right?
>> > >> > > > > > >> > >> And
>> > >> > > > > > >> > >> > it
>> > >> > > > > > >> > >> > > > > will
>> > >> > > > > > >> > >> > > > > > be much more intuitive.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 9. Should we consider batching these
>> requests?
>> > >> We
>> > >> > > > have
>> > >> > > > > > >> > generally
>> > >> > > > > > >> > >> > > tried
>> > >> > > > > > >> > >> > > > to
>> > >> > > > > > >> > >> > > > > > allow multiple operations to be
batched.
>> My
>> > >> > > suspicion
>> > >> > > > > is
>> > >> > > > > > >> that
>> > >> > > > > > >> > >> > without
>> > >> > > > > > >> > >> > > > > this
>> > >> > > > > > >> > >> > > > > > we will get a lot of code that does
>> something
>> > >> > like
>> > >> > > > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
>> > >> > > > > > >> > >> > > > > >       adminClient.describeTopic(topic)
>> > >> > > > > > >> > >> > > > > > this code will work great when you test
>> on 5
>> > >> > topics
>> > >> > > > but
>> > >> > > > > > >> not do
>> > >> > > > > > >> > >> as
>> > >> > > > > > >> > >> > > well
>> > >> > > > > > >> > >> > > > if
>> > >> > > > > > >> > >> > > > > > you have 50k.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss how
>> we want
>> > >> to
>> > >> > > > > expose
>> > >> > > > > > a
>> > >> > > > > > >> > >> > > programmatic
>> > >> > > > > > >> > >> > > > > JVM
>> > >> > > > > > >> > >> > > > > > client api for these operations.
>> Currently
>> > >> people
>> > >> > > > rely
>> > >> > > > > on
>> > >> > > > > > >> > >> > AdminUtils
>> > >> > > > > > >> > >> > > > > which
>> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we probably
>> need
>> > >> > > another
>> > >> > > > > > client
>> > >> > > > > > >> > >> under
>> > >> > > > > > >> > >> > > > > clients/
>> > >> > > > > > >> > >> > > > > > that exposes administrative
>> functionality. We
>> > >> > will
>> > >> > > > need
>> > >> > > > > > >> this
>> > >> > > > > > >> > >> just
>> > >> > > > > > >> > >> > to
>> > >> > > > > > >> > >> > > > > > properly test the new apis, I suspect.
We
>> > >> should
>> > >> > > > figure
>> > >> > > > > > out
>> > >> > > > > > >> > that
>> > >> > > > > > >> > >> > API.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 11. The other information that would be
>> really
>> > >> > > useful
>> > >> > > > > to
>> > >> > > > > > >> get
>> > >> > > > > > >> > >> would
>> > >> > > > > > >> > >> > be
>> > >> > > > > > >> > >> > > > > > information about partitions--how much
>> data is
>> > >> in
>> > >> > > the
>> > >> > > > > > >> > partition,
>> > >> > > > > > >> > >> > what
>> > >> > > > > > >> > >> > > > are
>> > >> > > > > > >> > >> > > > > > the segment offsets, what is the
log-end
>> offset
>> > >> > > (i.e.
>> > >> > > > > > last
>> > >> > > > > > >> > >> offset),
>> > >> > > > > > >> > >> > > > what
>> > >> > > > > > >> > >> > > > > is
>> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think that
>> done
>> > >> > right
>> > >> > > > this
>> > >> > > > > > >> would
>> > >> > > > > > >> > be
>> > >> > > > > > >> > >> > the
>> > >> > > > > > >> > >> > > > > > successor to the very awkward
>> OffsetRequest we
>> > >> > have
>> > >> > > > > > today.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > -Jay
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe
>> Stein <
>> > >> > > > > > >> > >> joe.stein@stealth.ly>
>> > >> > > > > > >> > >> > > > > wrote:
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
>> > >> > > > > > >> > >> > > > > > >
>> > >> > > > > > >> > >> > > > > > >
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >>
>> > >> > > > > > >> >
>> > >> > > > > > >>
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line
+and+centralized+administrative+operations
>> > >> > > > > > >> > >> > > > > > >
>> > >> > > > > > >> > >> > > > > > > JIRA
>> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
>> > >> > > > > > >> > >> > > > > > >
>> > >> > > > > > >> > >> > > > > > >
>> /*******************************************
>> > >> > > > > > >> > >> > > > > > >  Joe Stein
>> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
>> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
>> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > > > >
>> ********************************************/
>> > >> > > > > > >> > >> > > > > > >
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >> --
>> > >> > > > > > >> > >> -- Guozhang
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >
>> > >> > > > > > >> > >
>> > >> > > > > > >> >
>> > >> > > > > > >>
>> > >> > > > > > >
>> > >> > > > > > >
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >> >
>> > >> >
>> > >> > --
>> > >> > Jeff Holoman
>> > >> > Systems Engineer
>> > >> >
>> > >>
>>
>>
>
>
> --
> -- Guozhang
>



--
-- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Guozhang Wang <wa...@gmail.com>.
Folks,

Just want to elaborate a bit more on the create-topic metadata and batching
describe-topic based on config / metadata in my previous email as we work
on KAFKA-1694. The main motivation is to have some sort of topic management
mechanisms, which I think is quite important in a multi-tenant / cloud
architecture: today anyone can create topics in a shared Kafka cluster, but
there is no concept or "ownership" of topics that are created by different
users. For example, at LinkedIn we basically distinguish topic owners via
some casual topic name prefix, which is a bit awkward and does not fly as
we scale our customers. It would be great to use describe-topics such as:

Describe all topics that is created by me.

Describe all topics whose retention time is overriden to X.

Describe all topics whose writable group include user Y (this is related to
authorization), etc..

One possible way to achieve this is to add a metadata file in the
create-topic request, whose value will also be written ZK as we create the
topic; then describe-topics can choose to batch topics based on 1) name
regex, 2) config K-V matching, 3) metadata regex, etc.

Thoughts?

Guozhang

On Thu, Mar 5, 2015 at 4:37 PM, Guozhang Wang <wa...@gmail.com> wrote:

> Thanks for the updated wiki. A few comments below:
>
> 1. Error description in response: I think if some errorCode could indicate
> several different error cases then we should really change it to multiple
> codes. In general the errorCode itself would be precise and sufficient for
> describing the server side errors.
>
> 2. Describe topic request: it would be great to go beyond just batching on
> topic name regex for this request. For example, a very common use case of
> the topic command is to list all topics whose config A's value is B. With
> topic name regex then we have to first retrieve __all__ topics's
> description info and then filter at the client end, which will be a huge
> burden on ZK.
>
> 3. Config K-Vs in create topic: this is related to the previous point;
> maybe we can add another metadata K-V or just a metadata string along side
> with config K-V in create topic like we did for offset commit request. This
> field can be quite useful in storing information like "owner" of the topic
> who issue the create command, etc, which is quite important for a
> multi-tenant setting. Then in the describe topic request we can also batch
> on regex of the metadata field.
>
> 4. Today all the admin operations are async in the sense that command will
> return once it is written in ZK, and that is why we need extra verification
> like testUtil.waitForTopicCreated() / verify partition reassignment
> request, etc. With admin requests we could add a flag to enable / disable
> synchronous requests; when it is turned on, the response will not return
> until the request has been completed. And for async requests we can add a
> "token" field in the response, and then only need a general "admin
> verification request" with the given token to check if the async request
> has been completed.
>
> 5. +1 for extending Metadata request to include controller / coordinator
> information, and then we can remove the ConsumerMetadata / ClusterMetadata
> requests.
>
> Guozhang
>
> On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com> wrote:
>
>> Thanks for sending that out Joe - I don't think I will be able to make
>> it today, so if notes can be sent out afterward that would be great.
>>
>> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
>> > Thanks for sending this out Joe. Looking forward to chatting with
>> everyone :)
>> >
>> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly> wrote:
>> > > Hey, I just sent out a google hangout invite to all pmc, committers
>> and
>> > > everyone I found working on a KIP. If I missed anyone in the invite
>> please
>> > > let me know and can update it, np.
>> > >
>> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can get
>> INFRA
>> > > help to make a google account so we can manage better?
>> > >
>> > > To discuss
>> > >
>> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
>> > > in progress and related JIRA that are interdependent and common work.
>> > >
>> > > ~ Joe Stein
>> > >
>> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com>
>> wrote:
>> > >
>> > >> Let's stay on Google hangouts that will also record and make the
>> sessions
>> > >> available on youtube.
>> > >>
>> > >> -Jay
>> > >>
>> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <
>> jholoman@cloudera.com>
>> > >> wrote:
>> > >>
>> > >> > Jay / Joe
>> > >> >
>> > >> > We're happy to send out a Webex for this purpose. We could record
>> the
>> > >> > sessions if there is interest and publish them out.
>> > >> >
>> > >> > Thanks
>> > >> >
>> > >> > Jeff
>> > >> >
>> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <ja...@gmail.com>
>> wrote:
>> > >> >
>> > >> > > Let's try to get the technical hang-ups sorted out, though. I
>> really
>> > >> > think
>> > >> > > there is some benefit to live discussion vs writing. I am
>> hopeful that
>> > >> if
>> > >> > > we post instructions and give ourselves a few attempts we can
>> get it
>> > >> > > working.
>> > >> > >
>> > >> > > Tuesday at that time would work for me...any objections?
>> > >> > >
>> > >> > > -Jay
>> > >> > >
>> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <joe.stein@stealth.ly
>> >
>> > >> wrote:
>> > >> > >
>> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am
>> PT
>> > >> ????
>> > >> > > >
>> > >> > > > I don't mind google hangout but there is always some issue or
>> > >> whatever
>> > >> > so
>> > >> > > > we know the apache irc channel works. We can start there and
>> see how
>> > >> it
>> > >> > > > goes? We can pull transcripts too and associate to tickets if
>> need be
>> > >> > > makes
>> > >> > > > it helpful for things.
>> > >> > > >
>> > >> > > > ~ Joestein
>> > >> > > >
>> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
>> jay.kreps@gmail.com>
>> > >> > wrote:
>> > >> > > >
>> > >> > > > > We'd talked about doing a Google Hangout to chat about this.
>> What
>> > >> > about
>> > >> > > > > generalizing that a little further...I actually think it
>> would be
>> > >> > good
>> > >> > > > for
>> > >> > > > > everyone spending a reasonable chunk of their week on Kafka
>> stuff
>> > >> to
>> > >> > > > maybe
>> > >> > > > > sync up once a week. I think we could use time to talk
>> through
>> > >> design
>> > >> > > > > stuff, make sure we are on top of code reviews, talk through
>> any
>> > >> > tricky
>> > >> > > > > issues, etc.
>> > >> > > > >
>> > >> > > > > We can make it publicly available so that any one can follow
>> along
>> > >> > who
>> > >> > > > > likes.
>> > >> > > > >
>> > >> > > > > Any interest in doing this? If so I'll try to set it up
>> starting
>> > >> next
>> > >> > > > week.
>> > >> > > > >
>> > >> > > > > -Jay
>> > >> > > > >
>> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
>> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > >
>> > >> > > > > > Hi all,
>> > >> > > > > >
>> > >> > > > > > I've updated KIP page, fixed / aligned document structure.
>> Also I
>> > >> > > added
>> > >> > > > > > some
>> > >> > > > > > very initial proposal for AdminClient so we have something
>> to
>> > >> start
>> > >> > > > from
>> > >> > > > > > while
>> > >> > > > > > discussing the KIP.
>> > >> > > > > >
>> > >> > > > > >
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>> > >> > > > > >
>> > >> > > > > > Thanks,
>> > >> > > > > > Andrii Biletskyi
>> > >> > > > > >
>> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
>> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > > >
>> > >> > > > > > > Jay,
>> > >> > > > > > >
>> > >> > > > > > > Re error messages: you are right, in most cases client
>> will
>> > >> have
>> > >> > > > enough
>> > >> > > > > > > context to show descriptive error message. My concern is
>> that
>> > >> we
>> > >> > > will
>> > >> > > > > > have
>> > >> > > > > > > to
>> > >> > > > > > > add lots of new error codes for each possible error. Of
>> course,
>> > >> > we
>> > >> > > > > could
>> > >> > > > > > > reuse
>> > >> > > > > > > some of existing like UknownTopicOrPartitionCode, but we
>> will
>> > >> > also
>> > >> > > > need
>> > >> > > > > > to
>> > >> > > > > > > add smth like: TopicAlreadyExistsCode,
>> TopicConfigInvalid (both
>> > >> > for
>> > >> > > > > topic
>> > >> > > > > > > name and config, and probably user would like to know
>> what
>> > >> > exactly
>> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
>> > >> InternalError
>> > >> > > > (e.g.
>> > >> > > > > > > zookeeper failure) etc.
>> > >> > > > > > > And this is only for TopicCommand, we will also need to
>> add
>> > >> > similar
>> > >> > > > > stuff
>> > >> > > > > > > for
>> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up
>> with a
>> > >> > large
>> > >> > > > list
>> > >> > > > > > of
>> > >> > > > > > > error codes, used only in Admin protocol.
>> > >> > > > > > > Having said that, I agree my proposal is not consistent
>> with
>> > >> > other
>> > >> > > > > cases.
>> > >> > > > > > > Maybe we can find better solution or something
>> in-between.
>> > >> > > > > > >
>> > >> > > > > > > Re Hangout chat: I think it is a great idea. This way we
>> can
>> > >> move
>> > >> > > on
>> > >> > > > > > > faster.
>> > >> > > > > > > Let's agree somehow on date/time so people can join.
>> Will work
>> > >> > for
>> > >> > > me
>> > >> > > > > > this
>> > >> > > > > > > and
>> > >> > > > > > > next week almost anytime if agreed in advance.
>> > >> > > > > > >
>> > >> > > > > > > Thanks,
>> > >> > > > > > > Andrii
>> > >> > > > > > >
>> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
>> > >> jay.kreps@gmail.com>
>> > >> > > > > wrote:
>> > >> > > > > > >
>> > >> > > > > > >> Hey Andrii,
>> > >> > > > > > >>
>> > >> > > > > > >> Generally we can do good error handling without needing
>> custom
>> > >> > > > > > server-side
>> > >> > > > > > >> messages. I.e. generally the client has the context to
>> know
>> > >> that
>> > >> > > if
>> > >> > > > it
>> > >> > > > > > got
>> > >> > > > > > >> an error that the topic doesn't exist to say "Topic X
>> doesn't
>> > >> > > exist"
>> > >> > > > > > >> rather
>> > >> > > > > > >> than "error code 14" (or whatever). Maybe there are
>> specific
>> > >> > cases
>> > >> > > > > where
>> > >> > > > > > >> this is hard? If we want to add server-side error
>> messages we
>> > >> > > really
>> > >> > > > > do
>> > >> > > > > > >> need to do this in a consistent way across the protocol.
>> > >> > > > > > >>
>> > >> > > > > > >> I still have a bunch of open questions here from my
>> previous
>> > >> > > list. I
>> > >> > > > > > will
>> > >> > > > > > >> be out for the next few days for Strata though. Maybe
>> we could
>> > >> > do
>> > >> > > a
>> > >> > > > > > Google
>> > >> > > > > > >> Hangout chat on any open issues some time towards the
>> end of
>> > >> > next
>> > >> > > > week
>> > >> > > > > > for
>> > >> > > > > > >> anyone interested in this ticket? I have a feeling that
>> might
>> > >> > > > progress
>> > >> > > > > > >> things a little faster than email--I think we could talk
>> > >> through
>> > >> > > > those
>> > >> > > > > > >> issues I brought up fairly quickly...
>> > >> > > > > > >>
>> > >> > > > > > >> -Jay
>> > >> > > > > > >>
>> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
>> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > > > >>
>> > >> > > > > > >> > Hi all,
>> > >> > > > > > >> >
>> > >> > > > > > >> > I'm trying to address some of the issues which were
>> > >> mentioned
>> > >> > > > > earlier
>> > >> > > > > > >> about
>> > >> > > > > > >> > Admin RQ/RP format. One of those was about batching
>> > >> > operations.
>> > >> > > > What
>> > >> > > > > > if
>> > >> > > > > > >> we
>> > >> > > > > > >> > follow TopicCommand approach and let people specify
>> > >> topic-name
>> > >> > > by
>> > >> > > > > > >> regexp -
>> > >> > > > > > >> > would that cover most of the use cases?
>> > >> > > > > > >> >
>> > >> > > > > > >> > Secondly, is what information should we generally
>> provide in
>> > >> > > Admin
>> > >> > > > > > >> > responses.
>> > >> > > > > > >> > I realize that Admin commands don't imply they will
>> be used
>> > >> > only
>> > >> > > > in
>> > >> > > > > > CLI
>> > >> > > > > > >> > but,
>> > >> > > > > > >> > it seems to me, CLI is a very important client of this
>> > >> > feature.
>> > >> > > In
>> > >> > > > > > this
>> > >> > > > > > >> > case,
>> > >> > > > > > >> > seems logical, we would like to provide users with
>> rich
>> > >> > > experience
>> > >> > > > > in
>> > >> > > > > > >> terms
>> > >> > > > > > >> > of
>> > >> > > > > > >> > getting results / errors of the executed commands.
>> Usually
>> > >> we
>> > >> > > > supply
>> > >> > > > > > >> with
>> > >> > > > > > >> > responses only errorCode, which looks very limiting,
>> in case
>> > >> > of
>> > >> > > > CLI
>> > >> > > > > we
>> > >> > > > > > >> may
>> > >> > > > > > >> > want to print human readable error description.
>> > >> > > > > > >> >
>> > >> > > > > > >> > So, taking into account previous item about batching,
>> what
>> > >> do
>> > >> > > you
>> > >> > > > > > think
>> > >> > > > > > >> > about
>> > >> > > > > > >> > having smth like:
>> > >> > > > > > >> >
>> > >> > > > > > >> > ('create' doesn't support regexp)
>> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions Replicas
>> > >> > > > > ReplicaAssignment
>> > >> > > > > > >> > [Config]
>> > >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
>> > >> > > > > > >> >   ErrorCode => int16
>> > >> > > > > > >> >   ErrorDescription => string (empty if successful)
>> > >> > > > > > >> >
>> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
>> > >> > > ReplicaAssignment
>> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
>> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
>> ErrorDescription]
>> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
>> > >> > > > > > >> >   CommandErrorCode => int16
>> > >> > > > > > >> >   CommandErrorDescription => string (nonempty in case
>> of
>> > >> fatal
>> > >> > > > > error,
>> > >> > > > > > >> e.g.
>> > >> > > > > > >> > we couldn't get topics by regexp)
>> > >> > > > > > >> >
>> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
>> > >> > > > > > >> > DescribeTopicResponse -> [TopicName TopicDescription
>> > >> ErrorCode
>> > >> > > > > > >> > ErrorDescription] CommandErrorCode
>> CommandErrorDescription
>> > >> > > > > > >> >
>> > >> > > > > > >> > Also, any thoughts about our discussion regarding
>> re-routing
>> > >> > > > > facility?
>> > >> > > > > > >> In
>> > >> > > > > > >> > my
>> > >> > > > > > >> > understanding, it is like between augmenting
>> > >> > > TopicMetadataRequest
>> > >> > > > > > >> > (to include at least controllerId) and implementing
>> new
>> > >> > generic
>> > >> > > > > > >> re-routing
>> > >> > > > > > >> > facility so sending messages to controller will be
>> handled
>> > >> by
>> > >> > > it.
>> > >> > > > > > >> >
>> > >> > > > > > >> > Thanks,
>> > >> > > > > > >> > Andrii Biletskyi
>> > >> > > > > > >> >
>> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
>> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > > > >> >
>> > >> > > > > > >> > > @Guozhang:
>> > >> > > > > > >> > > Thanks for your comments, I've answered some of
>> those. The
>> > >> > > main
>> > >> > > > > > thing
>> > >> > > > > > >> is
>> > >> > > > > > >> > > having merged request for
>> create-alter-delete-describe - I
>> > >> > > have
>> > >> > > > > some
>> > >> > > > > > >> > > concerns about this approach.
>> > >> > > > > > >> > >
>> > >> > > > > > >> > > @*Jay*:
>> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is also
>> one of
>> > >> > the
>> > >> > > > > > >> concerns.
>> > >> > > > > > >> > We
>> > >> > > > > > >> > > can solve it if we implement re-routing facility.
>> But I
>> > >> > agree
>> > >> > > > with
>> > >> > > > > > >> > > Guozhang - it will make clients' internals a little
>> bit
>> > >> > easier
>> > >> > > > but
>> > >> > > > > > >> this
>> > >> > > > > > >> > > seems to be a complex logic to implement and
>> support then.
>> > >> > > > > > Especially
>> > >> > > > > > >> for
>> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing later
>> for
>> > >> these
>> > >> > > > > > >> requests).
>> > >> > > > > > >> > > Also people will tend to avoid this re-routing
>> facility
>> > >> and
>> > >> > > hold
>> > >> > > > > > local
>> > >> > > > > > >> > > cluster cache to ensure their high-priority requests
>> > >> (which
>> > >> > > some
>> > >> > > > > of
>> > >> > > > > > >> the
>> > >> > > > > > >> > > admin requests are) not sent to some busy broker
>> where
>> > >> they
>> > >> > > wait
>> > >> > > > > to
>> > >> > > > > > be
>> > >> > > > > > >> > > routed to the correct one.
>> > >> > > > > > >> > > As pointed out by Jun here (
>> > >> > > > > > >> > >
>> > >> > > > > > >> >
>> > >> > > > > > >>
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
>> > >> > > > > > >> > )
>> > >> > > > > > >> > > to solve the issue we might introduce a message
>> type to
>> > >> get
>> > >> > > > > cluster
>> > >> > > > > > >> > state.
>> > >> > > > > > >> > > But I agree we can just update
>> TopicMetadataResponse to
>> > >> > > include
>> > >> > > > > > >> > > controllerId (and probably smth else).
>> > >> > > > > > >> > > What are you thougths?
>> > >> > > > > > >> > >
>> > >> > > > > > >> > > Thanks,
>> > >> > > > > > >> > > Andrii
>> > >> > > > > > >> > >
>> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
>> > >> > > > > wangguoz@gmail.com>
>> > >> > > > > > >> > wrote:
>> > >> > > > > > >> > >
>> > >> > > > > > >> > >> I think for the topics commands we can actually
>> merge
>> > >> > > > > > >> > >> create/alter/delete/describe as one request type
>> since
>> > >> > their
>> > >> > > > > > formats
>> > >> > > > > > >> are
>> > >> > > > > > >> > >> very much similar, and keep list-topics and others
>> like
>> > >> > > > > > >> > >> partition-reassignment / preferred-leader-election
>> as
>> > >> > > separate
>> > >> > > > > > >> request
>> > >> > > > > > >> > >> types, I also left some other comments on the RB (
>> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
>> > >> > > > jay.kreps@gmail.com>
>> > >> > > > > > >> wrote:
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to just
>> have
>> > >> one
>> > >> > > "do
>> > >> > > > > > admin
>> > >> > > > > > >> > >> stuff"
>> > >> > > > > > >> > >> > command that has the union of all parameters.
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >> > What I am saying is that command line tools are
>> one
>> > >> > client
>> > >> > > of
>> > >> > > > > the
>> > >> > > > > > >> > >> > administrative apis, but these will be used in a
>> number
>> > >> > of
>> > >> > > > > > >> scenarios
>> > >> > > > > > >> > so
>> > >> > > > > > >> > >> > they should make logical sense even in the
>> absence of
>> > >> the
>> > >> > > > > command
>> > >> > > > > > >> line
>> > >> > > > > > >> > >> > tool. Hence comments like trying to clarify the
>> > >> > > relationship
>> > >> > > > > > >> between
>> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds
>> of
>> > >> things
>> > >> > > > > really
>> > >> > > > > > >> need
>> > >> > > > > > >> > >> to be
>> > >> > > > > > >> > >> > thought through.
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >> > Hope that makes sense.
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >> > -Jay
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii
>> Biletskyi <
>> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >> > > Jay,
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> > > Thanks for answering. You understood
>> correctly, most
>> > >> of
>> > >> > > my
>> > >> > > > > > >> comments
>> > >> > > > > > >> > >> were
>> > >> > > > > > >> > >> > > related to your point 1) - about "well
>> thought-out"
>> > >> > apis.
>> > >> > > > > Also,
>> > >> > > > > > >> yes,
>> > >> > > > > > >> > >> as I
>> > >> > > > > > >> > >> > > understood we would like to introduce a single
>> > >> unified
>> > >> > > CLI
>> > >> > > > > tool
>> > >> > > > > > >> with
>> > >> > > > > > >> > >> > > centralized server-side request handling for
>> lots of
>> > >> > > > existing
>> > >> > > > > > >> ones
>> > >> > > > > > >> > >> (incl.
>> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
>> > >> ReassignPartitions,
>> > >> > > smth
>> > >> > > > > > else
>> > >> > > > > > >> if
>> > >> > > > > > >> > >> added
>> > >> > > > > > >> > >> > > in future). In our previous discussion (
>> > >> > > > > > >> > >> > >
>> https://issues.apache.org/jira/browse/KAFKA-1694)
>> > >> > people
>> > >> > > > > said
>> > >> > > > > > >> > they'd
>> > >> > > > > > >> > >> > > rather
>> > >> > > > > > >> > >> > > have a separate message for each command, so,
>> yes,
>> > >> this
>> > >> > > > way I
>> > >> > > > > > >> came
>> > >> > > > > > >> > to
>> > >> > > > > > >> > >> 1-1
>> > >> > > > > > >> > >> > > mapping between commands in the tool and
>> protocol
>> > >> > > > additions.
>> > >> > > > > > But
>> > >> > > > > > >> I
>> > >> > > > > > >> > >> might
>> > >> > > > > > >> > >> > be
>> > >> > > > > > >> > >> > > wrong.
>> > >> > > > > > >> > >> > > At the end I just try to start discussion how
>> at
>> > >> least
>> > >> > > > > > generally
>> > >> > > > > > >> > this
>> > >> > > > > > >> > >> > > protocol should look like.
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> > > Thanks,
>> > >> > > > > > >> > >> > > Andrii
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
>> > >> > > > > > jay.kreps@gmail.com
>> > >> > > > > > >> >
>> > >> > > > > > >> > >> wrote:
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> > > > Hey Andrii,
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > To answer your earlier question we just
>> really
>> > >> can't
>> > >> > be
>> > >> > > > > > adding
>> > >> > > > > > >> any
>> > >> > > > > > >> > >> more
>> > >> > > > > > >> > >> > > > scala protocol objects. These things are
>> super hard
>> > >> > to
>> > >> > > > > > maintain
>> > >> > > > > > >> > >> because
>> > >> > > > > > >> > >> > > > they hand code the byte parsing and don't
>> have good
>> > >> > > > > > versioning
>> > >> > > > > > >> > >> support.
>> > >> > > > > > >> > >> > > > Since we are already planning on converting
>> we
>> > >> > > definitely
>> > >> > > > > > don't
>> > >> > > > > > >> > >> want to
>> > >> > > > > > >> > >> > > add
>> > >> > > > > > >> > >> > > > a ton more of these--they are total tech
>> debt.
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > What does it mean that the changes are
>> isolated
>> > >> from
>> > >> > > the
>> > >> > > > > > >> current
>> > >> > > > > > >> > >> code
>> > >> > > > > > >> > >> > > base?
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > I actually didn't understand the remaining
>> > >> comments,
>> > >> > > > which
>> > >> > > > > of
>> > >> > > > > > >> the
>> > >> > > > > > >> > >> > points
>> > >> > > > > > >> > >> > > > are you responding to?
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > Maybe one sticking point here is that it
>> seems like
>> > >> > you
>> > >> > > > > want
>> > >> > > > > > to
>> > >> > > > > > >> > make
>> > >> > > > > > >> > >> > some
>> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1 mapping
>> > >> between
>> > >> > > > > > commands
>> > >> > > > > > >> you
>> > >> > > > > > >> > >> > > imagine
>> > >> > > > > > >> > >> > > > in the tool and protocol additions. I want
>> to make
>> > >> > sure
>> > >> > > > we
>> > >> > > > > > >> don't
>> > >> > > > > > >> > do
>> > >> > > > > > >> > >> > that.
>> > >> > > > > > >> > >> > > > The protocol needs to be really really well
>> thought
>> > >> > out
>> > >> > > > > > against
>> > >> > > > > > >> > many
>> > >> > > > > > >> > >> > use
>> > >> > > > > > >> > >> > > > cases so it should make perfect logical
>> sense in
>> > >> the
>> > >> > > > > absence
>> > >> > > > > > of
>> > >> > > > > > >> > >> knowing
>> > >> > > > > > >> > >> > > the
>> > >> > > > > > >> > >> > > > command line tool, right?
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > -Jay
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii
>> Biletskyi
>> > >> <
>> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > > Hey Jay,
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > I would like to continue this discussion
>> as it
>> > >> seem
>> > >> > > > there
>> > >> > > > > > is
>> > >> > > > > > >> no
>> > >> > > > > > >> > >> > > progress
>> > >> > > > > > >> > >> > > > > here.
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > First of all, could you please explain
>> what did
>> > >> you
>> > >> > > > mean
>> > >> > > > > in
>> > >> > > > > > >> 2?
>> > >> > > > > > >> > How
>> > >> > > > > > >> > >> > > > exactly
>> > >> > > > > > >> > >> > > > > are we going to migrate to the new java
>> protocol
>> > >> > > > > > definitions.
>> > >> > > > > > >> > And
>> > >> > > > > > >> > >> why
>> > >> > > > > > >> > >> > > > it's
>> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > I agree with you, this feature includes
>> lots of
>> > >> > > stuff,
>> > >> > > > > but
>> > >> > > > > > >> > >> thankfully
>> > >> > > > > > >> > >> > > > > almost all changes are isolated from the
>> current
>> > >> > code
>> > >> > > > > base,
>> > >> > > > > > >> > >> > > > > so the main thing, I think, we need to
>> agree is
>> > >> > RQ/RP
>> > >> > > > > > format.
>> > >> > > > > > >> > >> > > > > So how can we start discussion about the
>> concrete
>> > >> > > > > messages
>> > >> > > > > > >> > format?
>> > >> > > > > > >> > >> > > > > Can we take (
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >>
>> > >> > > > > > >> >
>> > >> > > > > > >>
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
>> > >> > > > > > >> > >> > > > > )
>> > >> > > > > > >> > >> > > > > as starting point?
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > We had some doubts earlier whether it worth
>> > >> > > introducing
>> > >> > > > > one
>> > >> > > > > > >> > >> generic
>> > >> > > > > > >> > >> > > Admin
>> > >> > > > > > >> > >> > > > > Request for all commands (
>> > >> > > > > > >> > >> > > >
>> https://issues.apache.org/jira/browse/KAFKA-1694
>> > >> > > > > > >> > >> > > > > )
>> > >> > > > > > >> > >> > > > > but then everybody agreed it would be
>> better to
>> > >> > have
>> > >> > > > > > separate
>> > >> > > > > > >> > >> message
>> > >> > > > > > >> > >> > > for
>> > >> > > > > > >> > >> > > > > each admin command. The Request part is
>> really
>> > >> > > dictated
>> > >> > > > > > from
>> > >> > > > > > >> the
>> > >> > > > > > >> > >> > > command
>> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so
>> the
>> > >> > proposed
>> > >> > > > > > version
>> > >> > > > > > >> > >> should
>> > >> > > > > > >> > >> > be
>> > >> > > > > > >> > >> > > > > fine (let's put aside for now remarks about
>> > >> > Optional
>> > >> > > > > type,
>> > >> > > > > > >> > >> batching,
>> > >> > > > > > >> > >> > > > > configs normalization - I agree with all of
>> > >> them).
>> > >> > > > > > >> > >> > > > > So the second part is Response. I see
>> there are
>> > >> two
>> > >> > > > cases
>> > >> > > > > > >> here.
>> > >> > > > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ;
>> b)
>> > >> "Get"
>> > >> > > > > > requests -
>> > >> > > > > > >> > >> > > > > List/Describe...
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > a) should only hold request result
>> (regardless
>> > >> what
>> > >> > > we
>> > >> > > > > > decide
>> > >> > > > > > >> > >> about
>> > >> > > > > > >> > >> > > > > blocking/non-blocking commands execution).
>> > >> > > > > > >> > >> > > > > Usually we provide error code in response
>> but
>> > >> since
>> > >> > > we
>> > >> > > > > will
>> > >> > > > > > >> use
>> > >> > > > > > >> > >> this
>> > >> > > > > > >> > >> > in
>> > >> > > > > > >> > >> > > > > interactive shell we need some human
>> readable
>> > >> error
>> > >> > > > > > >> description
>> > >> > > > > > >> > -
>> > >> > > > > > >> > >> so
>> > >> > > > > > >> > >> > I
>> > >> > > > > > >> > >> > > > > added errorDesription field where you can
>> at
>> > >> least
>> > >> > > > leave
>> > >> > > > > > >> > >> > > > > exception.getMessage.
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > b) in addition to previous item message
>> should
>> > >> hold
>> > >> > > > > command
>> > >> > > > > > >> > >> specific
>> > >> > > > > > >> > >> > > > > response data. We can discuss in detail
>> each of
>> > >> > them
>> > >> > > > but
>> > >> > > > > > >> let's
>> > >> > > > > > >> > for
>> > >> > > > > > >> > >> > now
>> > >> > > > > > >> > >> > > > > agree about the overall pattern.
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > Thanks,
>> > >> > > > > > >> > >> > > > > Andrii Biletskyi
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps
>> <
>> > >> > > > > > >> jay.kreps@gmail.com
>> > >> > > > > > >> > >
>> > >> > > > > > >> > >> > > wrote:
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > > > > Hey Joe,
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > This is great. A few comments on KIP-4
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 1. This is much needed functionality,
>> but there
>> > >> > > are a
>> > >> > > > > lot
>> > >> > > > > > >> of
>> > >> > > > > > >> > >> the so
>> > >> > > > > > >> > >> > > > let's
>> > >> > > > > > >> > >> > > > > > really think these protocols through. We
>> really
>> > >> > > want
>> > >> > > > to
>> > >> > > > > > >> end up
>> > >> > > > > > >> > >> > with a
>> > >> > > > > > >> > >> > > > set
>> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis.
>> For this
>> > >> > > > reason I
>> > >> > > > > > >> think
>> > >> > > > > > >> > >> it is
>> > >> > > > > > >> > >> > > > > really
>> > >> > > > > > >> > >> > > > > > important to think through the end state
>> even
>> > >> if
>> > >> > > that
>> > >> > > > > > >> includes
>> > >> > > > > > >> > >> APIs
>> > >> > > > > > >> > >> > > we
>> > >> > > > > > >> > >> > > > > > won't implement in the first phase.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait until
>> we
>> > >> have
>> > >> > > > > switched
>> > >> > > > > > >> the
>> > >> > > > > > >> > >> > server
>> > >> > > > > > >> > >> > > > over
>> > >> > > > > > >> > >> > > > > > to the new java protocol definitions. If
>> we add
>> > >> > > > upteen
>> > >> > > > > > >> more ad
>> > >> > > > > > >> > >> hoc
>> > >> > > > > > >> > >> > > > scala
>> > >> > > > > > >> > >> > > > > > objects that is just generating more
>> work for
>> > >> the
>> > >> > > > > > >> conversion
>> > >> > > > > > >> > we
>> > >> > > > > > >> > >> > know
>> > >> > > > > > >> > >> > > we
>> > >> > > > > > >> > >> > > > > > have to do.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new type of
>> > >> > optional
>> > >> > > > > > >> parameter.
>> > >> > > > > > >> > >> This
>> > >> > > > > > >> > >> > is
>> > >> > > > > > >> > >> > > > > > inconsistent with everything else in the
>> > >> protocol
>> > >> > > > where
>> > >> > > > > > we
>> > >> > > > > > >> use
>> > >> > > > > > >> > >> -1
>> > >> > > > > > >> > >> > or
>> > >> > > > > > >> > >> > > > some
>> > >> > > > > > >> > >> > > > > > other marker value. You could argue
>> either way
>> > >> > but
>> > >> > > > > let's
>> > >> > > > > > >> stick
>> > >> > > > > > >> > >> with
>> > >> > > > > > >> > >> > > > that
>> > >> > > > > > >> > >> > > > > > for consistency. For clients that
>> implemented
>> > >> the
>> > >> > > > > > protocol
>> > >> > > > > > >> in
>> > >> > > > > > >> > a
>> > >> > > > > > >> > >> > > better
>> > >> > > > > > >> > >> > > > > way
>> > >> > > > > > >> > >> > > > > > than our scala code these basic
>> primitives are
>> > >> > hard
>> > >> > > > to
>> > >> > > > > > >> change.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
>> duplicate
>> > >> > > > > > >> > TopicMetadataRequest
>> > >> > > > > > >> > >> > > which
>> > >> > > > > > >> > >> > > > > has
>> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I think
>> we
>> > >> > should
>> > >> > > > > rename
>> > >> > > > > > >> that
>> > >> > > > > > >> > >> > > request
>> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
>> > >> MetadataRequest)
>> > >> > > and
>> > >> > > > > > >> include
>> > >> > > > > > >> > >> the id
>> > >> > > > > > >> > >> > > of
>> > >> > > > > > >> > >> > > > > the
>> > >> > > > > > >> > >> > > > > > controller. Or are there other things we
>> could
>> > >> > add
>> > >> > > > > here?
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to make a
>> lot of
>> > >> > > > requests
>> > >> > > > > > that
>> > >> > > > > > >> > can
>> > >> > > > > > >> > >> > only
>> > >> > > > > > >> > >> > > go
>> > >> > > > > > >> > >> > > > > to
>> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of
>> burden for
>> > >> > > > client
>> > >> > > > > > >> > >> > > implementations
>> > >> > > > > > >> > >> > > > > (it
>> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can fail
>> in many
>> > >> > > parts
>> > >> > > > > so
>> > >> > > > > > it
>> > >> > > > > > >> > >> ends up
>> > >> > > > > > >> > >> > > > > being a
>> > >> > > > > > >> > >> > > > > > full state machine to do right). I think
>> we
>> > >> > should
>> > >> > > > > > consider
>> > >> > > > > > >> > >> making
>> > >> > > > > > >> > >> > > > admin
>> > >> > > > > > >> > >> > > > > > commands and ideally as many of the
>> other apis
>> > >> as
>> > >> > > > > > possible
>> > >> > > > > > >> > >> > available
>> > >> > > > > > >> > >> > > on
>> > >> > > > > > >> > >> > > > > all
>> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
>> controller on
>> > >> > the
>> > >> > > > > broker
>> > >> > > > > > >> > side.
>> > >> > > > > > >> > >> > > Perhaps
>> > >> > > > > > >> > >> > > > > > there would be a general way to
>> encapsulate
>> > >> this
>> > >> > > > > > re-routing
>> > >> > > > > > >> > >> > behavior.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 6. We should probably normalize the key
>> value
>> > >> > pairs
>> > >> > > > > used
>> > >> > > > > > >> for
>> > >> > > > > > >> > >> > configs
>> > >> > > > > > >> > >> > > > > rather
>> > >> > > > > > >> > >> > > > > > than embedding a new formatting. So two
>> strings
>> > >> > > > rather
>> > >> > > > > > than
>> > >> > > > > > >> > one
>> > >> > > > > > >> > >> > with
>> > >> > > > > > >> > >> > > an
>> > >> > > > > > >> > >> > > > > > internal equals sign.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs
>> that the
>> > >> > > > command
>> > >> > > > > > has
>> > >> > > > > > >> > >> begun or
>> > >> > > > > > >> > >> > > > that
>> > >> > > > > > >> > >> > > > > > the command has been completed? It is a
>> lot
>> > >> more
>> > >> > > > usable
>> > >> > > > > > if
>> > >> > > > > > >> the
>> > >> > > > > > >> > >> > > command
>> > >> > > > > > >> > >> > > > > has
>> > >> > > > > > >> > >> > > > > > been completed so you know that if you
>> create a
>> > >> > > topic
>> > >> > > > > and
>> > >> > > > > > >> then
>> > >> > > > > > >> > >> > > publish
>> > >> > > > > > >> > >> > > > to
>> > >> > > > > > >> > >> > > > > > it you won't get an exception about
>> there being
>> > >> > no
>> > >> > > > such
>> > >> > > > > > >> topic.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
>> duplicate a
>> > >> lot
>> > >> > > of
>> > >> > > > > > stuff
>> > >> > > > > > >> in
>> > >> > > > > > >> > >> the
>> > >> > > > > > >> > >> > > > > metadata
>> > >> > > > > > >> > >> > > > > > request. Is there a reason to give back
>> topics
>> > >> > > marked
>> > >> > > > > for
>> > >> > > > > > >> > >> > deletion? I
>> > >> > > > > > >> > >> > > > > feel
>> > >> > > > > > >> > >> > > > > > like if we just make the post-condition
>> of the
>> > >> > > delete
>> > >> > > > > > >> command
>> > >> > > > > > >> > be
>> > >> > > > > > >> > >> > that
>> > >> > > > > > >> > >> > > > the
>> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid of
>> the need
>> > >> > for
>> > >> > > > this
>> > >> > > > > > >> right?
>> > >> > > > > > >> > >> And
>> > >> > > > > > >> > >> > it
>> > >> > > > > > >> > >> > > > > will
>> > >> > > > > > >> > >> > > > > > be much more intuitive.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 9. Should we consider batching these
>> requests?
>> > >> We
>> > >> > > > have
>> > >> > > > > > >> > generally
>> > >> > > > > > >> > >> > > tried
>> > >> > > > > > >> > >> > > > to
>> > >> > > > > > >> > >> > > > > > allow multiple operations to be batched.
>> My
>> > >> > > suspicion
>> > >> > > > > is
>> > >> > > > > > >> that
>> > >> > > > > > >> > >> > without
>> > >> > > > > > >> > >> > > > > this
>> > >> > > > > > >> > >> > > > > > we will get a lot of code that does
>> something
>> > >> > like
>> > >> > > > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
>> > >> > > > > > >> > >> > > > > >       adminClient.describeTopic(topic)
>> > >> > > > > > >> > >> > > > > > this code will work great when you test
>> on 5
>> > >> > topics
>> > >> > > > but
>> > >> > > > > > >> not do
>> > >> > > > > > >> > >> as
>> > >> > > > > > >> > >> > > well
>> > >> > > > > > >> > >> > > > if
>> > >> > > > > > >> > >> > > > > > you have 50k.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss how
>> we want
>> > >> to
>> > >> > > > > expose
>> > >> > > > > > a
>> > >> > > > > > >> > >> > > programmatic
>> > >> > > > > > >> > >> > > > > JVM
>> > >> > > > > > >> > >> > > > > > client api for these operations.
>> Currently
>> > >> people
>> > >> > > > rely
>> > >> > > > > on
>> > >> > > > > > >> > >> > AdminUtils
>> > >> > > > > > >> > >> > > > > which
>> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we probably
>> need
>> > >> > > another
>> > >> > > > > > client
>> > >> > > > > > >> > >> under
>> > >> > > > > > >> > >> > > > > clients/
>> > >> > > > > > >> > >> > > > > > that exposes administrative
>> functionality. We
>> > >> > will
>> > >> > > > need
>> > >> > > > > > >> this
>> > >> > > > > > >> > >> just
>> > >> > > > > > >> > >> > to
>> > >> > > > > > >> > >> > > > > > properly test the new apis, I suspect. We
>> > >> should
>> > >> > > > figure
>> > >> > > > > > out
>> > >> > > > > > >> > that
>> > >> > > > > > >> > >> > API.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > 11. The other information that would be
>> really
>> > >> > > useful
>> > >> > > > > to
>> > >> > > > > > >> get
>> > >> > > > > > >> > >> would
>> > >> > > > > > >> > >> > be
>> > >> > > > > > >> > >> > > > > > information about partitions--how much
>> data is
>> > >> in
>> > >> > > the
>> > >> > > > > > >> > partition,
>> > >> > > > > > >> > >> > what
>> > >> > > > > > >> > >> > > > are
>> > >> > > > > > >> > >> > > > > > the segment offsets, what is the log-end
>> offset
>> > >> > > (i.e.
>> > >> > > > > > last
>> > >> > > > > > >> > >> offset),
>> > >> > > > > > >> > >> > > > what
>> > >> > > > > > >> > >> > > > > is
>> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think that
>> done
>> > >> > right
>> > >> > > > this
>> > >> > > > > > >> would
>> > >> > > > > > >> > be
>> > >> > > > > > >> > >> > the
>> > >> > > > > > >> > >> > > > > > successor to the very awkward
>> OffsetRequest we
>> > >> > have
>> > >> > > > > > today.
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > -Jay
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe
>> Stein <
>> > >> > > > > > >> > >> joe.stein@stealth.ly>
>> > >> > > > > > >> > >> > > > > wrote:
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
>> > >> > > > > > >> > >> > > > > > >
>> > >> > > > > > >> > >> > > > > > >
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >>
>> > >> > > > > > >> >
>> > >> > > > > > >>
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>> > >> > > > > > >> > >> > > > > > >
>> > >> > > > > > >> > >> > > > > > > JIRA
>> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
>> > >> > > > > > >> > >> > > > > > >
>> > >> > > > > > >> > >> > > > > > >
>> /*******************************************
>> > >> > > > > > >> > >> > > > > > >  Joe Stein
>> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
>> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
>> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
>> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
>> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > > > > > >
>> ********************************************/
>> > >> > > > > > >> > >> > > > > > >
>> > >> > > > > > >> > >> > > > > >
>> > >> > > > > > >> > >> > > > >
>> > >> > > > > > >> > >> > > >
>> > >> > > > > > >> > >> > >
>> > >> > > > > > >> > >> >
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >> --
>> > >> > > > > > >> > >> -- Guozhang
>> > >> > > > > > >> > >>
>> > >> > > > > > >> > >
>> > >> > > > > > >> > >
>> > >> > > > > > >> >
>> > >> > > > > > >>
>> > >> > > > > > >
>> > >> > > > > > >
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >> >
>> > >> >
>> > >> > --
>> > >> > Jeff Holoman
>> > >> > Systems Engineer
>> > >> >
>> > >>
>>
>>
>
>
> --
> -- Guozhang
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Guozhang Wang <wa...@gmail.com>.
Thanks for the updated wiki. A few comments below:

1. Error description in response: I think if some errorCode could indicate
several different error cases then we should really change it to multiple
codes. In general the errorCode itself would be precise and sufficient for
describing the server side errors.

2. Describe topic request: it would be great to go beyond just batching on
topic name regex for this request. For example, a very common use case of
the topic command is to list all topics whose config A's value is B. With
topic name regex then we have to first retrieve __all__ topics's
description info and then filter at the client end, which will be a huge
burden on ZK.

3. Config K-Vs in create topic: this is related to the previous point;
maybe we can add another metadata K-V or just a metadata string along side
with config K-V in create topic like we did for offset commit request. This
field can be quite useful in storing information like "owner" of the topic
who issue the create command, etc, which is quite important for a
multi-tenant setting. Then in the describe topic request we can also batch
on regex of the metadata field.

4. Today all the admin operations are async in the sense that command will
return once it is written in ZK, and that is why we need extra verification
like testUtil.waitForTopicCreated() / verify partition reassignment
request, etc. With admin requests we could add a flag to enable / disable
synchronous requests; when it is turned on, the response will not return
until the request has been completed. And for async requests we can add a
"token" field in the response, and then only need a general "admin
verification request" with the given token to check if the async request
has been completed.

5. +1 for extending Metadata request to include controller / coordinator
information, and then we can remove the ConsumerMetadata / ClusterMetadata
requests.

Guozhang

On Tue, Mar 3, 2015 at 10:23 AM, Joel Koshy <jj...@gmail.com> wrote:

> Thanks for sending that out Joe - I don't think I will be able to make
> it today, so if notes can be sent out afterward that would be great.
>
> On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> > Thanks for sending this out Joe. Looking forward to chatting with
> everyone :)
> >
> > On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly> wrote:
> > > Hey, I just sent out a google hangout invite to all pmc, committers and
> > > everyone I found working on a KIP. If I missed anyone in the invite
> please
> > > let me know and can update it, np.
> > >
> > > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can get
> INFRA
> > > help to make a google account so we can manage better?
> > >
> > > To discuss
> > >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > > in progress and related JIRA that are interdependent and common work.
> > >
> > > ~ Joe Stein
> > >
> > > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > >> Let's stay on Google hangouts that will also record and make the
> sessions
> > >> available on youtube.
> > >>
> > >> -Jay
> > >>
> > >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <jholoman@cloudera.com
> >
> > >> wrote:
> > >>
> > >> > Jay / Joe
> > >> >
> > >> > We're happy to send out a Webex for this purpose. We could record
> the
> > >> > sessions if there is interest and publish them out.
> > >> >
> > >> > Thanks
> > >> >
> > >> > Jeff
> > >> >
> > >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >> >
> > >> > > Let's try to get the technical hang-ups sorted out, though. I
> really
> > >> > think
> > >> > > there is some benefit to live discussion vs writing. I am hopeful
> that
> > >> if
> > >> > > we post instructions and give ourselves a few attempts we can get
> it
> > >> > > working.
> > >> > >
> > >> > > Tuesday at that time would work for me...any objections?
> > >> > >
> > >> > > -Jay
> > >> > >
> > >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <jo...@stealth.ly>
> > >> wrote:
> > >> > >
> > >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am
> PT
> > >> ????
> > >> > > >
> > >> > > > I don't mind google hangout but there is always some issue or
> > >> whatever
> > >> > so
> > >> > > > we know the apache irc channel works. We can start there and
> see how
> > >> it
> > >> > > > goes? We can pull transcripts too and associate to tickets if
> need be
> > >> > > makes
> > >> > > > it helpful for things.
> > >> > > >
> > >> > > > ~ Joestein
> > >> > > >
> > >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <
> jay.kreps@gmail.com>
> > >> > wrote:
> > >> > > >
> > >> > > > > We'd talked about doing a Google Hangout to chat about this.
> What
> > >> > about
> > >> > > > > generalizing that a little further...I actually think it
> would be
> > >> > good
> > >> > > > for
> > >> > > > > everyone spending a reasonable chunk of their week on Kafka
> stuff
> > >> to
> > >> > > > maybe
> > >> > > > > sync up once a week. I think we could use time to talk through
> > >> design
> > >> > > > > stuff, make sure we are on top of code reviews, talk through
> any
> > >> > tricky
> > >> > > > > issues, etc.
> > >> > > > >
> > >> > > > > We can make it publicly available so that any one can follow
> along
> > >> > who
> > >> > > > > likes.
> > >> > > > >
> > >> > > > > Any interest in doing this? If so I'll try to set it up
> starting
> > >> next
> > >> > > > week.
> > >> > > > >
> > >> > > > > -Jay
> > >> > > > >
> > >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> > >> > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > >
> > >> > > > > > Hi all,
> > >> > > > > >
> > >> > > > > > I've updated KIP page, fixed / aligned document structure.
> Also I
> > >> > > added
> > >> > > > > > some
> > >> > > > > > very initial proposal for AdminClient so we have something
> to
> > >> start
> > >> > > > from
> > >> > > > > > while
> > >> > > > > > discussing the KIP.
> > >> > > > > >
> > >> > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >> > > > > >
> > >> > > > > > Thanks,
> > >> > > > > > Andrii Biletskyi
> > >> > > > > >
> > >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> > >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > >
> > >> > > > > > > Jay,
> > >> > > > > > >
> > >> > > > > > > Re error messages: you are right, in most cases client
> will
> > >> have
> > >> > > > enough
> > >> > > > > > > context to show descriptive error message. My concern is
> that
> > >> we
> > >> > > will
> > >> > > > > > have
> > >> > > > > > > to
> > >> > > > > > > add lots of new error codes for each possible error. Of
> course,
> > >> > we
> > >> > > > > could
> > >> > > > > > > reuse
> > >> > > > > > > some of existing like UknownTopicOrPartitionCode, but we
> will
> > >> > also
> > >> > > > need
> > >> > > > > > to
> > >> > > > > > > add smth like: TopicAlreadyExistsCode, TopicConfigInvalid
> (both
> > >> > for
> > >> > > > > topic
> > >> > > > > > > name and config, and probably user would like to know what
> > >> > exactly
> > >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
> > >> InternalError
> > >> > > > (e.g.
> > >> > > > > > > zookeeper failure) etc.
> > >> > > > > > > And this is only for TopicCommand, we will also need to
> add
> > >> > similar
> > >> > > > > stuff
> > >> > > > > > > for
> > >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up
> with a
> > >> > large
> > >> > > > list
> > >> > > > > > of
> > >> > > > > > > error codes, used only in Admin protocol.
> > >> > > > > > > Having said that, I agree my proposal is not consistent
> with
> > >> > other
> > >> > > > > cases.
> > >> > > > > > > Maybe we can find better solution or something in-between.
> > >> > > > > > >
> > >> > > > > > > Re Hangout chat: I think it is a great idea. This way we
> can
> > >> move
> > >> > > on
> > >> > > > > > > faster.
> > >> > > > > > > Let's agree somehow on date/time so people can join. Will
> work
> > >> > for
> > >> > > me
> > >> > > > > > this
> > >> > > > > > > and
> > >> > > > > > > next week almost anytime if agreed in advance.
> > >> > > > > > >
> > >> > > > > > > Thanks,
> > >> > > > > > > Andrii
> > >> > > > > > >
> > >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> > >> jay.kreps@gmail.com>
> > >> > > > > wrote:
> > >> > > > > > >
> > >> > > > > > >> Hey Andrii,
> > >> > > > > > >>
> > >> > > > > > >> Generally we can do good error handling without needing
> custom
> > >> > > > > > server-side
> > >> > > > > > >> messages. I.e. generally the client has the context to
> know
> > >> that
> > >> > > if
> > >> > > > it
> > >> > > > > > got
> > >> > > > > > >> an error that the topic doesn't exist to say "Topic X
> doesn't
> > >> > > exist"
> > >> > > > > > >> rather
> > >> > > > > > >> than "error code 14" (or whatever). Maybe there are
> specific
> > >> > cases
> > >> > > > > where
> > >> > > > > > >> this is hard? If we want to add server-side error
> messages we
> > >> > > really
> > >> > > > > do
> > >> > > > > > >> need to do this in a consistent way across the protocol.
> > >> > > > > > >>
> > >> > > > > > >> I still have a bunch of open questions here from my
> previous
> > >> > > list. I
> > >> > > > > > will
> > >> > > > > > >> be out for the next few days for Strata though. Maybe we
> could
> > >> > do
> > >> > > a
> > >> > > > > > Google
> > >> > > > > > >> Hangout chat on any open issues some time towards the
> end of
> > >> > next
> > >> > > > week
> > >> > > > > > for
> > >> > > > > > >> anyone interested in this ticket? I have a feeling that
> might
> > >> > > > progress
> > >> > > > > > >> things a little faster than email--I think we could talk
> > >> through
> > >> > > > those
> > >> > > > > > >> issues I brought up fairly quickly...
> > >> > > > > > >>
> > >> > > > > > >> -Jay
> > >> > > > > > >>
> > >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> > >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > >>
> > >> > > > > > >> > Hi all,
> > >> > > > > > >> >
> > >> > > > > > >> > I'm trying to address some of the issues which were
> > >> mentioned
> > >> > > > > earlier
> > >> > > > > > >> about
> > >> > > > > > >> > Admin RQ/RP format. One of those was about batching
> > >> > operations.
> > >> > > > What
> > >> > > > > > if
> > >> > > > > > >> we
> > >> > > > > > >> > follow TopicCommand approach and let people specify
> > >> topic-name
> > >> > > by
> > >> > > > > > >> regexp -
> > >> > > > > > >> > would that cover most of the use cases?
> > >> > > > > > >> >
> > >> > > > > > >> > Secondly, is what information should we generally
> provide in
> > >> > > Admin
> > >> > > > > > >> > responses.
> > >> > > > > > >> > I realize that Admin commands don't imply they will be
> used
> > >> > only
> > >> > > > in
> > >> > > > > > CLI
> > >> > > > > > >> > but,
> > >> > > > > > >> > it seems to me, CLI is a very important client of this
> > >> > feature.
> > >> > > In
> > >> > > > > > this
> > >> > > > > > >> > case,
> > >> > > > > > >> > seems logical, we would like to provide users with rich
> > >> > > experience
> > >> > > > > in
> > >> > > > > > >> terms
> > >> > > > > > >> > of
> > >> > > > > > >> > getting results / errors of the executed commands.
> Usually
> > >> we
> > >> > > > supply
> > >> > > > > > >> with
> > >> > > > > > >> > responses only errorCode, which looks very limiting,
> in case
> > >> > of
> > >> > > > CLI
> > >> > > > > we
> > >> > > > > > >> may
> > >> > > > > > >> > want to print human readable error description.
> > >> > > > > > >> >
> > >> > > > > > >> > So, taking into account previous item about batching,
> what
> > >> do
> > >> > > you
> > >> > > > > > think
> > >> > > > > > >> > about
> > >> > > > > > >> > having smth like:
> > >> > > > > > >> >
> > >> > > > > > >> > ('create' doesn't support regexp)
> > >> > > > > > >> > CreateTopicRequest => TopicName Partitions Replicas
> > >> > > > > ReplicaAssignment
> > >> > > > > > >> > [Config]
> > >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
> > >> > > > > > >> >   ErrorCode => int16
> > >> > > > > > >> >   ErrorDescription => string (empty if successful)
> > >> > > > > > >> >
> > >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> > >> > > ReplicaAssignment
> > >> > > > > > >> > [AddedConfig] [DeletedConfig]
> > >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
> ErrorDescription]
> > >> > > > > > >> > CommandErrorCode CommandErrorDescription
> > >> > > > > > >> >   CommandErrorCode => int16
> > >> > > > > > >> >   CommandErrorDescription => string (nonempty in case
> of
> > >> fatal
> > >> > > > > error,
> > >> > > > > > >> e.g.
> > >> > > > > > >> > we couldn't get topics by regexp)
> > >> > > > > > >> >
> > >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> > >> > > > > > >> > DescribeTopicResponse -> [TopicName TopicDescription
> > >> ErrorCode
> > >> > > > > > >> > ErrorDescription] CommandErrorCode
> CommandErrorDescription
> > >> > > > > > >> >
> > >> > > > > > >> > Also, any thoughts about our discussion regarding
> re-routing
> > >> > > > > facility?
> > >> > > > > > >> In
> > >> > > > > > >> > my
> > >> > > > > > >> > understanding, it is like between augmenting
> > >> > > TopicMetadataRequest
> > >> > > > > > >> > (to include at least controllerId) and implementing new
> > >> > generic
> > >> > > > > > >> re-routing
> > >> > > > > > >> > facility so sending messages to controller will be
> handled
> > >> by
> > >> > > it.
> > >> > > > > > >> >
> > >> > > > > > >> > Thanks,
> > >> > > > > > >> > Andrii Biletskyi
> > >> > > > > > >> >
> > >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> > >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > >> >
> > >> > > > > > >> > > @Guozhang:
> > >> > > > > > >> > > Thanks for your comments, I've answered some of
> those. The
> > >> > > main
> > >> > > > > > thing
> > >> > > > > > >> is
> > >> > > > > > >> > > having merged request for
> create-alter-delete-describe - I
> > >> > > have
> > >> > > > > some
> > >> > > > > > >> > > concerns about this approach.
> > >> > > > > > >> > >
> > >> > > > > > >> > > @*Jay*:
> > >> > > > > > >> > > I see that introduced ClusterMetadaRequest is also
> one of
> > >> > the
> > >> > > > > > >> concerns.
> > >> > > > > > >> > We
> > >> > > > > > >> > > can solve it if we implement re-routing facility.
> But I
> > >> > agree
> > >> > > > with
> > >> > > > > > >> > > Guozhang - it will make clients' internals a little
> bit
> > >> > easier
> > >> > > > but
> > >> > > > > > >> this
> > >> > > > > > >> > > seems to be a complex logic to implement and support
> then.
> > >> > > > > > Especially
> > >> > > > > > >> for
> > >> > > > > > >> > > Fetch and Produce (even if we add re-routing later
> for
> > >> these
> > >> > > > > > >> requests).
> > >> > > > > > >> > > Also people will tend to avoid this re-routing
> facility
> > >> and
> > >> > > hold
> > >> > > > > > local
> > >> > > > > > >> > > cluster cache to ensure their high-priority requests
> > >> (which
> > >> > > some
> > >> > > > > of
> > >> > > > > > >> the
> > >> > > > > > >> > > admin requests are) not sent to some busy broker
> where
> > >> they
> > >> > > wait
> > >> > > > > to
> > >> > > > > > be
> > >> > > > > > >> > > routed to the correct one.
> > >> > > > > > >> > > As pointed out by Jun here (
> > >> > > > > > >> > >
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > >> > > > > > >> > )
> > >> > > > > > >> > > to solve the issue we might introduce a message type
> to
> > >> get
> > >> > > > > cluster
> > >> > > > > > >> > state.
> > >> > > > > > >> > > But I agree we can just update TopicMetadataResponse
> to
> > >> > > include
> > >> > > > > > >> > > controllerId (and probably smth else).
> > >> > > > > > >> > > What are you thougths?
> > >> > > > > > >> > >
> > >> > > > > > >> > > Thanks,
> > >> > > > > > >> > > Andrii
> > >> > > > > > >> > >
> > >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> > >> > > > > wangguoz@gmail.com>
> > >> > > > > > >> > wrote:
> > >> > > > > > >> > >
> > >> > > > > > >> > >> I think for the topics commands we can actually
> merge
> > >> > > > > > >> > >> create/alter/delete/describe as one request type
> since
> > >> > their
> > >> > > > > > formats
> > >> > > > > > >> are
> > >> > > > > > >> > >> very much similar, and keep list-topics and others
> like
> > >> > > > > > >> > >> partition-reassignment / preferred-leader-election
> as
> > >> > > separate
> > >> > > > > > >> request
> > >> > > > > > >> > >> types, I also left some other comments on the RB (
> > >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> > >> > > > > > >> > >>
> > >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> > >> > > > jay.kreps@gmail.com>
> > >> > > > > > >> wrote:
> > >> > > > > > >> > >>
> > >> > > > > > >> > >> > Yeah I totally agree that we don't want to just
> have
> > >> one
> > >> > > "do
> > >> > > > > > admin
> > >> > > > > > >> > >> stuff"
> > >> > > > > > >> > >> > command that has the union of all parameters.
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > What I am saying is that command line tools are
> one
> > >> > client
> > >> > > of
> > >> > > > > the
> > >> > > > > > >> > >> > administrative apis, but these will be used in a
> number
> > >> > of
> > >> > > > > > >> scenarios
> > >> > > > > > >> > so
> > >> > > > > > >> > >> > they should make logical sense even in the
> absence of
> > >> the
> > >> > > > > command
> > >> > > > > > >> line
> > >> > > > > > >> > >> > tool. Hence comments like trying to clarify the
> > >> > > relationship
> > >> > > > > > >> between
> > >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds of
> > >> things
> > >> > > > > really
> > >> > > > > > >> need
> > >> > > > > > >> > >> to be
> > >> > > > > > >> > >> > thought through.
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > Hope that makes sense.
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > -Jay
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi
> <
> > >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >> > > Jay,
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > Thanks for answering. You understood correctly,
> most
> > >> of
> > >> > > my
> > >> > > > > > >> comments
> > >> > > > > > >> > >> were
> > >> > > > > > >> > >> > > related to your point 1) - about "well
> thought-out"
> > >> > apis.
> > >> > > > > Also,
> > >> > > > > > >> yes,
> > >> > > > > > >> > >> as I
> > >> > > > > > >> > >> > > understood we would like to introduce a single
> > >> unified
> > >> > > CLI
> > >> > > > > tool
> > >> > > > > > >> with
> > >> > > > > > >> > >> > > centralized server-side request handling for
> lots of
> > >> > > > existing
> > >> > > > > > >> ones
> > >> > > > > > >> > >> (incl.
> > >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> > >> ReassignPartitions,
> > >> > > smth
> > >> > > > > > else
> > >> > > > > > >> if
> > >> > > > > > >> > >> added
> > >> > > > > > >> > >> > > in future). In our previous discussion (
> > >> > > > > > >> > >> > >
> https://issues.apache.org/jira/browse/KAFKA-1694)
> > >> > people
> > >> > > > > said
> > >> > > > > > >> > they'd
> > >> > > > > > >> > >> > > rather
> > >> > > > > > >> > >> > > have a separate message for each command, so,
> yes,
> > >> this
> > >> > > > way I
> > >> > > > > > >> came
> > >> > > > > > >> > to
> > >> > > > > > >> > >> 1-1
> > >> > > > > > >> > >> > > mapping between commands in the tool and
> protocol
> > >> > > > additions.
> > >> > > > > > But
> > >> > > > > > >> I
> > >> > > > > > >> > >> might
> > >> > > > > > >> > >> > be
> > >> > > > > > >> > >> > > wrong.
> > >> > > > > > >> > >> > > At the end I just try to start discussion how at
> > >> least
> > >> > > > > > generally
> > >> > > > > > >> > this
> > >> > > > > > >> > >> > > protocol should look like.
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > Thanks,
> > >> > > > > > >> > >> > > Andrii
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
> > >> > > > > > jay.kreps@gmail.com
> > >> > > > > > >> >
> > >> > > > > > >> > >> wrote:
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> > > > Hey Andrii,
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > To answer your earlier question we just really
> > >> can't
> > >> > be
> > >> > > > > > adding
> > >> > > > > > >> any
> > >> > > > > > >> > >> more
> > >> > > > > > >> > >> > > > scala protocol objects. These things are
> super hard
> > >> > to
> > >> > > > > > maintain
> > >> > > > > > >> > >> because
> > >> > > > > > >> > >> > > > they hand code the byte parsing and don't
> have good
> > >> > > > > > versioning
> > >> > > > > > >> > >> support.
> > >> > > > > > >> > >> > > > Since we are already planning on converting we
> > >> > > definitely
> > >> > > > > > don't
> > >> > > > > > >> > >> want to
> > >> > > > > > >> > >> > > add
> > >> > > > > > >> > >> > > > a ton more of these--they are total tech debt.
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > What does it mean that the changes are
> isolated
> > >> from
> > >> > > the
> > >> > > > > > >> current
> > >> > > > > > >> > >> code
> > >> > > > > > >> > >> > > base?
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > I actually didn't understand the remaining
> > >> comments,
> > >> > > > which
> > >> > > > > of
> > >> > > > > > >> the
> > >> > > > > > >> > >> > points
> > >> > > > > > >> > >> > > > are you responding to?
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > Maybe one sticking point here is that it
> seems like
> > >> > you
> > >> > > > > want
> > >> > > > > > to
> > >> > > > > > >> > make
> > >> > > > > > >> > >> > some
> > >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1 mapping
> > >> between
> > >> > > > > > commands
> > >> > > > > > >> you
> > >> > > > > > >> > >> > > imagine
> > >> > > > > > >> > >> > > > in the tool and protocol additions. I want to
> make
> > >> > sure
> > >> > > > we
> > >> > > > > > >> don't
> > >> > > > > > >> > do
> > >> > > > > > >> > >> > that.
> > >> > > > > > >> > >> > > > The protocol needs to be really really well
> thought
> > >> > out
> > >> > > > > > against
> > >> > > > > > >> > many
> > >> > > > > > >> > >> > use
> > >> > > > > > >> > >> > > > cases so it should make perfect logical sense
> in
> > >> the
> > >> > > > > absence
> > >> > > > > > of
> > >> > > > > > >> > >> knowing
> > >> > > > > > >> > >> > > the
> > >> > > > > > >> > >> > > > command line tool, right?
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > -Jay
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii
> Biletskyi
> > >> <
> > >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > > Hey Jay,
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > > > I would like to continue this discussion as
> it
> > >> seem
> > >> > > > there
> > >> > > > > > is
> > >> > > > > > >> no
> > >> > > > > > >> > >> > > progress
> > >> > > > > > >> > >> > > > > here.
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > > > First of all, could you please explain what
> did
> > >> you
> > >> > > > mean
> > >> > > > > in
> > >> > > > > > >> 2?
> > >> > > > > > >> > How
> > >> > > > > > >> > >> > > > exactly
> > >> > > > > > >> > >> > > > > are we going to migrate to the new java
> protocol
> > >> > > > > > definitions.
> > >> > > > > > >> > And
> > >> > > > > > >> > >> why
> > >> > > > > > >> > >> > > > it's
> > >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > > > I agree with you, this feature includes
> lots of
> > >> > > stuff,
> > >> > > > > but
> > >> > > > > > >> > >> thankfully
> > >> > > > > > >> > >> > > > > almost all changes are isolated from the
> current
> > >> > code
> > >> > > > > base,
> > >> > > > > > >> > >> > > > > so the main thing, I think, we need to
> agree is
> > >> > RQ/RP
> > >> > > > > > format.
> > >> > > > > > >> > >> > > > > So how can we start discussion about the
> concrete
> > >> > > > > messages
> > >> > > > > > >> > format?
> > >> > > > > > >> > >> > > > > Can we take (
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >>
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > >> > > > > > >> > >> > > > > )
> > >> > > > > > >> > >> > > > > as starting point?
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > > > We had some doubts earlier whether it worth
> > >> > > introducing
> > >> > > > > one
> > >> > > > > > >> > >> generic
> > >> > > > > > >> > >> > > Admin
> > >> > > > > > >> > >> > > > > Request for all commands (
> > >> > > > > > >> > >> > > >
> https://issues.apache.org/jira/browse/KAFKA-1694
> > >> > > > > > >> > >> > > > > )
> > >> > > > > > >> > >> > > > > but then everybody agreed it would be
> better to
> > >> > have
> > >> > > > > > separate
> > >> > > > > > >> > >> message
> > >> > > > > > >> > >> > > for
> > >> > > > > > >> > >> > > > > each admin command. The Request part is
> really
> > >> > > dictated
> > >> > > > > > from
> > >> > > > > > >> the
> > >> > > > > > >> > >> > > command
> > >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so the
> > >> > proposed
> > >> > > > > > version
> > >> > > > > > >> > >> should
> > >> > > > > > >> > >> > be
> > >> > > > > > >> > >> > > > > fine (let's put aside for now remarks about
> > >> > Optional
> > >> > > > > type,
> > >> > > > > > >> > >> batching,
> > >> > > > > > >> > >> > > > > configs normalization - I agree with all of
> > >> them).
> > >> > > > > > >> > >> > > > > So the second part is Response. I see there
> are
> > >> two
> > >> > > > cases
> > >> > > > > > >> here.
> > >> > > > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b)
> > >> "Get"
> > >> > > > > > requests -
> > >> > > > > > >> > >> > > > > List/Describe...
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > > > a) should only hold request result
> (regardless
> > >> what
> > >> > > we
> > >> > > > > > decide
> > >> > > > > > >> > >> about
> > >> > > > > > >> > >> > > > > blocking/non-blocking commands execution).
> > >> > > > > > >> > >> > > > > Usually we provide error code in response
> but
> > >> since
> > >> > > we
> > >> > > > > will
> > >> > > > > > >> use
> > >> > > > > > >> > >> this
> > >> > > > > > >> > >> > in
> > >> > > > > > >> > >> > > > > interactive shell we need some human
> readable
> > >> error
> > >> > > > > > >> description
> > >> > > > > > >> > -
> > >> > > > > > >> > >> so
> > >> > > > > > >> > >> > I
> > >> > > > > > >> > >> > > > > added errorDesription field where you can at
> > >> least
> > >> > > > leave
> > >> > > > > > >> > >> > > > > exception.getMessage.
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > > > b) in addition to previous item message
> should
> > >> hold
> > >> > > > > command
> > >> > > > > > >> > >> specific
> > >> > > > > > >> > >> > > > > response data. We can discuss in detail
> each of
> > >> > them
> > >> > > > but
> > >> > > > > > >> let's
> > >> > > > > > >> > for
> > >> > > > > > >> > >> > now
> > >> > > > > > >> > >> > > > > agree about the overall pattern.
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > > > Thanks,
> > >> > > > > > >> > >> > > > > Andrii Biletskyi
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
> > >> > > > > > >> jay.kreps@gmail.com
> > >> > > > > > >> > >
> > >> > > > > > >> > >> > > wrote:
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > > > > Hey Joe,
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > This is great. A few comments on KIP-4
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > 1. This is much needed functionality, but
> there
> > >> > > are a
> > >> > > > > lot
> > >> > > > > > >> of
> > >> > > > > > >> > >> the so
> > >> > > > > > >> > >> > > > let's
> > >> > > > > > >> > >> > > > > > really think these protocols through. We
> really
> > >> > > want
> > >> > > > to
> > >> > > > > > >> end up
> > >> > > > > > >> > >> > with a
> > >> > > > > > >> > >> > > > set
> > >> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis. For
> this
> > >> > > > reason I
> > >> > > > > > >> think
> > >> > > > > > >> > >> it is
> > >> > > > > > >> > >> > > > > really
> > >> > > > > > >> > >> > > > > > important to think through the end state
> even
> > >> if
> > >> > > that
> > >> > > > > > >> includes
> > >> > > > > > >> > >> APIs
> > >> > > > > > >> > >> > > we
> > >> > > > > > >> > >> > > > > > won't implement in the first phase.
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > 2. Let's please please please wait until
> we
> > >> have
> > >> > > > > switched
> > >> > > > > > >> the
> > >> > > > > > >> > >> > server
> > >> > > > > > >> > >> > > > over
> > >> > > > > > >> > >> > > > > > to the new java protocol definitions. If
> we add
> > >> > > > upteen
> > >> > > > > > >> more ad
> > >> > > > > > >> > >> hoc
> > >> > > > > > >> > >> > > > scala
> > >> > > > > > >> > >> > > > > > objects that is just generating more work
> for
> > >> the
> > >> > > > > > >> conversion
> > >> > > > > > >> > we
> > >> > > > > > >> > >> > know
> > >> > > > > > >> > >> > > we
> > >> > > > > > >> > >> > > > > > have to do.
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > 3. This proposal introduces a new type of
> > >> > optional
> > >> > > > > > >> parameter.
> > >> > > > > > >> > >> This
> > >> > > > > > >> > >> > is
> > >> > > > > > >> > >> > > > > > inconsistent with everything else in the
> > >> protocol
> > >> > > > where
> > >> > > > > > we
> > >> > > > > > >> use
> > >> > > > > > >> > >> -1
> > >> > > > > > >> > >> > or
> > >> > > > > > >> > >> > > > some
> > >> > > > > > >> > >> > > > > > other marker value. You could argue
> either way
> > >> > but
> > >> > > > > let's
> > >> > > > > > >> stick
> > >> > > > > > >> > >> with
> > >> > > > > > >> > >> > > > that
> > >> > > > > > >> > >> > > > > > for consistency. For clients that
> implemented
> > >> the
> > >> > > > > > protocol
> > >> > > > > > >> in
> > >> > > > > > >> > a
> > >> > > > > > >> > >> > > better
> > >> > > > > > >> > >> > > > > way
> > >> > > > > > >> > >> > > > > > than our scala code these basic
> primitives are
> > >> > hard
> > >> > > > to
> > >> > > > > > >> change.
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to
> duplicate
> > >> > > > > > >> > TopicMetadataRequest
> > >> > > > > > >> > >> > > which
> > >> > > > > > >> > >> > > > > has
> > >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I think
> we
> > >> > should
> > >> > > > > rename
> > >> > > > > > >> that
> > >> > > > > > >> > >> > > request
> > >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> > >> MetadataRequest)
> > >> > > and
> > >> > > > > > >> include
> > >> > > > > > >> > >> the id
> > >> > > > > > >> > >> > > of
> > >> > > > > > >> > >> > > > > the
> > >> > > > > > >> > >> > > > > > controller. Or are there other things we
> could
> > >> > add
> > >> > > > > here?
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > 5. We have a tendency to try to make a
> lot of
> > >> > > > requests
> > >> > > > > > that
> > >> > > > > > >> > can
> > >> > > > > > >> > >> > only
> > >> > > > > > >> > >> > > go
> > >> > > > > > >> > >> > > > > to
> > >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of
> burden for
> > >> > > > client
> > >> > > > > > >> > >> > > implementations
> > >> > > > > > >> > >> > > > > (it
> > >> > > > > > >> > >> > > > > > sounds easy but each discovery can fail
> in many
> > >> > > parts
> > >> > > > > so
> > >> > > > > > it
> > >> > > > > > >> > >> ends up
> > >> > > > > > >> > >> > > > > being a
> > >> > > > > > >> > >> > > > > > full state machine to do right). I think
> we
> > >> > should
> > >> > > > > > consider
> > >> > > > > > >> > >> making
> > >> > > > > > >> > >> > > > admin
> > >> > > > > > >> > >> > > > > > commands and ideally as many of the other
> apis
> > >> as
> > >> > > > > > possible
> > >> > > > > > >> > >> > available
> > >> > > > > > >> > >> > > on
> > >> > > > > > >> > >> > > > > all
> > >> > > > > > >> > >> > > > > > brokers and just redirect to the
> controller on
> > >> > the
> > >> > > > > broker
> > >> > > > > > >> > side.
> > >> > > > > > >> > >> > > Perhaps
> > >> > > > > > >> > >> > > > > > there would be a general way to
> encapsulate
> > >> this
> > >> > > > > > re-routing
> > >> > > > > > >> > >> > behavior.
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > 6. We should probably normalize the key
> value
> > >> > pairs
> > >> > > > > used
> > >> > > > > > >> for
> > >> > > > > > >> > >> > configs
> > >> > > > > > >> > >> > > > > rather
> > >> > > > > > >> > >> > > > > > than embedding a new formatting. So two
> strings
> > >> > > > rather
> > >> > > > > > than
> > >> > > > > > >> > one
> > >> > > > > > >> > >> > with
> > >> > > > > > >> > >> > > an
> > >> > > > > > >> > >> > > > > > internal equals sign.
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs
> that the
> > >> > > > command
> > >> > > > > > has
> > >> > > > > > >> > >> begun or
> > >> > > > > > >> > >> > > > that
> > >> > > > > > >> > >> > > > > > the command has been completed? It is a
> lot
> > >> more
> > >> > > > usable
> > >> > > > > > if
> > >> > > > > > >> the
> > >> > > > > > >> > >> > > command
> > >> > > > > > >> > >> > > > > has
> > >> > > > > > >> > >> > > > > > been completed so you know that if you
> create a
> > >> > > topic
> > >> > > > > and
> > >> > > > > > >> then
> > >> > > > > > >> > >> > > publish
> > >> > > > > > >> > >> > > > to
> > >> > > > > > >> > >> > > > > > it you won't get an exception about there
> being
> > >> > no
> > >> > > > such
> > >> > > > > > >> topic.
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > 8. Describe topic and list topics
> duplicate a
> > >> lot
> > >> > > of
> > >> > > > > > stuff
> > >> > > > > > >> in
> > >> > > > > > >> > >> the
> > >> > > > > > >> > >> > > > > metadata
> > >> > > > > > >> > >> > > > > > request. Is there a reason to give back
> topics
> > >> > > marked
> > >> > > > > for
> > >> > > > > > >> > >> > deletion? I
> > >> > > > > > >> > >> > > > > feel
> > >> > > > > > >> > >> > > > > > like if we just make the post-condition
> of the
> > >> > > delete
> > >> > > > > > >> command
> > >> > > > > > >> > be
> > >> > > > > > >> > >> > that
> > >> > > > > > >> > >> > > > the
> > >> > > > > > >> > >> > > > > > topic is deleted that will get rid of the
> need
> > >> > for
> > >> > > > this
> > >> > > > > > >> right?
> > >> > > > > > >> > >> And
> > >> > > > > > >> > >> > it
> > >> > > > > > >> > >> > > > > will
> > >> > > > > > >> > >> > > > > > be much more intuitive.
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > 9. Should we consider batching these
> requests?
> > >> We
> > >> > > > have
> > >> > > > > > >> > generally
> > >> > > > > > >> > >> > > tried
> > >> > > > > > >> > >> > > > to
> > >> > > > > > >> > >> > > > > > allow multiple operations to be batched.
> My
> > >> > > suspicion
> > >> > > > > is
> > >> > > > > > >> that
> > >> > > > > > >> > >> > without
> > >> > > > > > >> > >> > > > > this
> > >> > > > > > >> > >> > > > > > we will get a lot of code that does
> something
> > >> > like
> > >> > > > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
> > >> > > > > > >> > >> > > > > >       adminClient.describeTopic(topic)
> > >> > > > > > >> > >> > > > > > this code will work great when you test
> on 5
> > >> > topics
> > >> > > > but
> > >> > > > > > >> not do
> > >> > > > > > >> > >> as
> > >> > > > > > >> > >> > > well
> > >> > > > > > >> > >> > > > if
> > >> > > > > > >> > >> > > > > > you have 50k.
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > 10. I think we should also discuss how we
> want
> > >> to
> > >> > > > > expose
> > >> > > > > > a
> > >> > > > > > >> > >> > > programmatic
> > >> > > > > > >> > >> > > > > JVM
> > >> > > > > > >> > >> > > > > > client api for these operations. Currently
> > >> people
> > >> > > > rely
> > >> > > > > on
> > >> > > > > > >> > >> > AdminUtils
> > >> > > > > > >> > >> > > > > which
> > >> > > > > > >> > >> > > > > > is totally sketchy. I think we probably
> need
> > >> > > another
> > >> > > > > > client
> > >> > > > > > >> > >> under
> > >> > > > > > >> > >> > > > > clients/
> > >> > > > > > >> > >> > > > > > that exposes administrative
> functionality. We
> > >> > will
> > >> > > > need
> > >> > > > > > >> this
> > >> > > > > > >> > >> just
> > >> > > > > > >> > >> > to
> > >> > > > > > >> > >> > > > > > properly test the new apis, I suspect. We
> > >> should
> > >> > > > figure
> > >> > > > > > out
> > >> > > > > > >> > that
> > >> > > > > > >> > >> > API.
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > 11. The other information that would be
> really
> > >> > > useful
> > >> > > > > to
> > >> > > > > > >> get
> > >> > > > > > >> > >> would
> > >> > > > > > >> > >> > be
> > >> > > > > > >> > >> > > > > > information about partitions--how much
> data is
> > >> in
> > >> > > the
> > >> > > > > > >> > partition,
> > >> > > > > > >> > >> > what
> > >> > > > > > >> > >> > > > are
> > >> > > > > > >> > >> > > > > > the segment offsets, what is the log-end
> offset
> > >> > > (i.e.
> > >> > > > > > last
> > >> > > > > > >> > >> offset),
> > >> > > > > > >> > >> > > > what
> > >> > > > > > >> > >> > > > > is
> > >> > > > > > >> > >> > > > > > the compaction point, etc. I think that
> done
> > >> > right
> > >> > > > this
> > >> > > > > > >> would
> > >> > > > > > >> > be
> > >> > > > > > >> > >> > the
> > >> > > > > > >> > >> > > > > > successor to the very awkward
> OffsetRequest we
> > >> > have
> > >> > > > > > today.
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > -Jay
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe
> Stein <
> > >> > > > > > >> > >> joe.stein@stealth.ly>
> > >> > > > > > >> > >> > > > > wrote:
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > > > > Hi, created a KIP
> > >> > > > > > >> > >> > > > > > >
> > >> > > > > > >> > >> > > > > > >
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >>
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >> > > > > > >> > >> > > > > > >
> > >> > > > > > >> > >> > > > > > > JIRA
> > >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > >> > > > > > >> > >> > > > > > >
> > >> > > > > > >> > >> > > > > > >
> /*******************************************
> > >> > > > > > >> > >> > > > > > >  Joe Stein
> > >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> > >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
> > >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> > >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> > >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > > > > > >
> ********************************************/
> > >> > > > > > >> > >> > > > > > >
> > >> > > > > > >> > >> > > > > >
> > >> > > > > > >> > >> > > > >
> > >> > > > > > >> > >> > > >
> > >> > > > > > >> > >> > >
> > >> > > > > > >> > >> >
> > >> > > > > > >> > >>
> > >> > > > > > >> > >>
> > >> > > > > > >> > >>
> > >> > > > > > >> > >> --
> > >> > > > > > >> > >> -- Guozhang
> > >> > > > > > >> > >>
> > >> > > > > > >> > >
> > >> > > > > > >> > >
> > >> > > > > > >> >
> > >> > > > > > >>
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >> >
> > >> >
> > >> > --
> > >> > Jeff Holoman
> > >> > Systems Engineer
> > >> >
> > >>
>
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Joel Koshy <jj...@gmail.com>.
Thanks for sending that out Joe - I don't think I will be able to make
it today, so if notes can be sent out afterward that would be great.

On Mon, Mar 02, 2015 at 09:16:13AM -0800, Gwen Shapira wrote:
> Thanks for sending this out Joe. Looking forward to chatting with everyone :)
> 
> On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly> wrote:
> > Hey, I just sent out a google hangout invite to all pmc, committers and
> > everyone I found working on a KIP. If I missed anyone in the invite please
> > let me know and can update it, np.
> >
> > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can get INFRA
> > help to make a google account so we can manage better?
> >
> > To discuss
> > https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > in progress and related JIRA that are interdependent and common work.
> >
> > ~ Joe Stein
> >
> > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> >> Let's stay on Google hangouts that will also record and make the sessions
> >> available on youtube.
> >>
> >> -Jay
> >>
> >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <jh...@cloudera.com>
> >> wrote:
> >>
> >> > Jay / Joe
> >> >
> >> > We're happy to send out a Webex for this purpose. We could record the
> >> > sessions if there is interest and publish them out.
> >> >
> >> > Thanks
> >> >
> >> > Jeff
> >> >
> >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <ja...@gmail.com> wrote:
> >> >
> >> > > Let's try to get the technical hang-ups sorted out, though. I really
> >> > think
> >> > > there is some benefit to live discussion vs writing. I am hopeful that
> >> if
> >> > > we post instructions and give ourselves a few attempts we can get it
> >> > > working.
> >> > >
> >> > > Tuesday at that time would work for me...any objections?
> >> > >
> >> > > -Jay
> >> > >
> >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <jo...@stealth.ly>
> >> wrote:
> >> > >
> >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am PT
> >> ????
> >> > > >
> >> > > > I don't mind google hangout but there is always some issue or
> >> whatever
> >> > so
> >> > > > we know the apache irc channel works. We can start there and see how
> >> it
> >> > > > goes? We can pull transcripts too and associate to tickets if need be
> >> > > makes
> >> > > > it helpful for things.
> >> > > >
> >> > > > ~ Joestein
> >> > > >
> >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <ja...@gmail.com>
> >> > wrote:
> >> > > >
> >> > > > > We'd talked about doing a Google Hangout to chat about this. What
> >> > about
> >> > > > > generalizing that a little further...I actually think it would be
> >> > good
> >> > > > for
> >> > > > > everyone spending a reasonable chunk of their week on Kafka stuff
> >> to
> >> > > > maybe
> >> > > > > sync up once a week. I think we could use time to talk through
> >> design
> >> > > > > stuff, make sure we are on top of code reviews, talk through any
> >> > tricky
> >> > > > > issues, etc.
> >> > > > >
> >> > > > > We can make it publicly available so that any one can follow along
> >> > who
> >> > > > > likes.
> >> > > > >
> >> > > > > Any interest in doing this? If so I'll try to set it up starting
> >> next
> >> > > > week.
> >> > > > >
> >> > > > > -Jay
> >> > > > >
> >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> >> > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > >
> >> > > > > > Hi all,
> >> > > > > >
> >> > > > > > I've updated KIP page, fixed / aligned document structure. Also I
> >> > > added
> >> > > > > > some
> >> > > > > > very initial proposal for AdminClient so we have something to
> >> start
> >> > > > from
> >> > > > > > while
> >> > > > > > discussing the KIP.
> >> > > > > >
> >> > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > > Andrii Biletskyi
> >> > > > > >
> >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > >
> >> > > > > > > Jay,
> >> > > > > > >
> >> > > > > > > Re error messages: you are right, in most cases client will
> >> have
> >> > > > enough
> >> > > > > > > context to show descriptive error message. My concern is that
> >> we
> >> > > will
> >> > > > > > have
> >> > > > > > > to
> >> > > > > > > add lots of new error codes for each possible error. Of course,
> >> > we
> >> > > > > could
> >> > > > > > > reuse
> >> > > > > > > some of existing like UknownTopicOrPartitionCode, but we will
> >> > also
> >> > > > need
> >> > > > > > to
> >> > > > > > > add smth like: TopicAlreadyExistsCode, TopicConfigInvalid (both
> >> > for
> >> > > > > topic
> >> > > > > > > name and config, and probably user would like to know what
> >> > exactly
> >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
> >> InternalError
> >> > > > (e.g.
> >> > > > > > > zookeeper failure) etc.
> >> > > > > > > And this is only for TopicCommand, we will also need to add
> >> > similar
> >> > > > > stuff
> >> > > > > > > for
> >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up with a
> >> > large
> >> > > > list
> >> > > > > > of
> >> > > > > > > error codes, used only in Admin protocol.
> >> > > > > > > Having said that, I agree my proposal is not consistent with
> >> > other
> >> > > > > cases.
> >> > > > > > > Maybe we can find better solution or something in-between.
> >> > > > > > >
> >> > > > > > > Re Hangout chat: I think it is a great idea. This way we can
> >> move
> >> > > on
> >> > > > > > > faster.
> >> > > > > > > Let's agree somehow on date/time so people can join. Will work
> >> > for
> >> > > me
> >> > > > > > this
> >> > > > > > > and
> >> > > > > > > next week almost anytime if agreed in advance.
> >> > > > > > >
> >> > > > > > > Thanks,
> >> > > > > > > Andrii
> >> > > > > > >
> >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> >> jay.kreps@gmail.com>
> >> > > > > wrote:
> >> > > > > > >
> >> > > > > > >> Hey Andrii,
> >> > > > > > >>
> >> > > > > > >> Generally we can do good error handling without needing custom
> >> > > > > > server-side
> >> > > > > > >> messages. I.e. generally the client has the context to know
> >> that
> >> > > if
> >> > > > it
> >> > > > > > got
> >> > > > > > >> an error that the topic doesn't exist to say "Topic X doesn't
> >> > > exist"
> >> > > > > > >> rather
> >> > > > > > >> than "error code 14" (or whatever). Maybe there are specific
> >> > cases
> >> > > > > where
> >> > > > > > >> this is hard? If we want to add server-side error messages we
> >> > > really
> >> > > > > do
> >> > > > > > >> need to do this in a consistent way across the protocol.
> >> > > > > > >>
> >> > > > > > >> I still have a bunch of open questions here from my previous
> >> > > list. I
> >> > > > > > will
> >> > > > > > >> be out for the next few days for Strata though. Maybe we could
> >> > do
> >> > > a
> >> > > > > > Google
> >> > > > > > >> Hangout chat on any open issues some time towards the end of
> >> > next
> >> > > > week
> >> > > > > > for
> >> > > > > > >> anyone interested in this ticket? I have a feeling that might
> >> > > > progress
> >> > > > > > >> things a little faster than email--I think we could talk
> >> through
> >> > > > those
> >> > > > > > >> issues I brought up fairly quickly...
> >> > > > > > >>
> >> > > > > > >> -Jay
> >> > > > > > >>
> >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > >>
> >> > > > > > >> > Hi all,
> >> > > > > > >> >
> >> > > > > > >> > I'm trying to address some of the issues which were
> >> mentioned
> >> > > > > earlier
> >> > > > > > >> about
> >> > > > > > >> > Admin RQ/RP format. One of those was about batching
> >> > operations.
> >> > > > What
> >> > > > > > if
> >> > > > > > >> we
> >> > > > > > >> > follow TopicCommand approach and let people specify
> >> topic-name
> >> > > by
> >> > > > > > >> regexp -
> >> > > > > > >> > would that cover most of the use cases?
> >> > > > > > >> >
> >> > > > > > >> > Secondly, is what information should we generally provide in
> >> > > Admin
> >> > > > > > >> > responses.
> >> > > > > > >> > I realize that Admin commands don't imply they will be used
> >> > only
> >> > > > in
> >> > > > > > CLI
> >> > > > > > >> > but,
> >> > > > > > >> > it seems to me, CLI is a very important client of this
> >> > feature.
> >> > > In
> >> > > > > > this
> >> > > > > > >> > case,
> >> > > > > > >> > seems logical, we would like to provide users with rich
> >> > > experience
> >> > > > > in
> >> > > > > > >> terms
> >> > > > > > >> > of
> >> > > > > > >> > getting results / errors of the executed commands. Usually
> >> we
> >> > > > supply
> >> > > > > > >> with
> >> > > > > > >> > responses only errorCode, which looks very limiting, in case
> >> > of
> >> > > > CLI
> >> > > > > we
> >> > > > > > >> may
> >> > > > > > >> > want to print human readable error description.
> >> > > > > > >> >
> >> > > > > > >> > So, taking into account previous item about batching, what
> >> do
> >> > > you
> >> > > > > > think
> >> > > > > > >> > about
> >> > > > > > >> > having smth like:
> >> > > > > > >> >
> >> > > > > > >> > ('create' doesn't support regexp)
> >> > > > > > >> > CreateTopicRequest => TopicName Partitions Replicas
> >> > > > > ReplicaAssignment
> >> > > > > > >> > [Config]
> >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
> >> > > > > > >> >   ErrorCode => int16
> >> > > > > > >> >   ErrorDescription => string (empty if successful)
> >> > > > > > >> >
> >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> >> > > ReplicaAssignment
> >> > > > > > >> > [AddedConfig] [DeletedConfig]
> >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
> >> > > > > > >> > CommandErrorCode CommandErrorDescription
> >> > > > > > >> >   CommandErrorCode => int16
> >> > > > > > >> >   CommandErrorDescription => string (nonempty in case of
> >> fatal
> >> > > > > error,
> >> > > > > > >> e.g.
> >> > > > > > >> > we couldn't get topics by regexp)
> >> > > > > > >> >
> >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> >> > > > > > >> > DescribeTopicResponse -> [TopicName TopicDescription
> >> ErrorCode
> >> > > > > > >> > ErrorDescription] CommandErrorCode CommandErrorDescription
> >> > > > > > >> >
> >> > > > > > >> > Also, any thoughts about our discussion regarding re-routing
> >> > > > > facility?
> >> > > > > > >> In
> >> > > > > > >> > my
> >> > > > > > >> > understanding, it is like between augmenting
> >> > > TopicMetadataRequest
> >> > > > > > >> > (to include at least controllerId) and implementing new
> >> > generic
> >> > > > > > >> re-routing
> >> > > > > > >> > facility so sending messages to controller will be handled
> >> by
> >> > > it.
> >> > > > > > >> >
> >> > > > > > >> > Thanks,
> >> > > > > > >> > Andrii Biletskyi
> >> > > > > > >> >
> >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > >> >
> >> > > > > > >> > > @Guozhang:
> >> > > > > > >> > > Thanks for your comments, I've answered some of those. The
> >> > > main
> >> > > > > > thing
> >> > > > > > >> is
> >> > > > > > >> > > having merged request for create-alter-delete-describe - I
> >> > > have
> >> > > > > some
> >> > > > > > >> > > concerns about this approach.
> >> > > > > > >> > >
> >> > > > > > >> > > @*Jay*:
> >> > > > > > >> > > I see that introduced ClusterMetadaRequest is also one of
> >> > the
> >> > > > > > >> concerns.
> >> > > > > > >> > We
> >> > > > > > >> > > can solve it if we implement re-routing facility. But I
> >> > agree
> >> > > > with
> >> > > > > > >> > > Guozhang - it will make clients' internals a little bit
> >> > easier
> >> > > > but
> >> > > > > > >> this
> >> > > > > > >> > > seems to be a complex logic to implement and support then.
> >> > > > > > Especially
> >> > > > > > >> for
> >> > > > > > >> > > Fetch and Produce (even if we add re-routing later for
> >> these
> >> > > > > > >> requests).
> >> > > > > > >> > > Also people will tend to avoid this re-routing facility
> >> and
> >> > > hold
> >> > > > > > local
> >> > > > > > >> > > cluster cache to ensure their high-priority requests
> >> (which
> >> > > some
> >> > > > > of
> >> > > > > > >> the
> >> > > > > > >> > > admin requests are) not sent to some busy broker where
> >> they
> >> > > wait
> >> > > > > to
> >> > > > > > be
> >> > > > > > >> > > routed to the correct one.
> >> > > > > > >> > > As pointed out by Jun here (
> >> > > > > > >> > >
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> >> > > > > > >> > )
> >> > > > > > >> > > to solve the issue we might introduce a message type to
> >> get
> >> > > > > cluster
> >> > > > > > >> > state.
> >> > > > > > >> > > But I agree we can just update TopicMetadataResponse to
> >> > > include
> >> > > > > > >> > > controllerId (and probably smth else).
> >> > > > > > >> > > What are you thougths?
> >> > > > > > >> > >
> >> > > > > > >> > > Thanks,
> >> > > > > > >> > > Andrii
> >> > > > > > >> > >
> >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> >> > > > > wangguoz@gmail.com>
> >> > > > > > >> > wrote:
> >> > > > > > >> > >
> >> > > > > > >> > >> I think for the topics commands we can actually merge
> >> > > > > > >> > >> create/alter/delete/describe as one request type since
> >> > their
> >> > > > > > formats
> >> > > > > > >> are
> >> > > > > > >> > >> very much similar, and keep list-topics and others like
> >> > > > > > >> > >> partition-reassignment / preferred-leader-election as
> >> > > separate
> >> > > > > > >> request
> >> > > > > > >> > >> types, I also left some other comments on the RB (
> >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> >> > > > > > >> > >>
> >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> >> > > > jay.kreps@gmail.com>
> >> > > > > > >> wrote:
> >> > > > > > >> > >>
> >> > > > > > >> > >> > Yeah I totally agree that we don't want to just have
> >> one
> >> > > "do
> >> > > > > > admin
> >> > > > > > >> > >> stuff"
> >> > > > > > >> > >> > command that has the union of all parameters.
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > What I am saying is that command line tools are one
> >> > client
> >> > > of
> >> > > > > the
> >> > > > > > >> > >> > administrative apis, but these will be used in a number
> >> > of
> >> > > > > > >> scenarios
> >> > > > > > >> > so
> >> > > > > > >> > >> > they should make logical sense even in the absence of
> >> the
> >> > > > > command
> >> > > > > > >> line
> >> > > > > > >> > >> > tool. Hence comments like trying to clarify the
> >> > > relationship
> >> > > > > > >> between
> >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds of
> >> things
> >> > > > > really
> >> > > > > > >> need
> >> > > > > > >> > >> to be
> >> > > > > > >> > >> > thought through.
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > Hope that makes sense.
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > -Jay
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > > Jay,
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > Thanks for answering. You understood correctly, most
> >> of
> >> > > my
> >> > > > > > >> comments
> >> > > > > > >> > >> were
> >> > > > > > >> > >> > > related to your point 1) - about "well thought-out"
> >> > apis.
> >> > > > > Also,
> >> > > > > > >> yes,
> >> > > > > > >> > >> as I
> >> > > > > > >> > >> > > understood we would like to introduce a single
> >> unified
> >> > > CLI
> >> > > > > tool
> >> > > > > > >> with
> >> > > > > > >> > >> > > centralized server-side request handling for lots of
> >> > > > existing
> >> > > > > > >> ones
> >> > > > > > >> > >> (incl.
> >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> >> ReassignPartitions,
> >> > > smth
> >> > > > > > else
> >> > > > > > >> if
> >> > > > > > >> > >> added
> >> > > > > > >> > >> > > in future). In our previous discussion (
> >> > > > > > >> > >> > > https://issues.apache.org/jira/browse/KAFKA-1694)
> >> > people
> >> > > > > said
> >> > > > > > >> > they'd
> >> > > > > > >> > >> > > rather
> >> > > > > > >> > >> > > have a separate message for each command, so, yes,
> >> this
> >> > > > way I
> >> > > > > > >> came
> >> > > > > > >> > to
> >> > > > > > >> > >> 1-1
> >> > > > > > >> > >> > > mapping between commands in the tool and protocol
> >> > > > additions.
> >> > > > > > But
> >> > > > > > >> I
> >> > > > > > >> > >> might
> >> > > > > > >> > >> > be
> >> > > > > > >> > >> > > wrong.
> >> > > > > > >> > >> > > At the end I just try to start discussion how at
> >> least
> >> > > > > > generally
> >> > > > > > >> > this
> >> > > > > > >> > >> > > protocol should look like.
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > Thanks,
> >> > > > > > >> > >> > > Andrii
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
> >> > > > > > jay.kreps@gmail.com
> >> > > > > > >> >
> >> > > > > > >> > >> wrote:
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > > Hey Andrii,
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > To answer your earlier question we just really
> >> can't
> >> > be
> >> > > > > > adding
> >> > > > > > >> any
> >> > > > > > >> > >> more
> >> > > > > > >> > >> > > > scala protocol objects. These things are super hard
> >> > to
> >> > > > > > maintain
> >> > > > > > >> > >> because
> >> > > > > > >> > >> > > > they hand code the byte parsing and don't have good
> >> > > > > > versioning
> >> > > > > > >> > >> support.
> >> > > > > > >> > >> > > > Since we are already planning on converting we
> >> > > definitely
> >> > > > > > don't
> >> > > > > > >> > >> want to
> >> > > > > > >> > >> > > add
> >> > > > > > >> > >> > > > a ton more of these--they are total tech debt.
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > What does it mean that the changes are isolated
> >> from
> >> > > the
> >> > > > > > >> current
> >> > > > > > >> > >> code
> >> > > > > > >> > >> > > base?
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > I actually didn't understand the remaining
> >> comments,
> >> > > > which
> >> > > > > of
> >> > > > > > >> the
> >> > > > > > >> > >> > points
> >> > > > > > >> > >> > > > are you responding to?
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > Maybe one sticking point here is that it seems like
> >> > you
> >> > > > > want
> >> > > > > > to
> >> > > > > > >> > make
> >> > > > > > >> > >> > some
> >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1 mapping
> >> between
> >> > > > > > commands
> >> > > > > > >> you
> >> > > > > > >> > >> > > imagine
> >> > > > > > >> > >> > > > in the tool and protocol additions. I want to make
> >> > sure
> >> > > > we
> >> > > > > > >> don't
> >> > > > > > >> > do
> >> > > > > > >> > >> > that.
> >> > > > > > >> > >> > > > The protocol needs to be really really well thought
> >> > out
> >> > > > > > against
> >> > > > > > >> > many
> >> > > > > > >> > >> > use
> >> > > > > > >> > >> > > > cases so it should make perfect logical sense in
> >> the
> >> > > > > absence
> >> > > > > > of
> >> > > > > > >> > >> knowing
> >> > > > > > >> > >> > > the
> >> > > > > > >> > >> > > > command line tool, right?
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > -Jay
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi
> >> <
> >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > > Hey Jay,
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > I would like to continue this discussion as it
> >> seem
> >> > > > there
> >> > > > > > is
> >> > > > > > >> no
> >> > > > > > >> > >> > > progress
> >> > > > > > >> > >> > > > > here.
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > First of all, could you please explain what did
> >> you
> >> > > > mean
> >> > > > > in
> >> > > > > > >> 2?
> >> > > > > > >> > How
> >> > > > > > >> > >> > > > exactly
> >> > > > > > >> > >> > > > > are we going to migrate to the new java protocol
> >> > > > > > definitions.
> >> > > > > > >> > And
> >> > > > > > >> > >> why
> >> > > > > > >> > >> > > > it's
> >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > I agree with you, this feature includes lots of
> >> > > stuff,
> >> > > > > but
> >> > > > > > >> > >> thankfully
> >> > > > > > >> > >> > > > > almost all changes are isolated from the current
> >> > code
> >> > > > > base,
> >> > > > > > >> > >> > > > > so the main thing, I think, we need to agree is
> >> > RQ/RP
> >> > > > > > format.
> >> > > > > > >> > >> > > > > So how can we start discussion about the concrete
> >> > > > > messages
> >> > > > > > >> > format?
> >> > > > > > >> > >> > > > > Can we take (
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> >
> >> > > > > > >> > >>
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> >> > > > > > >> > >> > > > > )
> >> > > > > > >> > >> > > > > as starting point?
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > We had some doubts earlier whether it worth
> >> > > introducing
> >> > > > > one
> >> > > > > > >> > >> generic
> >> > > > > > >> > >> > > Admin
> >> > > > > > >> > >> > > > > Request for all commands (
> >> > > > > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> >> > > > > > >> > >> > > > > )
> >> > > > > > >> > >> > > > > but then everybody agreed it would be better to
> >> > have
> >> > > > > > separate
> >> > > > > > >> > >> message
> >> > > > > > >> > >> > > for
> >> > > > > > >> > >> > > > > each admin command. The Request part is really
> >> > > dictated
> >> > > > > > from
> >> > > > > > >> the
> >> > > > > > >> > >> > > command
> >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so the
> >> > proposed
> >> > > > > > version
> >> > > > > > >> > >> should
> >> > > > > > >> > >> > be
> >> > > > > > >> > >> > > > > fine (let's put aside for now remarks about
> >> > Optional
> >> > > > > type,
> >> > > > > > >> > >> batching,
> >> > > > > > >> > >> > > > > configs normalization - I agree with all of
> >> them).
> >> > > > > > >> > >> > > > > So the second part is Response. I see there are
> >> two
> >> > > > cases
> >> > > > > > >> here.
> >> > > > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b)
> >> "Get"
> >> > > > > > requests -
> >> > > > > > >> > >> > > > > List/Describe...
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > a) should only hold request result (regardless
> >> what
> >> > > we
> >> > > > > > decide
> >> > > > > > >> > >> about
> >> > > > > > >> > >> > > > > blocking/non-blocking commands execution).
> >> > > > > > >> > >> > > > > Usually we provide error code in response but
> >> since
> >> > > we
> >> > > > > will
> >> > > > > > >> use
> >> > > > > > >> > >> this
> >> > > > > > >> > >> > in
> >> > > > > > >> > >> > > > > interactive shell we need some human readable
> >> error
> >> > > > > > >> description
> >> > > > > > >> > -
> >> > > > > > >> > >> so
> >> > > > > > >> > >> > I
> >> > > > > > >> > >> > > > > added errorDesription field where you can at
> >> least
> >> > > > leave
> >> > > > > > >> > >> > > > > exception.getMessage.
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > b) in addition to previous item message should
> >> hold
> >> > > > > command
> >> > > > > > >> > >> specific
> >> > > > > > >> > >> > > > > response data. We can discuss in detail each of
> >> > them
> >> > > > but
> >> > > > > > >> let's
> >> > > > > > >> > for
> >> > > > > > >> > >> > now
> >> > > > > > >> > >> > > > > agree about the overall pattern.
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > Thanks,
> >> > > > > > >> > >> > > > > Andrii Biletskyi
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
> >> > > > > > >> jay.kreps@gmail.com
> >> > > > > > >> > >
> >> > > > > > >> > >> > > wrote:
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > > Hey Joe,
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > This is great. A few comments on KIP-4
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 1. This is much needed functionality, but there
> >> > > are a
> >> > > > > lot
> >> > > > > > >> of
> >> > > > > > >> > >> the so
> >> > > > > > >> > >> > > > let's
> >> > > > > > >> > >> > > > > > really think these protocols through. We really
> >> > > want
> >> > > > to
> >> > > > > > >> end up
> >> > > > > > >> > >> > with a
> >> > > > > > >> > >> > > > set
> >> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis. For this
> >> > > > reason I
> >> > > > > > >> think
> >> > > > > > >> > >> it is
> >> > > > > > >> > >> > > > > really
> >> > > > > > >> > >> > > > > > important to think through the end state even
> >> if
> >> > > that
> >> > > > > > >> includes
> >> > > > > > >> > >> APIs
> >> > > > > > >> > >> > > we
> >> > > > > > >> > >> > > > > > won't implement in the first phase.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 2. Let's please please please wait until we
> >> have
> >> > > > > switched
> >> > > > > > >> the
> >> > > > > > >> > >> > server
> >> > > > > > >> > >> > > > over
> >> > > > > > >> > >> > > > > > to the new java protocol definitions. If we add
> >> > > > upteen
> >> > > > > > >> more ad
> >> > > > > > >> > >> hoc
> >> > > > > > >> > >> > > > scala
> >> > > > > > >> > >> > > > > > objects that is just generating more work for
> >> the
> >> > > > > > >> conversion
> >> > > > > > >> > we
> >> > > > > > >> > >> > know
> >> > > > > > >> > >> > > we
> >> > > > > > >> > >> > > > > > have to do.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 3. This proposal introduces a new type of
> >> > optional
> >> > > > > > >> parameter.
> >> > > > > > >> > >> This
> >> > > > > > >> > >> > is
> >> > > > > > >> > >> > > > > > inconsistent with everything else in the
> >> protocol
> >> > > > where
> >> > > > > > we
> >> > > > > > >> use
> >> > > > > > >> > >> -1
> >> > > > > > >> > >> > or
> >> > > > > > >> > >> > > > some
> >> > > > > > >> > >> > > > > > other marker value. You could argue either way
> >> > but
> >> > > > > let's
> >> > > > > > >> stick
> >> > > > > > >> > >> with
> >> > > > > > >> > >> > > > that
> >> > > > > > >> > >> > > > > > for consistency. For clients that implemented
> >> the
> >> > > > > > protocol
> >> > > > > > >> in
> >> > > > > > >> > a
> >> > > > > > >> > >> > > better
> >> > > > > > >> > >> > > > > way
> >> > > > > > >> > >> > > > > > than our scala code these basic primitives are
> >> > hard
> >> > > > to
> >> > > > > > >> change.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to duplicate
> >> > > > > > >> > TopicMetadataRequest
> >> > > > > > >> > >> > > which
> >> > > > > > >> > >> > > > > has
> >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I think we
> >> > should
> >> > > > > rename
> >> > > > > > >> that
> >> > > > > > >> > >> > > request
> >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> >> MetadataRequest)
> >> > > and
> >> > > > > > >> include
> >> > > > > > >> > >> the id
> >> > > > > > >> > >> > > of
> >> > > > > > >> > >> > > > > the
> >> > > > > > >> > >> > > > > > controller. Or are there other things we could
> >> > add
> >> > > > > here?
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 5. We have a tendency to try to make a lot of
> >> > > > requests
> >> > > > > > that
> >> > > > > > >> > can
> >> > > > > > >> > >> > only
> >> > > > > > >> > >> > > go
> >> > > > > > >> > >> > > > > to
> >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of burden for
> >> > > > client
> >> > > > > > >> > >> > > implementations
> >> > > > > > >> > >> > > > > (it
> >> > > > > > >> > >> > > > > > sounds easy but each discovery can fail in many
> >> > > parts
> >> > > > > so
> >> > > > > > it
> >> > > > > > >> > >> ends up
> >> > > > > > >> > >> > > > > being a
> >> > > > > > >> > >> > > > > > full state machine to do right). I think we
> >> > should
> >> > > > > > consider
> >> > > > > > >> > >> making
> >> > > > > > >> > >> > > > admin
> >> > > > > > >> > >> > > > > > commands and ideally as many of the other apis
> >> as
> >> > > > > > possible
> >> > > > > > >> > >> > available
> >> > > > > > >> > >> > > on
> >> > > > > > >> > >> > > > > all
> >> > > > > > >> > >> > > > > > brokers and just redirect to the controller on
> >> > the
> >> > > > > broker
> >> > > > > > >> > side.
> >> > > > > > >> > >> > > Perhaps
> >> > > > > > >> > >> > > > > > there would be a general way to encapsulate
> >> this
> >> > > > > > re-routing
> >> > > > > > >> > >> > behavior.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 6. We should probably normalize the key value
> >> > pairs
> >> > > > > used
> >> > > > > > >> for
> >> > > > > > >> > >> > configs
> >> > > > > > >> > >> > > > > rather
> >> > > > > > >> > >> > > > > > than embedding a new formatting. So two strings
> >> > > > rather
> >> > > > > > than
> >> > > > > > >> > one
> >> > > > > > >> > >> > with
> >> > > > > > >> > >> > > an
> >> > > > > > >> > >> > > > > > internal equals sign.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs that the
> >> > > > command
> >> > > > > > has
> >> > > > > > >> > >> begun or
> >> > > > > > >> > >> > > > that
> >> > > > > > >> > >> > > > > > the command has been completed? It is a lot
> >> more
> >> > > > usable
> >> > > > > > if
> >> > > > > > >> the
> >> > > > > > >> > >> > > command
> >> > > > > > >> > >> > > > > has
> >> > > > > > >> > >> > > > > > been completed so you know that if you create a
> >> > > topic
> >> > > > > and
> >> > > > > > >> then
> >> > > > > > >> > >> > > publish
> >> > > > > > >> > >> > > > to
> >> > > > > > >> > >> > > > > > it you won't get an exception about there being
> >> > no
> >> > > > such
> >> > > > > > >> topic.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 8. Describe topic and list topics duplicate a
> >> lot
> >> > > of
> >> > > > > > stuff
> >> > > > > > >> in
> >> > > > > > >> > >> the
> >> > > > > > >> > >> > > > > metadata
> >> > > > > > >> > >> > > > > > request. Is there a reason to give back topics
> >> > > marked
> >> > > > > for
> >> > > > > > >> > >> > deletion? I
> >> > > > > > >> > >> > > > > feel
> >> > > > > > >> > >> > > > > > like if we just make the post-condition of the
> >> > > delete
> >> > > > > > >> command
> >> > > > > > >> > be
> >> > > > > > >> > >> > that
> >> > > > > > >> > >> > > > the
> >> > > > > > >> > >> > > > > > topic is deleted that will get rid of the need
> >> > for
> >> > > > this
> >> > > > > > >> right?
> >> > > > > > >> > >> And
> >> > > > > > >> > >> > it
> >> > > > > > >> > >> > > > > will
> >> > > > > > >> > >> > > > > > be much more intuitive.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 9. Should we consider batching these requests?
> >> We
> >> > > > have
> >> > > > > > >> > generally
> >> > > > > > >> > >> > > tried
> >> > > > > > >> > >> > > > to
> >> > > > > > >> > >> > > > > > allow multiple operations to be batched. My
> >> > > suspicion
> >> > > > > is
> >> > > > > > >> that
> >> > > > > > >> > >> > without
> >> > > > > > >> > >> > > > > this
> >> > > > > > >> > >> > > > > > we will get a lot of code that does something
> >> > like
> >> > > > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
> >> > > > > > >> > >> > > > > >       adminClient.describeTopic(topic)
> >> > > > > > >> > >> > > > > > this code will work great when you test on 5
> >> > topics
> >> > > > but
> >> > > > > > >> not do
> >> > > > > > >> > >> as
> >> > > > > > >> > >> > > well
> >> > > > > > >> > >> > > > if
> >> > > > > > >> > >> > > > > > you have 50k.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 10. I think we should also discuss how we want
> >> to
> >> > > > > expose
> >> > > > > > a
> >> > > > > > >> > >> > > programmatic
> >> > > > > > >> > >> > > > > JVM
> >> > > > > > >> > >> > > > > > client api for these operations. Currently
> >> people
> >> > > > rely
> >> > > > > on
> >> > > > > > >> > >> > AdminUtils
> >> > > > > > >> > >> > > > > which
> >> > > > > > >> > >> > > > > > is totally sketchy. I think we probably need
> >> > > another
> >> > > > > > client
> >> > > > > > >> > >> under
> >> > > > > > >> > >> > > > > clients/
> >> > > > > > >> > >> > > > > > that exposes administrative functionality. We
> >> > will
> >> > > > need
> >> > > > > > >> this
> >> > > > > > >> > >> just
> >> > > > > > >> > >> > to
> >> > > > > > >> > >> > > > > > properly test the new apis, I suspect. We
> >> should
> >> > > > figure
> >> > > > > > out
> >> > > > > > >> > that
> >> > > > > > >> > >> > API.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 11. The other information that would be really
> >> > > useful
> >> > > > > to
> >> > > > > > >> get
> >> > > > > > >> > >> would
> >> > > > > > >> > >> > be
> >> > > > > > >> > >> > > > > > information about partitions--how much data is
> >> in
> >> > > the
> >> > > > > > >> > partition,
> >> > > > > > >> > >> > what
> >> > > > > > >> > >> > > > are
> >> > > > > > >> > >> > > > > > the segment offsets, what is the log-end offset
> >> > > (i.e.
> >> > > > > > last
> >> > > > > > >> > >> offset),
> >> > > > > > >> > >> > > > what
> >> > > > > > >> > >> > > > > is
> >> > > > > > >> > >> > > > > > the compaction point, etc. I think that done
> >> > right
> >> > > > this
> >> > > > > > >> would
> >> > > > > > >> > be
> >> > > > > > >> > >> > the
> >> > > > > > >> > >> > > > > > successor to the very awkward OffsetRequest we
> >> > have
> >> > > > > > today.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > -Jay
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
> >> > > > > > >> > >> joe.stein@stealth.ly>
> >> > > > > > >> > >> > > > > wrote:
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > > Hi, created a KIP
> >> > > > > > >> > >> > > > > > >
> >> > > > > > >> > >> > > > > > >
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> >
> >> > > > > > >> > >>
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> > > > > > >> > >> > > > > > >
> >> > > > > > >> > >> > > > > > > JIRA
> >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> >> > > > > > >> > >> > > > > > >
> >> > > > > > >> > >> > > > > > > /*******************************************
> >> > > > > > >> > >> > > > > > >  Joe Stein
> >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
> >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > > > > ********************************************/
> >> > > > > > >> > >> > > > > > >
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> >
> >> > > > > > >> > >>
> >> > > > > > >> > >>
> >> > > > > > >> > >>
> >> > > > > > >> > >> --
> >> > > > > > >> > >> -- Guozhang
> >> > > > > > >> > >>
> >> > > > > > >> > >
> >> > > > > > >> > >
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >> >
> >> >
> >> > --
> >> > Jeff Holoman
> >> > Systems Engineer
> >> >
> >>


Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jun Rao <ju...@confluent.io>.
It would also be good to think through how we can use those new admin
requests in the producer/consumer client as well. Currently, both the
producer and the consumer use TopicMetadataRequest to obtain the metadata,
which will trigger a topic creation if auto topic creation is enabled. This
is a bit weird for the consumer since the reader shouldn't be creating new
topics. With the new admin requests, we can potentially decouple topic
creation from obtaining the metadata. The consumer can just be issuing the
metadata requests without triggering the topic creation. The producer can
fetch the metadata first and then issue a create topic request if the topic
doesn't exist. We will have to think through how this works with the auto
topic creation logic though.

Thanks,

Jun

On Mon, Mar 2, 2015 at 9:16 AM, Gwen Shapira <gs...@cloudera.com> wrote:

> Thanks for sending this out Joe. Looking forward to chatting with everyone
> :)
>
> On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly> wrote:
> > Hey, I just sent out a google hangout invite to all pmc, committers and
> > everyone I found working on a KIP. If I missed anyone in the invite
> please
> > let me know and can update it, np.
> >
> > We should do this every Tuesday @ 2pm Eastern Time. Maybe we can get
> INFRA
> > help to make a google account so we can manage better?
> >
> > To discuss
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> > in progress and related JIRA that are interdependent and common work.
> >
> > ~ Joe Stein
> >
> > On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> >> Let's stay on Google hangouts that will also record and make the
> sessions
> >> available on youtube.
> >>
> >> -Jay
> >>
> >> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <jh...@cloudera.com>
> >> wrote:
> >>
> >> > Jay / Joe
> >> >
> >> > We're happy to send out a Webex for this purpose. We could record the
> >> > sessions if there is interest and publish them out.
> >> >
> >> > Thanks
> >> >
> >> > Jeff
> >> >
> >> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> >> >
> >> > > Let's try to get the technical hang-ups sorted out, though. I really
> >> > think
> >> > > there is some benefit to live discussion vs writing. I am hopeful
> that
> >> if
> >> > > we post instructions and give ourselves a few attempts we can get it
> >> > > working.
> >> > >
> >> > > Tuesday at that time would work for me...any objections?
> >> > >
> >> > > -Jay
> >> > >
> >> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <jo...@stealth.ly>
> >> wrote:
> >> > >
> >> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am PT
> >> ????
> >> > > >
> >> > > > I don't mind google hangout but there is always some issue or
> >> whatever
> >> > so
> >> > > > we know the apache irc channel works. We can start there and see
> how
> >> it
> >> > > > goes? We can pull transcripts too and associate to tickets if
> need be
> >> > > makes
> >> > > > it helpful for things.
> >> > > >
> >> > > > ~ Joestein
> >> > > >
> >> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <ja...@gmail.com>
> >> > wrote:
> >> > > >
> >> > > > > We'd talked about doing a Google Hangout to chat about this.
> What
> >> > about
> >> > > > > generalizing that a little further...I actually think it would
> be
> >> > good
> >> > > > for
> >> > > > > everyone spending a reasonable chunk of their week on Kafka
> stuff
> >> to
> >> > > > maybe
> >> > > > > sync up once a week. I think we could use time to talk through
> >> design
> >> > > > > stuff, make sure we are on top of code reviews, talk through any
> >> > tricky
> >> > > > > issues, etc.
> >> > > > >
> >> > > > > We can make it publicly available so that any one can follow
> along
> >> > who
> >> > > > > likes.
> >> > > > >
> >> > > > > Any interest in doing this? If so I'll try to set it up starting
> >> next
> >> > > > week.
> >> > > > >
> >> > > > > -Jay
> >> > > > >
> >> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> >> > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > >
> >> > > > > > Hi all,
> >> > > > > >
> >> > > > > > I've updated KIP page, fixed / aligned document structure.
> Also I
> >> > > added
> >> > > > > > some
> >> > > > > > very initial proposal for AdminClient so we have something to
> >> start
> >> > > > from
> >> > > > > > while
> >> > > > > > discussing the KIP.
> >> > > > > >
> >> > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> > > > > >
> >> > > > > > Thanks,
> >> > > > > > Andrii Biletskyi
> >> > > > > >
> >> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> >> > > > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > >
> >> > > > > > > Jay,
> >> > > > > > >
> >> > > > > > > Re error messages: you are right, in most cases client will
> >> have
> >> > > > enough
> >> > > > > > > context to show descriptive error message. My concern is
> that
> >> we
> >> > > will
> >> > > > > > have
> >> > > > > > > to
> >> > > > > > > add lots of new error codes for each possible error. Of
> course,
> >> > we
> >> > > > > could
> >> > > > > > > reuse
> >> > > > > > > some of existing like UknownTopicOrPartitionCode, but we
> will
> >> > also
> >> > > > need
> >> > > > > > to
> >> > > > > > > add smth like: TopicAlreadyExistsCode, TopicConfigInvalid
> (both
> >> > for
> >> > > > > topic
> >> > > > > > > name and config, and probably user would like to know what
> >> > exactly
> >> > > > > > > is wrong in his config), InvalidReplicaAssignment,
> >> InternalError
> >> > > > (e.g.
> >> > > > > > > zookeeper failure) etc.
> >> > > > > > > And this is only for TopicCommand, we will also need to add
> >> > similar
> >> > > > > stuff
> >> > > > > > > for
> >> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up with a
> >> > large
> >> > > > list
> >> > > > > > of
> >> > > > > > > error codes, used only in Admin protocol.
> >> > > > > > > Having said that, I agree my proposal is not consistent with
> >> > other
> >> > > > > cases.
> >> > > > > > > Maybe we can find better solution or something in-between.
> >> > > > > > >
> >> > > > > > > Re Hangout chat: I think it is a great idea. This way we can
> >> move
> >> > > on
> >> > > > > > > faster.
> >> > > > > > > Let's agree somehow on date/time so people can join. Will
> work
> >> > for
> >> > > me
> >> > > > > > this
> >> > > > > > > and
> >> > > > > > > next week almost anytime if agreed in advance.
> >> > > > > > >
> >> > > > > > > Thanks,
> >> > > > > > > Andrii
> >> > > > > > >
> >> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> >> jay.kreps@gmail.com>
> >> > > > > wrote:
> >> > > > > > >
> >> > > > > > >> Hey Andrii,
> >> > > > > > >>
> >> > > > > > >> Generally we can do good error handling without needing
> custom
> >> > > > > > server-side
> >> > > > > > >> messages. I.e. generally the client has the context to know
> >> that
> >> > > if
> >> > > > it
> >> > > > > > got
> >> > > > > > >> an error that the topic doesn't exist to say "Topic X
> doesn't
> >> > > exist"
> >> > > > > > >> rather
> >> > > > > > >> than "error code 14" (or whatever). Maybe there are
> specific
> >> > cases
> >> > > > > where
> >> > > > > > >> this is hard? If we want to add server-side error messages
> we
> >> > > really
> >> > > > > do
> >> > > > > > >> need to do this in a consistent way across the protocol.
> >> > > > > > >>
> >> > > > > > >> I still have a bunch of open questions here from my
> previous
> >> > > list. I
> >> > > > > > will
> >> > > > > > >> be out for the next few days for Strata though. Maybe we
> could
> >> > do
> >> > > a
> >> > > > > > Google
> >> > > > > > >> Hangout chat on any open issues some time towards the end
> of
> >> > next
> >> > > > week
> >> > > > > > for
> >> > > > > > >> anyone interested in this ticket? I have a feeling that
> might
> >> > > > progress
> >> > > > > > >> things a little faster than email--I think we could talk
> >> through
> >> > > > those
> >> > > > > > >> issues I brought up fairly quickly...
> >> > > > > > >>
> >> > > > > > >> -Jay
> >> > > > > > >>
> >> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> >> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > >>
> >> > > > > > >> > Hi all,
> >> > > > > > >> >
> >> > > > > > >> > I'm trying to address some of the issues which were
> >> mentioned
> >> > > > > earlier
> >> > > > > > >> about
> >> > > > > > >> > Admin RQ/RP format. One of those was about batching
> >> > operations.
> >> > > > What
> >> > > > > > if
> >> > > > > > >> we
> >> > > > > > >> > follow TopicCommand approach and let people specify
> >> topic-name
> >> > > by
> >> > > > > > >> regexp -
> >> > > > > > >> > would that cover most of the use cases?
> >> > > > > > >> >
> >> > > > > > >> > Secondly, is what information should we generally
> provide in
> >> > > Admin
> >> > > > > > >> > responses.
> >> > > > > > >> > I realize that Admin commands don't imply they will be
> used
> >> > only
> >> > > > in
> >> > > > > > CLI
> >> > > > > > >> > but,
> >> > > > > > >> > it seems to me, CLI is a very important client of this
> >> > feature.
> >> > > In
> >> > > > > > this
> >> > > > > > >> > case,
> >> > > > > > >> > seems logical, we would like to provide users with rich
> >> > > experience
> >> > > > > in
> >> > > > > > >> terms
> >> > > > > > >> > of
> >> > > > > > >> > getting results / errors of the executed commands.
> Usually
> >> we
> >> > > > supply
> >> > > > > > >> with
> >> > > > > > >> > responses only errorCode, which looks very limiting, in
> case
> >> > of
> >> > > > CLI
> >> > > > > we
> >> > > > > > >> may
> >> > > > > > >> > want to print human readable error description.
> >> > > > > > >> >
> >> > > > > > >> > So, taking into account previous item about batching,
> what
> >> do
> >> > > you
> >> > > > > > think
> >> > > > > > >> > about
> >> > > > > > >> > having smth like:
> >> > > > > > >> >
> >> > > > > > >> > ('create' doesn't support regexp)
> >> > > > > > >> > CreateTopicRequest => TopicName Partitions Replicas
> >> > > > > ReplicaAssignment
> >> > > > > > >> > [Config]
> >> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
> >> > > > > > >> >   ErrorCode => int16
> >> > > > > > >> >   ErrorDescription => string (empty if successful)
> >> > > > > > >> >
> >> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> >> > > ReplicaAssignment
> >> > > > > > >> > [AddedConfig] [DeletedConfig]
> >> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode
> ErrorDescription]
> >> > > > > > >> > CommandErrorCode CommandErrorDescription
> >> > > > > > >> >   CommandErrorCode => int16
> >> > > > > > >> >   CommandErrorDescription => string (nonempty in case of
> >> fatal
> >> > > > > error,
> >> > > > > > >> e.g.
> >> > > > > > >> > we couldn't get topics by regexp)
> >> > > > > > >> >
> >> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> >> > > > > > >> > DescribeTopicResponse -> [TopicName TopicDescription
> >> ErrorCode
> >> > > > > > >> > ErrorDescription] CommandErrorCode
> CommandErrorDescription
> >> > > > > > >> >
> >> > > > > > >> > Also, any thoughts about our discussion regarding
> re-routing
> >> > > > > facility?
> >> > > > > > >> In
> >> > > > > > >> > my
> >> > > > > > >> > understanding, it is like between augmenting
> >> > > TopicMetadataRequest
> >> > > > > > >> > (to include at least controllerId) and implementing new
> >> > generic
> >> > > > > > >> re-routing
> >> > > > > > >> > facility so sending messages to controller will be
> handled
> >> by
> >> > > it.
> >> > > > > > >> >
> >> > > > > > >> > Thanks,
> >> > > > > > >> > Andrii Biletskyi
> >> > > > > > >> >
> >> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> >> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > >> >
> >> > > > > > >> > > @Guozhang:
> >> > > > > > >> > > Thanks for your comments, I've answered some of those.
> The
> >> > > main
> >> > > > > > thing
> >> > > > > > >> is
> >> > > > > > >> > > having merged request for create-alter-delete-describe
> - I
> >> > > have
> >> > > > > some
> >> > > > > > >> > > concerns about this approach.
> >> > > > > > >> > >
> >> > > > > > >> > > @*Jay*:
> >> > > > > > >> > > I see that introduced ClusterMetadaRequest is also one
> of
> >> > the
> >> > > > > > >> concerns.
> >> > > > > > >> > We
> >> > > > > > >> > > can solve it if we implement re-routing facility. But I
> >> > agree
> >> > > > with
> >> > > > > > >> > > Guozhang - it will make clients' internals a little bit
> >> > easier
> >> > > > but
> >> > > > > > >> this
> >> > > > > > >> > > seems to be a complex logic to implement and support
> then.
> >> > > > > > Especially
> >> > > > > > >> for
> >> > > > > > >> > > Fetch and Produce (even if we add re-routing later for
> >> these
> >> > > > > > >> requests).
> >> > > > > > >> > > Also people will tend to avoid this re-routing facility
> >> and
> >> > > hold
> >> > > > > > local
> >> > > > > > >> > > cluster cache to ensure their high-priority requests
> >> (which
> >> > > some
> >> > > > > of
> >> > > > > > >> the
> >> > > > > > >> > > admin requests are) not sent to some busy broker where
> >> they
> >> > > wait
> >> > > > > to
> >> > > > > > be
> >> > > > > > >> > > routed to the correct one.
> >> > > > > > >> > > As pointed out by Jun here (
> >> > > > > > >> > >
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> >> > > > > > >> > )
> >> > > > > > >> > > to solve the issue we might introduce a message type to
> >> get
> >> > > > > cluster
> >> > > > > > >> > state.
> >> > > > > > >> > > But I agree we can just update TopicMetadataResponse to
> >> > > include
> >> > > > > > >> > > controllerId (and probably smth else).
> >> > > > > > >> > > What are you thougths?
> >> > > > > > >> > >
> >> > > > > > >> > > Thanks,
> >> > > > > > >> > > Andrii
> >> > > > > > >> > >
> >> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> >> > > > > wangguoz@gmail.com>
> >> > > > > > >> > wrote:
> >> > > > > > >> > >
> >> > > > > > >> > >> I think for the topics commands we can actually merge
> >> > > > > > >> > >> create/alter/delete/describe as one request type since
> >> > their
> >> > > > > > formats
> >> > > > > > >> are
> >> > > > > > >> > >> very much similar, and keep list-topics and others
> like
> >> > > > > > >> > >> partition-reassignment / preferred-leader-election as
> >> > > separate
> >> > > > > > >> request
> >> > > > > > >> > >> types, I also left some other comments on the RB (
> >> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> >> > > > > > >> > >>
> >> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> >> > > > jay.kreps@gmail.com>
> >> > > > > > >> wrote:
> >> > > > > > >> > >>
> >> > > > > > >> > >> > Yeah I totally agree that we don't want to just have
> >> one
> >> > > "do
> >> > > > > > admin
> >> > > > > > >> > >> stuff"
> >> > > > > > >> > >> > command that has the union of all parameters.
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > What I am saying is that command line tools are one
> >> > client
> >> > > of
> >> > > > > the
> >> > > > > > >> > >> > administrative apis, but these will be used in a
> number
> >> > of
> >> > > > > > >> scenarios
> >> > > > > > >> > so
> >> > > > > > >> > >> > they should make logical sense even in the absence
> of
> >> the
> >> > > > > command
> >> > > > > > >> line
> >> > > > > > >> > >> > tool. Hence comments like trying to clarify the
> >> > > relationship
> >> > > > > > >> between
> >> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds of
> >> things
> >> > > > > really
> >> > > > > > >> need
> >> > > > > > >> > >> to be
> >> > > > > > >> > >> > thought through.
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > Hope that makes sense.
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > -Jay
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> >> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > >> > >> >
> >> > > > > > >> > >> > > Jay,
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > Thanks for answering. You understood correctly,
> most
> >> of
> >> > > my
> >> > > > > > >> comments
> >> > > > > > >> > >> were
> >> > > > > > >> > >> > > related to your point 1) - about "well
> thought-out"
> >> > apis.
> >> > > > > Also,
> >> > > > > > >> yes,
> >> > > > > > >> > >> as I
> >> > > > > > >> > >> > > understood we would like to introduce a single
> >> unified
> >> > > CLI
> >> > > > > tool
> >> > > > > > >> with
> >> > > > > > >> > >> > > centralized server-side request handling for lots
> of
> >> > > > existing
> >> > > > > > >> ones
> >> > > > > > >> > >> (incl.
> >> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> >> ReassignPartitions,
> >> > > smth
> >> > > > > > else
> >> > > > > > >> if
> >> > > > > > >> > >> added
> >> > > > > > >> > >> > > in future). In our previous discussion (
> >> > > > > > >> > >> > > https://issues.apache.org/jira/browse/KAFKA-1694)
> >> > people
> >> > > > > said
> >> > > > > > >> > they'd
> >> > > > > > >> > >> > > rather
> >> > > > > > >> > >> > > have a separate message for each command, so, yes,
> >> this
> >> > > > way I
> >> > > > > > >> came
> >> > > > > > >> > to
> >> > > > > > >> > >> 1-1
> >> > > > > > >> > >> > > mapping between commands in the tool and protocol
> >> > > > additions.
> >> > > > > > But
> >> > > > > > >> I
> >> > > > > > >> > >> might
> >> > > > > > >> > >> > be
> >> > > > > > >> > >> > > wrong.
> >> > > > > > >> > >> > > At the end I just try to start discussion how at
> >> least
> >> > > > > > generally
> >> > > > > > >> > this
> >> > > > > > >> > >> > > protocol should look like.
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > Thanks,
> >> > > > > > >> > >> > > Andrii
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
> >> > > > > > jay.kreps@gmail.com
> >> > > > > > >> >
> >> > > > > > >> > >> wrote:
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> > > > Hey Andrii,
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > To answer your earlier question we just really
> >> can't
> >> > be
> >> > > > > > adding
> >> > > > > > >> any
> >> > > > > > >> > >> more
> >> > > > > > >> > >> > > > scala protocol objects. These things are super
> hard
> >> > to
> >> > > > > > maintain
> >> > > > > > >> > >> because
> >> > > > > > >> > >> > > > they hand code the byte parsing and don't have
> good
> >> > > > > > versioning
> >> > > > > > >> > >> support.
> >> > > > > > >> > >> > > > Since we are already planning on converting we
> >> > > definitely
> >> > > > > > don't
> >> > > > > > >> > >> want to
> >> > > > > > >> > >> > > add
> >> > > > > > >> > >> > > > a ton more of these--they are total tech debt.
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > What does it mean that the changes are isolated
> >> from
> >> > > the
> >> > > > > > >> current
> >> > > > > > >> > >> code
> >> > > > > > >> > >> > > base?
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > I actually didn't understand the remaining
> >> comments,
> >> > > > which
> >> > > > > of
> >> > > > > > >> the
> >> > > > > > >> > >> > points
> >> > > > > > >> > >> > > > are you responding to?
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > Maybe one sticking point here is that it seems
> like
> >> > you
> >> > > > > want
> >> > > > > > to
> >> > > > > > >> > make
> >> > > > > > >> > >> > some
> >> > > > > > >> > >> > > > kind of tool, and you have made a 1-1 mapping
> >> between
> >> > > > > > commands
> >> > > > > > >> you
> >> > > > > > >> > >> > > imagine
> >> > > > > > >> > >> > > > in the tool and protocol additions. I want to
> make
> >> > sure
> >> > > > we
> >> > > > > > >> don't
> >> > > > > > >> > do
> >> > > > > > >> > >> > that.
> >> > > > > > >> > >> > > > The protocol needs to be really really well
> thought
> >> > out
> >> > > > > > against
> >> > > > > > >> > many
> >> > > > > > >> > >> > use
> >> > > > > > >> > >> > > > cases so it should make perfect logical sense in
> >> the
> >> > > > > absence
> >> > > > > > of
> >> > > > > > >> > >> knowing
> >> > > > > > >> > >> > > the
> >> > > > > > >> > >> > > > command line tool, right?
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > -Jay
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii
> Biletskyi
> >> <
> >> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > > Hey Jay,
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > I would like to continue this discussion as it
> >> seem
> >> > > > there
> >> > > > > > is
> >> > > > > > >> no
> >> > > > > > >> > >> > > progress
> >> > > > > > >> > >> > > > > here.
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > First of all, could you please explain what
> did
> >> you
> >> > > > mean
> >> > > > > in
> >> > > > > > >> 2?
> >> > > > > > >> > How
> >> > > > > > >> > >> > > > exactly
> >> > > > > > >> > >> > > > > are we going to migrate to the new java
> protocol
> >> > > > > > definitions.
> >> > > > > > >> > And
> >> > > > > > >> > >> why
> >> > > > > > >> > >> > > > it's
> >> > > > > > >> > >> > > > > a blocker for centralized CLI?
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > I agree with you, this feature includes lots
> of
> >> > > stuff,
> >> > > > > but
> >> > > > > > >> > >> thankfully
> >> > > > > > >> > >> > > > > almost all changes are isolated from the
> current
> >> > code
> >> > > > > base,
> >> > > > > > >> > >> > > > > so the main thing, I think, we need to agree
> is
> >> > RQ/RP
> >> > > > > > format.
> >> > > > > > >> > >> > > > > So how can we start discussion about the
> concrete
> >> > > > > messages
> >> > > > > > >> > format?
> >> > > > > > >> > >> > > > > Can we take (
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> >
> >> > > > > > >> > >>
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> >> > > > > > >> > >> > > > > )
> >> > > > > > >> > >> > > > > as starting point?
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > We had some doubts earlier whether it worth
> >> > > introducing
> >> > > > > one
> >> > > > > > >> > >> generic
> >> > > > > > >> > >> > > Admin
> >> > > > > > >> > >> > > > > Request for all commands (
> >> > > > > > >> > >> > > >
> https://issues.apache.org/jira/browse/KAFKA-1694
> >> > > > > > >> > >> > > > > )
> >> > > > > > >> > >> > > > > but then everybody agreed it would be better
> to
> >> > have
> >> > > > > > separate
> >> > > > > > >> > >> message
> >> > > > > > >> > >> > > for
> >> > > > > > >> > >> > > > > each admin command. The Request part is really
> >> > > dictated
> >> > > > > > from
> >> > > > > > >> the
> >> > > > > > >> > >> > > command
> >> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so the
> >> > proposed
> >> > > > > > version
> >> > > > > > >> > >> should
> >> > > > > > >> > >> > be
> >> > > > > > >> > >> > > > > fine (let's put aside for now remarks about
> >> > Optional
> >> > > > > type,
> >> > > > > > >> > >> batching,
> >> > > > > > >> > >> > > > > configs normalization - I agree with all of
> >> them).
> >> > > > > > >> > >> > > > > So the second part is Response. I see there
> are
> >> two
> >> > > > cases
> >> > > > > > >> here.
> >> > > > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b)
> >> "Get"
> >> > > > > > requests -
> >> > > > > > >> > >> > > > > List/Describe...
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > a) should only hold request result (regardless
> >> what
> >> > > we
> >> > > > > > decide
> >> > > > > > >> > >> about
> >> > > > > > >> > >> > > > > blocking/non-blocking commands execution).
> >> > > > > > >> > >> > > > > Usually we provide error code in response but
> >> since
> >> > > we
> >> > > > > will
> >> > > > > > >> use
> >> > > > > > >> > >> this
> >> > > > > > >> > >> > in
> >> > > > > > >> > >> > > > > interactive shell we need some human readable
> >> error
> >> > > > > > >> description
> >> > > > > > >> > -
> >> > > > > > >> > >> so
> >> > > > > > >> > >> > I
> >> > > > > > >> > >> > > > > added errorDesription field where you can at
> >> least
> >> > > > leave
> >> > > > > > >> > >> > > > > exception.getMessage.
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > b) in addition to previous item message should
> >> hold
> >> > > > > command
> >> > > > > > >> > >> specific
> >> > > > > > >> > >> > > > > response data. We can discuss in detail each
> of
> >> > them
> >> > > > but
> >> > > > > > >> let's
> >> > > > > > >> > for
> >> > > > > > >> > >> > now
> >> > > > > > >> > >> > > > > agree about the overall pattern.
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > Thanks,
> >> > > > > > >> > >> > > > > Andrii Biletskyi
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
> >> > > > > > >> jay.kreps@gmail.com
> >> > > > > > >> > >
> >> > > > > > >> > >> > > wrote:
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > > > > Hey Joe,
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > This is great. A few comments on KIP-4
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 1. This is much needed functionality, but
> there
> >> > > are a
> >> > > > > lot
> >> > > > > > >> of
> >> > > > > > >> > >> the so
> >> > > > > > >> > >> > > > let's
> >> > > > > > >> > >> > > > > > really think these protocols through. We
> really
> >> > > want
> >> > > > to
> >> > > > > > >> end up
> >> > > > > > >> > >> > with a
> >> > > > > > >> > >> > > > set
> >> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis. For
> this
> >> > > > reason I
> >> > > > > > >> think
> >> > > > > > >> > >> it is
> >> > > > > > >> > >> > > > > really
> >> > > > > > >> > >> > > > > > important to think through the end state
> even
> >> if
> >> > > that
> >> > > > > > >> includes
> >> > > > > > >> > >> APIs
> >> > > > > > >> > >> > > we
> >> > > > > > >> > >> > > > > > won't implement in the first phase.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 2. Let's please please please wait until we
> >> have
> >> > > > > switched
> >> > > > > > >> the
> >> > > > > > >> > >> > server
> >> > > > > > >> > >> > > > over
> >> > > > > > >> > >> > > > > > to the new java protocol definitions. If we
> add
> >> > > > upteen
> >> > > > > > >> more ad
> >> > > > > > >> > >> hoc
> >> > > > > > >> > >> > > > scala
> >> > > > > > >> > >> > > > > > objects that is just generating more work
> for
> >> the
> >> > > > > > >> conversion
> >> > > > > > >> > we
> >> > > > > > >> > >> > know
> >> > > > > > >> > >> > > we
> >> > > > > > >> > >> > > > > > have to do.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 3. This proposal introduces a new type of
> >> > optional
> >> > > > > > >> parameter.
> >> > > > > > >> > >> This
> >> > > > > > >> > >> > is
> >> > > > > > >> > >> > > > > > inconsistent with everything else in the
> >> protocol
> >> > > > where
> >> > > > > > we
> >> > > > > > >> use
> >> > > > > > >> > >> -1
> >> > > > > > >> > >> > or
> >> > > > > > >> > >> > > > some
> >> > > > > > >> > >> > > > > > other marker value. You could argue either
> way
> >> > but
> >> > > > > let's
> >> > > > > > >> stick
> >> > > > > > >> > >> with
> >> > > > > > >> > >> > > > that
> >> > > > > > >> > >> > > > > > for consistency. For clients that
> implemented
> >> the
> >> > > > > > protocol
> >> > > > > > >> in
> >> > > > > > >> > a
> >> > > > > > >> > >> > > better
> >> > > > > > >> > >> > > > > way
> >> > > > > > >> > >> > > > > > than our scala code these basic primitives
> are
> >> > hard
> >> > > > to
> >> > > > > > >> change.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to duplicate
> >> > > > > > >> > TopicMetadataRequest
> >> > > > > > >> > >> > > which
> >> > > > > > >> > >> > > > > has
> >> > > > > > >> > >> > > > > > brokers, topics, and partitions. I think we
> >> > should
> >> > > > > rename
> >> > > > > > >> that
> >> > > > > > >> > >> > > request
> >> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> >> MetadataRequest)
> >> > > and
> >> > > > > > >> include
> >> > > > > > >> > >> the id
> >> > > > > > >> > >> > > of
> >> > > > > > >> > >> > > > > the
> >> > > > > > >> > >> > > > > > controller. Or are there other things we
> could
> >> > add
> >> > > > > here?
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 5. We have a tendency to try to make a lot
> of
> >> > > > requests
> >> > > > > > that
> >> > > > > > >> > can
> >> > > > > > >> > >> > only
> >> > > > > > >> > >> > > go
> >> > > > > > >> > >> > > > > to
> >> > > > > > >> > >> > > > > > particular nodes. This adds a lot of burden
> for
> >> > > > client
> >> > > > > > >> > >> > > implementations
> >> > > > > > >> > >> > > > > (it
> >> > > > > > >> > >> > > > > > sounds easy but each discovery can fail in
> many
> >> > > parts
> >> > > > > so
> >> > > > > > it
> >> > > > > > >> > >> ends up
> >> > > > > > >> > >> > > > > being a
> >> > > > > > >> > >> > > > > > full state machine to do right). I think we
> >> > should
> >> > > > > > consider
> >> > > > > > >> > >> making
> >> > > > > > >> > >> > > > admin
> >> > > > > > >> > >> > > > > > commands and ideally as many of the other
> apis
> >> as
> >> > > > > > possible
> >> > > > > > >> > >> > available
> >> > > > > > >> > >> > > on
> >> > > > > > >> > >> > > > > all
> >> > > > > > >> > >> > > > > > brokers and just redirect to the controller
> on
> >> > the
> >> > > > > broker
> >> > > > > > >> > side.
> >> > > > > > >> > >> > > Perhaps
> >> > > > > > >> > >> > > > > > there would be a general way to encapsulate
> >> this
> >> > > > > > re-routing
> >> > > > > > >> > >> > behavior.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 6. We should probably normalize the key
> value
> >> > pairs
> >> > > > > used
> >> > > > > > >> for
> >> > > > > > >> > >> > configs
> >> > > > > > >> > >> > > > > rather
> >> > > > > > >> > >> > > > > > than embedding a new formatting. So two
> strings
> >> > > > rather
> >> > > > > > than
> >> > > > > > >> > one
> >> > > > > > >> > >> > with
> >> > > > > > >> > >> > > an
> >> > > > > > >> > >> > > > > > internal equals sign.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs that
> the
> >> > > > command
> >> > > > > > has
> >> > > > > > >> > >> begun or
> >> > > > > > >> > >> > > > that
> >> > > > > > >> > >> > > > > > the command has been completed? It is a lot
> >> more
> >> > > > usable
> >> > > > > > if
> >> > > > > > >> the
> >> > > > > > >> > >> > > command
> >> > > > > > >> > >> > > > > has
> >> > > > > > >> > >> > > > > > been completed so you know that if you
> create a
> >> > > topic
> >> > > > > and
> >> > > > > > >> then
> >> > > > > > >> > >> > > publish
> >> > > > > > >> > >> > > > to
> >> > > > > > >> > >> > > > > > it you won't get an exception about there
> being
> >> > no
> >> > > > such
> >> > > > > > >> topic.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 8. Describe topic and list topics duplicate
> a
> >> lot
> >> > > of
> >> > > > > > stuff
> >> > > > > > >> in
> >> > > > > > >> > >> the
> >> > > > > > >> > >> > > > > metadata
> >> > > > > > >> > >> > > > > > request. Is there a reason to give back
> topics
> >> > > marked
> >> > > > > for
> >> > > > > > >> > >> > deletion? I
> >> > > > > > >> > >> > > > > feel
> >> > > > > > >> > >> > > > > > like if we just make the post-condition of
> the
> >> > > delete
> >> > > > > > >> command
> >> > > > > > >> > be
> >> > > > > > >> > >> > that
> >> > > > > > >> > >> > > > the
> >> > > > > > >> > >> > > > > > topic is deleted that will get rid of the
> need
> >> > for
> >> > > > this
> >> > > > > > >> right?
> >> > > > > > >> > >> And
> >> > > > > > >> > >> > it
> >> > > > > > >> > >> > > > > will
> >> > > > > > >> > >> > > > > > be much more intuitive.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 9. Should we consider batching these
> requests?
> >> We
> >> > > > have
> >> > > > > > >> > generally
> >> > > > > > >> > >> > > tried
> >> > > > > > >> > >> > > > to
> >> > > > > > >> > >> > > > > > allow multiple operations to be batched. My
> >> > > suspicion
> >> > > > > is
> >> > > > > > >> that
> >> > > > > > >> > >> > without
> >> > > > > > >> > >> > > > > this
> >> > > > > > >> > >> > > > > > we will get a lot of code that does
> something
> >> > like
> >> > > > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
> >> > > > > > >> > >> > > > > >       adminClient.describeTopic(topic)
> >> > > > > > >> > >> > > > > > this code will work great when you test on 5
> >> > topics
> >> > > > but
> >> > > > > > >> not do
> >> > > > > > >> > >> as
> >> > > > > > >> > >> > > well
> >> > > > > > >> > >> > > > if
> >> > > > > > >> > >> > > > > > you have 50k.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 10. I think we should also discuss how we
> want
> >> to
> >> > > > > expose
> >> > > > > > a
> >> > > > > > >> > >> > > programmatic
> >> > > > > > >> > >> > > > > JVM
> >> > > > > > >> > >> > > > > > client api for these operations. Currently
> >> people
> >> > > > rely
> >> > > > > on
> >> > > > > > >> > >> > AdminUtils
> >> > > > > > >> > >> > > > > which
> >> > > > > > >> > >> > > > > > is totally sketchy. I think we probably need
> >> > > another
> >> > > > > > client
> >> > > > > > >> > >> under
> >> > > > > > >> > >> > > > > clients/
> >> > > > > > >> > >> > > > > > that exposes administrative functionality.
> We
> >> > will
> >> > > > need
> >> > > > > > >> this
> >> > > > > > >> > >> just
> >> > > > > > >> > >> > to
> >> > > > > > >> > >> > > > > > properly test the new apis, I suspect. We
> >> should
> >> > > > figure
> >> > > > > > out
> >> > > > > > >> > that
> >> > > > > > >> > >> > API.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > 11. The other information that would be
> really
> >> > > useful
> >> > > > > to
> >> > > > > > >> get
> >> > > > > > >> > >> would
> >> > > > > > >> > >> > be
> >> > > > > > >> > >> > > > > > information about partitions--how much data
> is
> >> in
> >> > > the
> >> > > > > > >> > partition,
> >> > > > > > >> > >> > what
> >> > > > > > >> > >> > > > are
> >> > > > > > >> > >> > > > > > the segment offsets, what is the log-end
> offset
> >> > > (i.e.
> >> > > > > > last
> >> > > > > > >> > >> offset),
> >> > > > > > >> > >> > > > what
> >> > > > > > >> > >> > > > > is
> >> > > > > > >> > >> > > > > > the compaction point, etc. I think that done
> >> > right
> >> > > > this
> >> > > > > > >> would
> >> > > > > > >> > be
> >> > > > > > >> > >> > the
> >> > > > > > >> > >> > > > > > successor to the very awkward OffsetRequest
> we
> >> > have
> >> > > > > > today.
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > -Jay
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein
> <
> >> > > > > > >> > >> joe.stein@stealth.ly>
> >> > > > > > >> > >> > > > > wrote:
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > > > > Hi, created a KIP
> >> > > > > > >> > >> > > > > > >
> >> > > > > > >> > >> > > > > > >
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> >
> >> > > > > > >> > >>
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> > > > > > >> > >> > > > > > >
> >> > > > > > >> > >> > > > > > > JIRA
> >> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> >> > > > > > >> > >> > > > > > >
> >> > > > > > >> > >> > > > > > >
> /*******************************************
> >> > > > > > >> > >> > > > > > >  Joe Stein
> >> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> >> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
> >> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> >> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> >> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > > > > > >
> ********************************************/
> >> > > > > > >> > >> > > > > > >
> >> > > > > > >> > >> > > > > >
> >> > > > > > >> > >> > > > >
> >> > > > > > >> > >> > > >
> >> > > > > > >> > >> > >
> >> > > > > > >> > >> >
> >> > > > > > >> > >>
> >> > > > > > >> > >>
> >> > > > > > >> > >>
> >> > > > > > >> > >> --
> >> > > > > > >> > >> -- Guozhang
> >> > > > > > >> > >>
> >> > > > > > >> > >
> >> > > > > > >> > >
> >> > > > > > >> >
> >> > > > > > >>
> >> > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >> >
> >> >
> >> > --
> >> > Jeff Holoman
> >> > Systems Engineer
> >> >
> >>
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Gwen Shapira <gs...@cloudera.com>.
Thanks for sending this out Joe. Looking forward to chatting with everyone :)

On Mon, Mar 2, 2015 at 6:46 AM, Joe Stein <jo...@stealth.ly> wrote:
> Hey, I just sent out a google hangout invite to all pmc, committers and
> everyone I found working on a KIP. If I missed anyone in the invite please
> let me know and can update it, np.
>
> We should do this every Tuesday @ 2pm Eastern Time. Maybe we can get INFRA
> help to make a google account so we can manage better?
>
> To discuss
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> in progress and related JIRA that are interdependent and common work.
>
> ~ Joe Stein
>
> On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com> wrote:
>
>> Let's stay on Google hangouts that will also record and make the sessions
>> available on youtube.
>>
>> -Jay
>>
>> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <jh...@cloudera.com>
>> wrote:
>>
>> > Jay / Joe
>> >
>> > We're happy to send out a Webex for this purpose. We could record the
>> > sessions if there is interest and publish them out.
>> >
>> > Thanks
>> >
>> > Jeff
>> >
>> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <ja...@gmail.com> wrote:
>> >
>> > > Let's try to get the technical hang-ups sorted out, though. I really
>> > think
>> > > there is some benefit to live discussion vs writing. I am hopeful that
>> if
>> > > we post instructions and give ourselves a few attempts we can get it
>> > > working.
>> > >
>> > > Tuesday at that time would work for me...any objections?
>> > >
>> > > -Jay
>> > >
>> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <jo...@stealth.ly>
>> wrote:
>> > >
>> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am PT
>> ????
>> > > >
>> > > > I don't mind google hangout but there is always some issue or
>> whatever
>> > so
>> > > > we know the apache irc channel works. We can start there and see how
>> it
>> > > > goes? We can pull transcripts too and associate to tickets if need be
>> > > makes
>> > > > it helpful for things.
>> > > >
>> > > > ~ Joestein
>> > > >
>> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <ja...@gmail.com>
>> > wrote:
>> > > >
>> > > > > We'd talked about doing a Google Hangout to chat about this. What
>> > about
>> > > > > generalizing that a little further...I actually think it would be
>> > good
>> > > > for
>> > > > > everyone spending a reasonable chunk of their week on Kafka stuff
>> to
>> > > > maybe
>> > > > > sync up once a week. I think we could use time to talk through
>> design
>> > > > > stuff, make sure we are on top of code reviews, talk through any
>> > tricky
>> > > > > issues, etc.
>> > > > >
>> > > > > We can make it publicly available so that any one can follow along
>> > who
>> > > > > likes.
>> > > > >
>> > > > > Any interest in doing this? If so I'll try to set it up starting
>> next
>> > > > week.
>> > > > >
>> > > > > -Jay
>> > > > >
>> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
>> > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > >
>> > > > > > Hi all,
>> > > > > >
>> > > > > > I've updated KIP page, fixed / aligned document structure. Also I
>> > > added
>> > > > > > some
>> > > > > > very initial proposal for AdminClient so we have something to
>> start
>> > > > from
>> > > > > > while
>> > > > > > discussing the KIP.
>> > > > > >
>> > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>> > > > > >
>> > > > > > Thanks,
>> > > > > > Andrii Biletskyi
>> > > > > >
>> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
>> > > > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > >
>> > > > > > > Jay,
>> > > > > > >
>> > > > > > > Re error messages: you are right, in most cases client will
>> have
>> > > > enough
>> > > > > > > context to show descriptive error message. My concern is that
>> we
>> > > will
>> > > > > > have
>> > > > > > > to
>> > > > > > > add lots of new error codes for each possible error. Of course,
>> > we
>> > > > > could
>> > > > > > > reuse
>> > > > > > > some of existing like UknownTopicOrPartitionCode, but we will
>> > also
>> > > > need
>> > > > > > to
>> > > > > > > add smth like: TopicAlreadyExistsCode, TopicConfigInvalid (both
>> > for
>> > > > > topic
>> > > > > > > name and config, and probably user would like to know what
>> > exactly
>> > > > > > > is wrong in his config), InvalidReplicaAssignment,
>> InternalError
>> > > > (e.g.
>> > > > > > > zookeeper failure) etc.
>> > > > > > > And this is only for TopicCommand, we will also need to add
>> > similar
>> > > > > stuff
>> > > > > > > for
>> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up with a
>> > large
>> > > > list
>> > > > > > of
>> > > > > > > error codes, used only in Admin protocol.
>> > > > > > > Having said that, I agree my proposal is not consistent with
>> > other
>> > > > > cases.
>> > > > > > > Maybe we can find better solution or something in-between.
>> > > > > > >
>> > > > > > > Re Hangout chat: I think it is a great idea. This way we can
>> move
>> > > on
>> > > > > > > faster.
>> > > > > > > Let's agree somehow on date/time so people can join. Will work
>> > for
>> > > me
>> > > > > > this
>> > > > > > > and
>> > > > > > > next week almost anytime if agreed in advance.
>> > > > > > >
>> > > > > > > Thanks,
>> > > > > > > Andrii
>> > > > > > >
>> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
>> jay.kreps@gmail.com>
>> > > > > wrote:
>> > > > > > >
>> > > > > > >> Hey Andrii,
>> > > > > > >>
>> > > > > > >> Generally we can do good error handling without needing custom
>> > > > > > server-side
>> > > > > > >> messages. I.e. generally the client has the context to know
>> that
>> > > if
>> > > > it
>> > > > > > got
>> > > > > > >> an error that the topic doesn't exist to say "Topic X doesn't
>> > > exist"
>> > > > > > >> rather
>> > > > > > >> than "error code 14" (or whatever). Maybe there are specific
>> > cases
>> > > > > where
>> > > > > > >> this is hard? If we want to add server-side error messages we
>> > > really
>> > > > > do
>> > > > > > >> need to do this in a consistent way across the protocol.
>> > > > > > >>
>> > > > > > >> I still have a bunch of open questions here from my previous
>> > > list. I
>> > > > > > will
>> > > > > > >> be out for the next few days for Strata though. Maybe we could
>> > do
>> > > a
>> > > > > > Google
>> > > > > > >> Hangout chat on any open issues some time towards the end of
>> > next
>> > > > week
>> > > > > > for
>> > > > > > >> anyone interested in this ticket? I have a feeling that might
>> > > > progress
>> > > > > > >> things a little faster than email--I think we could talk
>> through
>> > > > those
>> > > > > > >> issues I brought up fairly quickly...
>> > > > > > >>
>> > > > > > >> -Jay
>> > > > > > >>
>> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
>> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
>> > > > > > >>
>> > > > > > >> > Hi all,
>> > > > > > >> >
>> > > > > > >> > I'm trying to address some of the issues which were
>> mentioned
>> > > > > earlier
>> > > > > > >> about
>> > > > > > >> > Admin RQ/RP format. One of those was about batching
>> > operations.
>> > > > What
>> > > > > > if
>> > > > > > >> we
>> > > > > > >> > follow TopicCommand approach and let people specify
>> topic-name
>> > > by
>> > > > > > >> regexp -
>> > > > > > >> > would that cover most of the use cases?
>> > > > > > >> >
>> > > > > > >> > Secondly, is what information should we generally provide in
>> > > Admin
>> > > > > > >> > responses.
>> > > > > > >> > I realize that Admin commands don't imply they will be used
>> > only
>> > > > in
>> > > > > > CLI
>> > > > > > >> > but,
>> > > > > > >> > it seems to me, CLI is a very important client of this
>> > feature.
>> > > In
>> > > > > > this
>> > > > > > >> > case,
>> > > > > > >> > seems logical, we would like to provide users with rich
>> > > experience
>> > > > > in
>> > > > > > >> terms
>> > > > > > >> > of
>> > > > > > >> > getting results / errors of the executed commands. Usually
>> we
>> > > > supply
>> > > > > > >> with
>> > > > > > >> > responses only errorCode, which looks very limiting, in case
>> > of
>> > > > CLI
>> > > > > we
>> > > > > > >> may
>> > > > > > >> > want to print human readable error description.
>> > > > > > >> >
>> > > > > > >> > So, taking into account previous item about batching, what
>> do
>> > > you
>> > > > > > think
>> > > > > > >> > about
>> > > > > > >> > having smth like:
>> > > > > > >> >
>> > > > > > >> > ('create' doesn't support regexp)
>> > > > > > >> > CreateTopicRequest => TopicName Partitions Replicas
>> > > > > ReplicaAssignment
>> > > > > > >> > [Config]
>> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
>> > > > > > >> >   ErrorCode => int16
>> > > > > > >> >   ErrorDescription => string (empty if successful)
>> > > > > > >> >
>> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
>> > > ReplicaAssignment
>> > > > > > >> > [AddedConfig] [DeletedConfig]
>> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
>> > > > > > >> > CommandErrorCode CommandErrorDescription
>> > > > > > >> >   CommandErrorCode => int16
>> > > > > > >> >   CommandErrorDescription => string (nonempty in case of
>> fatal
>> > > > > error,
>> > > > > > >> e.g.
>> > > > > > >> > we couldn't get topics by regexp)
>> > > > > > >> >
>> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
>> > > > > > >> > DescribeTopicResponse -> [TopicName TopicDescription
>> ErrorCode
>> > > > > > >> > ErrorDescription] CommandErrorCode CommandErrorDescription
>> > > > > > >> >
>> > > > > > >> > Also, any thoughts about our discussion regarding re-routing
>> > > > > facility?
>> > > > > > >> In
>> > > > > > >> > my
>> > > > > > >> > understanding, it is like between augmenting
>> > > TopicMetadataRequest
>> > > > > > >> > (to include at least controllerId) and implementing new
>> > generic
>> > > > > > >> re-routing
>> > > > > > >> > facility so sending messages to controller will be handled
>> by
>> > > it.
>> > > > > > >> >
>> > > > > > >> > Thanks,
>> > > > > > >> > Andrii Biletskyi
>> > > > > > >> >
>> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
>> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > >> >
>> > > > > > >> > > @Guozhang:
>> > > > > > >> > > Thanks for your comments, I've answered some of those. The
>> > > main
>> > > > > > thing
>> > > > > > >> is
>> > > > > > >> > > having merged request for create-alter-delete-describe - I
>> > > have
>> > > > > some
>> > > > > > >> > > concerns about this approach.
>> > > > > > >> > >
>> > > > > > >> > > @*Jay*:
>> > > > > > >> > > I see that introduced ClusterMetadaRequest is also one of
>> > the
>> > > > > > >> concerns.
>> > > > > > >> > We
>> > > > > > >> > > can solve it if we implement re-routing facility. But I
>> > agree
>> > > > with
>> > > > > > >> > > Guozhang - it will make clients' internals a little bit
>> > easier
>> > > > but
>> > > > > > >> this
>> > > > > > >> > > seems to be a complex logic to implement and support then.
>> > > > > > Especially
>> > > > > > >> for
>> > > > > > >> > > Fetch and Produce (even if we add re-routing later for
>> these
>> > > > > > >> requests).
>> > > > > > >> > > Also people will tend to avoid this re-routing facility
>> and
>> > > hold
>> > > > > > local
>> > > > > > >> > > cluster cache to ensure their high-priority requests
>> (which
>> > > some
>> > > > > of
>> > > > > > >> the
>> > > > > > >> > > admin requests are) not sent to some busy broker where
>> they
>> > > wait
>> > > > > to
>> > > > > > be
>> > > > > > >> > > routed to the correct one.
>> > > > > > >> > > As pointed out by Jun here (
>> > > > > > >> > >
>> > > > > > >> >
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
>> > > > > > >> > )
>> > > > > > >> > > to solve the issue we might introduce a message type to
>> get
>> > > > > cluster
>> > > > > > >> > state.
>> > > > > > >> > > But I agree we can just update TopicMetadataResponse to
>> > > include
>> > > > > > >> > > controllerId (and probably smth else).
>> > > > > > >> > > What are you thougths?
>> > > > > > >> > >
>> > > > > > >> > > Thanks,
>> > > > > > >> > > Andrii
>> > > > > > >> > >
>> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
>> > > > > wangguoz@gmail.com>
>> > > > > > >> > wrote:
>> > > > > > >> > >
>> > > > > > >> > >> I think for the topics commands we can actually merge
>> > > > > > >> > >> create/alter/delete/describe as one request type since
>> > their
>> > > > > > formats
>> > > > > > >> are
>> > > > > > >> > >> very much similar, and keep list-topics and others like
>> > > > > > >> > >> partition-reassignment / preferred-leader-election as
>> > > separate
>> > > > > > >> request
>> > > > > > >> > >> types, I also left some other comments on the RB (
>> > > > > > >> > >> https://reviews.apache.org/r/29301/).
>> > > > > > >> > >>
>> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
>> > > > jay.kreps@gmail.com>
>> > > > > > >> wrote:
>> > > > > > >> > >>
>> > > > > > >> > >> > Yeah I totally agree that we don't want to just have
>> one
>> > > "do
>> > > > > > admin
>> > > > > > >> > >> stuff"
>> > > > > > >> > >> > command that has the union of all parameters.
>> > > > > > >> > >> >
>> > > > > > >> > >> > What I am saying is that command line tools are one
>> > client
>> > > of
>> > > > > the
>> > > > > > >> > >> > administrative apis, but these will be used in a number
>> > of
>> > > > > > >> scenarios
>> > > > > > >> > so
>> > > > > > >> > >> > they should make logical sense even in the absence of
>> the
>> > > > > command
>> > > > > > >> line
>> > > > > > >> > >> > tool. Hence comments like trying to clarify the
>> > > relationship
>> > > > > > >> between
>> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds of
>> things
>> > > > > really
>> > > > > > >> need
>> > > > > > >> > >> to be
>> > > > > > >> > >> > thought through.
>> > > > > > >> > >> >
>> > > > > > >> > >> > Hope that makes sense.
>> > > > > > >> > >> >
>> > > > > > >> > >> > -Jay
>> > > > > > >> > >> >
>> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
>> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > >> > >> >
>> > > > > > >> > >> > > Jay,
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > Thanks for answering. You understood correctly, most
>> of
>> > > my
>> > > > > > >> comments
>> > > > > > >> > >> were
>> > > > > > >> > >> > > related to your point 1) - about "well thought-out"
>> > apis.
>> > > > > Also,
>> > > > > > >> yes,
>> > > > > > >> > >> as I
>> > > > > > >> > >> > > understood we would like to introduce a single
>> unified
>> > > CLI
>> > > > > tool
>> > > > > > >> with
>> > > > > > >> > >> > > centralized server-side request handling for lots of
>> > > > existing
>> > > > > > >> ones
>> > > > > > >> > >> (incl.
>> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
>> ReassignPartitions,
>> > > smth
>> > > > > > else
>> > > > > > >> if
>> > > > > > >> > >> added
>> > > > > > >> > >> > > in future). In our previous discussion (
>> > > > > > >> > >> > > https://issues.apache.org/jira/browse/KAFKA-1694)
>> > people
>> > > > > said
>> > > > > > >> > they'd
>> > > > > > >> > >> > > rather
>> > > > > > >> > >> > > have a separate message for each command, so, yes,
>> this
>> > > > way I
>> > > > > > >> came
>> > > > > > >> > to
>> > > > > > >> > >> 1-1
>> > > > > > >> > >> > > mapping between commands in the tool and protocol
>> > > > additions.
>> > > > > > But
>> > > > > > >> I
>> > > > > > >> > >> might
>> > > > > > >> > >> > be
>> > > > > > >> > >> > > wrong.
>> > > > > > >> > >> > > At the end I just try to start discussion how at
>> least
>> > > > > > generally
>> > > > > > >> > this
>> > > > > > >> > >> > > protocol should look like.
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > Thanks,
>> > > > > > >> > >> > > Andrii
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
>> > > > > > jay.kreps@gmail.com
>> > > > > > >> >
>> > > > > > >> > >> wrote:
>> > > > > > >> > >> > >
>> > > > > > >> > >> > > > Hey Andrii,
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > To answer your earlier question we just really
>> can't
>> > be
>> > > > > > adding
>> > > > > > >> any
>> > > > > > >> > >> more
>> > > > > > >> > >> > > > scala protocol objects. These things are super hard
>> > to
>> > > > > > maintain
>> > > > > > >> > >> because
>> > > > > > >> > >> > > > they hand code the byte parsing and don't have good
>> > > > > > versioning
>> > > > > > >> > >> support.
>> > > > > > >> > >> > > > Since we are already planning on converting we
>> > > definitely
>> > > > > > don't
>> > > > > > >> > >> want to
>> > > > > > >> > >> > > add
>> > > > > > >> > >> > > > a ton more of these--they are total tech debt.
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > What does it mean that the changes are isolated
>> from
>> > > the
>> > > > > > >> current
>> > > > > > >> > >> code
>> > > > > > >> > >> > > base?
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > I actually didn't understand the remaining
>> comments,
>> > > > which
>> > > > > of
>> > > > > > >> the
>> > > > > > >> > >> > points
>> > > > > > >> > >> > > > are you responding to?
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > Maybe one sticking point here is that it seems like
>> > you
>> > > > > want
>> > > > > > to
>> > > > > > >> > make
>> > > > > > >> > >> > some
>> > > > > > >> > >> > > > kind of tool, and you have made a 1-1 mapping
>> between
>> > > > > > commands
>> > > > > > >> you
>> > > > > > >> > >> > > imagine
>> > > > > > >> > >> > > > in the tool and protocol additions. I want to make
>> > sure
>> > > > we
>> > > > > > >> don't
>> > > > > > >> > do
>> > > > > > >> > >> > that.
>> > > > > > >> > >> > > > The protocol needs to be really really well thought
>> > out
>> > > > > > against
>> > > > > > >> > many
>> > > > > > >> > >> > use
>> > > > > > >> > >> > > > cases so it should make perfect logical sense in
>> the
>> > > > > absence
>> > > > > > of
>> > > > > > >> > >> knowing
>> > > > > > >> > >> > > the
>> > > > > > >> > >> > > > command line tool, right?
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > -Jay
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi
>> <
>> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > > Hey Jay,
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > > > I would like to continue this discussion as it
>> seem
>> > > > there
>> > > > > > is
>> > > > > > >> no
>> > > > > > >> > >> > > progress
>> > > > > > >> > >> > > > > here.
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > > > First of all, could you please explain what did
>> you
>> > > > mean
>> > > > > in
>> > > > > > >> 2?
>> > > > > > >> > How
>> > > > > > >> > >> > > > exactly
>> > > > > > >> > >> > > > > are we going to migrate to the new java protocol
>> > > > > > definitions.
>> > > > > > >> > And
>> > > > > > >> > >> why
>> > > > > > >> > >> > > > it's
>> > > > > > >> > >> > > > > a blocker for centralized CLI?
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > > > I agree with you, this feature includes lots of
>> > > stuff,
>> > > > > but
>> > > > > > >> > >> thankfully
>> > > > > > >> > >> > > > > almost all changes are isolated from the current
>> > code
>> > > > > base,
>> > > > > > >> > >> > > > > so the main thing, I think, we need to agree is
>> > RQ/RP
>> > > > > > format.
>> > > > > > >> > >> > > > > So how can we start discussion about the concrete
>> > > > > messages
>> > > > > > >> > format?
>> > > > > > >> > >> > > > > Can we take (
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > >
>> > > > > > >> > >> >
>> > > > > > >> > >>
>> > > > > > >> >
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
>> > > > > > >> > >> > > > > )
>> > > > > > >> > >> > > > > as starting point?
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > > > We had some doubts earlier whether it worth
>> > > introducing
>> > > > > one
>> > > > > > >> > >> generic
>> > > > > > >> > >> > > Admin
>> > > > > > >> > >> > > > > Request for all commands (
>> > > > > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-1694
>> > > > > > >> > >> > > > > )
>> > > > > > >> > >> > > > > but then everybody agreed it would be better to
>> > have
>> > > > > > separate
>> > > > > > >> > >> message
>> > > > > > >> > >> > > for
>> > > > > > >> > >> > > > > each admin command. The Request part is really
>> > > dictated
>> > > > > > from
>> > > > > > >> the
>> > > > > > >> > >> > > command
>> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so the
>> > proposed
>> > > > > > version
>> > > > > > >> > >> should
>> > > > > > >> > >> > be
>> > > > > > >> > >> > > > > fine (let's put aside for now remarks about
>> > Optional
>> > > > > type,
>> > > > > > >> > >> batching,
>> > > > > > >> > >> > > > > configs normalization - I agree with all of
>> them).
>> > > > > > >> > >> > > > > So the second part is Response. I see there are
>> two
>> > > > cases
>> > > > > > >> here.
>> > > > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b)
>> "Get"
>> > > > > > requests -
>> > > > > > >> > >> > > > > List/Describe...
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > > > a) should only hold request result (regardless
>> what
>> > > we
>> > > > > > decide
>> > > > > > >> > >> about
>> > > > > > >> > >> > > > > blocking/non-blocking commands execution).
>> > > > > > >> > >> > > > > Usually we provide error code in response but
>> since
>> > > we
>> > > > > will
>> > > > > > >> use
>> > > > > > >> > >> this
>> > > > > > >> > >> > in
>> > > > > > >> > >> > > > > interactive shell we need some human readable
>> error
>> > > > > > >> description
>> > > > > > >> > -
>> > > > > > >> > >> so
>> > > > > > >> > >> > I
>> > > > > > >> > >> > > > > added errorDesription field where you can at
>> least
>> > > > leave
>> > > > > > >> > >> > > > > exception.getMessage.
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > > > b) in addition to previous item message should
>> hold
>> > > > > command
>> > > > > > >> > >> specific
>> > > > > > >> > >> > > > > response data. We can discuss in detail each of
>> > them
>> > > > but
>> > > > > > >> let's
>> > > > > > >> > for
>> > > > > > >> > >> > now
>> > > > > > >> > >> > > > > agree about the overall pattern.
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > > > Thanks,
>> > > > > > >> > >> > > > > Andrii Biletskyi
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
>> > > > > > >> jay.kreps@gmail.com
>> > > > > > >> > >
>> > > > > > >> > >> > > wrote:
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > > > > Hey Joe,
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > This is great. A few comments on KIP-4
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > 1. This is much needed functionality, but there
>> > > are a
>> > > > > lot
>> > > > > > >> of
>> > > > > > >> > >> the so
>> > > > > > >> > >> > > > let's
>> > > > > > >> > >> > > > > > really think these protocols through. We really
>> > > want
>> > > > to
>> > > > > > >> end up
>> > > > > > >> > >> > with a
>> > > > > > >> > >> > > > set
>> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis. For this
>> > > > reason I
>> > > > > > >> think
>> > > > > > >> > >> it is
>> > > > > > >> > >> > > > > really
>> > > > > > >> > >> > > > > > important to think through the end state even
>> if
>> > > that
>> > > > > > >> includes
>> > > > > > >> > >> APIs
>> > > > > > >> > >> > > we
>> > > > > > >> > >> > > > > > won't implement in the first phase.
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > 2. Let's please please please wait until we
>> have
>> > > > > switched
>> > > > > > >> the
>> > > > > > >> > >> > server
>> > > > > > >> > >> > > > over
>> > > > > > >> > >> > > > > > to the new java protocol definitions. If we add
>> > > > upteen
>> > > > > > >> more ad
>> > > > > > >> > >> hoc
>> > > > > > >> > >> > > > scala
>> > > > > > >> > >> > > > > > objects that is just generating more work for
>> the
>> > > > > > >> conversion
>> > > > > > >> > we
>> > > > > > >> > >> > know
>> > > > > > >> > >> > > we
>> > > > > > >> > >> > > > > > have to do.
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > 3. This proposal introduces a new type of
>> > optional
>> > > > > > >> parameter.
>> > > > > > >> > >> This
>> > > > > > >> > >> > is
>> > > > > > >> > >> > > > > > inconsistent with everything else in the
>> protocol
>> > > > where
>> > > > > > we
>> > > > > > >> use
>> > > > > > >> > >> -1
>> > > > > > >> > >> > or
>> > > > > > >> > >> > > > some
>> > > > > > >> > >> > > > > > other marker value. You could argue either way
>> > but
>> > > > > let's
>> > > > > > >> stick
>> > > > > > >> > >> with
>> > > > > > >> > >> > > > that
>> > > > > > >> > >> > > > > > for consistency. For clients that implemented
>> the
>> > > > > > protocol
>> > > > > > >> in
>> > > > > > >> > a
>> > > > > > >> > >> > > better
>> > > > > > >> > >> > > > > way
>> > > > > > >> > >> > > > > > than our scala code these basic primitives are
>> > hard
>> > > > to
>> > > > > > >> change.
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to duplicate
>> > > > > > >> > TopicMetadataRequest
>> > > > > > >> > >> > > which
>> > > > > > >> > >> > > > > has
>> > > > > > >> > >> > > > > > brokers, topics, and partitions. I think we
>> > should
>> > > > > rename
>> > > > > > >> that
>> > > > > > >> > >> > > request
>> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
>> MetadataRequest)
>> > > and
>> > > > > > >> include
>> > > > > > >> > >> the id
>> > > > > > >> > >> > > of
>> > > > > > >> > >> > > > > the
>> > > > > > >> > >> > > > > > controller. Or are there other things we could
>> > add
>> > > > > here?
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > 5. We have a tendency to try to make a lot of
>> > > > requests
>> > > > > > that
>> > > > > > >> > can
>> > > > > > >> > >> > only
>> > > > > > >> > >> > > go
>> > > > > > >> > >> > > > > to
>> > > > > > >> > >> > > > > > particular nodes. This adds a lot of burden for
>> > > > client
>> > > > > > >> > >> > > implementations
>> > > > > > >> > >> > > > > (it
>> > > > > > >> > >> > > > > > sounds easy but each discovery can fail in many
>> > > parts
>> > > > > so
>> > > > > > it
>> > > > > > >> > >> ends up
>> > > > > > >> > >> > > > > being a
>> > > > > > >> > >> > > > > > full state machine to do right). I think we
>> > should
>> > > > > > consider
>> > > > > > >> > >> making
>> > > > > > >> > >> > > > admin
>> > > > > > >> > >> > > > > > commands and ideally as many of the other apis
>> as
>> > > > > > possible
>> > > > > > >> > >> > available
>> > > > > > >> > >> > > on
>> > > > > > >> > >> > > > > all
>> > > > > > >> > >> > > > > > brokers and just redirect to the controller on
>> > the
>> > > > > broker
>> > > > > > >> > side.
>> > > > > > >> > >> > > Perhaps
>> > > > > > >> > >> > > > > > there would be a general way to encapsulate
>> this
>> > > > > > re-routing
>> > > > > > >> > >> > behavior.
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > 6. We should probably normalize the key value
>> > pairs
>> > > > > used
>> > > > > > >> for
>> > > > > > >> > >> > configs
>> > > > > > >> > >> > > > > rather
>> > > > > > >> > >> > > > > > than embedding a new formatting. So two strings
>> > > > rather
>> > > > > > than
>> > > > > > >> > one
>> > > > > > >> > >> > with
>> > > > > > >> > >> > > an
>> > > > > > >> > >> > > > > > internal equals sign.
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs that the
>> > > > command
>> > > > > > has
>> > > > > > >> > >> begun or
>> > > > > > >> > >> > > > that
>> > > > > > >> > >> > > > > > the command has been completed? It is a lot
>> more
>> > > > usable
>> > > > > > if
>> > > > > > >> the
>> > > > > > >> > >> > > command
>> > > > > > >> > >> > > > > has
>> > > > > > >> > >> > > > > > been completed so you know that if you create a
>> > > topic
>> > > > > and
>> > > > > > >> then
>> > > > > > >> > >> > > publish
>> > > > > > >> > >> > > > to
>> > > > > > >> > >> > > > > > it you won't get an exception about there being
>> > no
>> > > > such
>> > > > > > >> topic.
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > 8. Describe topic and list topics duplicate a
>> lot
>> > > of
>> > > > > > stuff
>> > > > > > >> in
>> > > > > > >> > >> the
>> > > > > > >> > >> > > > > metadata
>> > > > > > >> > >> > > > > > request. Is there a reason to give back topics
>> > > marked
>> > > > > for
>> > > > > > >> > >> > deletion? I
>> > > > > > >> > >> > > > > feel
>> > > > > > >> > >> > > > > > like if we just make the post-condition of the
>> > > delete
>> > > > > > >> command
>> > > > > > >> > be
>> > > > > > >> > >> > that
>> > > > > > >> > >> > > > the
>> > > > > > >> > >> > > > > > topic is deleted that will get rid of the need
>> > for
>> > > > this
>> > > > > > >> right?
>> > > > > > >> > >> And
>> > > > > > >> > >> > it
>> > > > > > >> > >> > > > > will
>> > > > > > >> > >> > > > > > be much more intuitive.
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > 9. Should we consider batching these requests?
>> We
>> > > > have
>> > > > > > >> > generally
>> > > > > > >> > >> > > tried
>> > > > > > >> > >> > > > to
>> > > > > > >> > >> > > > > > allow multiple operations to be batched. My
>> > > suspicion
>> > > > > is
>> > > > > > >> that
>> > > > > > >> > >> > without
>> > > > > > >> > >> > > > > this
>> > > > > > >> > >> > > > > > we will get a lot of code that does something
>> > like
>> > > > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
>> > > > > > >> > >> > > > > >       adminClient.describeTopic(topic)
>> > > > > > >> > >> > > > > > this code will work great when you test on 5
>> > topics
>> > > > but
>> > > > > > >> not do
>> > > > > > >> > >> as
>> > > > > > >> > >> > > well
>> > > > > > >> > >> > > > if
>> > > > > > >> > >> > > > > > you have 50k.
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > 10. I think we should also discuss how we want
>> to
>> > > > > expose
>> > > > > > a
>> > > > > > >> > >> > > programmatic
>> > > > > > >> > >> > > > > JVM
>> > > > > > >> > >> > > > > > client api for these operations. Currently
>> people
>> > > > rely
>> > > > > on
>> > > > > > >> > >> > AdminUtils
>> > > > > > >> > >> > > > > which
>> > > > > > >> > >> > > > > > is totally sketchy. I think we probably need
>> > > another
>> > > > > > client
>> > > > > > >> > >> under
>> > > > > > >> > >> > > > > clients/
>> > > > > > >> > >> > > > > > that exposes administrative functionality. We
>> > will
>> > > > need
>> > > > > > >> this
>> > > > > > >> > >> just
>> > > > > > >> > >> > to
>> > > > > > >> > >> > > > > > properly test the new apis, I suspect. We
>> should
>> > > > figure
>> > > > > > out
>> > > > > > >> > that
>> > > > > > >> > >> > API.
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > 11. The other information that would be really
>> > > useful
>> > > > > to
>> > > > > > >> get
>> > > > > > >> > >> would
>> > > > > > >> > >> > be
>> > > > > > >> > >> > > > > > information about partitions--how much data is
>> in
>> > > the
>> > > > > > >> > partition,
>> > > > > > >> > >> > what
>> > > > > > >> > >> > > > are
>> > > > > > >> > >> > > > > > the segment offsets, what is the log-end offset
>> > > (i.e.
>> > > > > > last
>> > > > > > >> > >> offset),
>> > > > > > >> > >> > > > what
>> > > > > > >> > >> > > > > is
>> > > > > > >> > >> > > > > > the compaction point, etc. I think that done
>> > right
>> > > > this
>> > > > > > >> would
>> > > > > > >> > be
>> > > > > > >> > >> > the
>> > > > > > >> > >> > > > > > successor to the very awkward OffsetRequest we
>> > have
>> > > > > > today.
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > -Jay
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
>> > > > > > >> > >> joe.stein@stealth.ly>
>> > > > > > >> > >> > > > > wrote:
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > > > > Hi, created a KIP
>> > > > > > >> > >> > > > > > >
>> > > > > > >> > >> > > > > > >
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > >
>> > > > > > >> > >> >
>> > > > > > >> > >>
>> > > > > > >> >
>> > > > > > >>
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>> > > > > > >> > >> > > > > > >
>> > > > > > >> > >> > > > > > > JIRA
>> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
>> > > > > > >> > >> > > > > > >
>> > > > > > >> > >> > > > > > > /*******************************************
>> > > > > > >> > >> > > > > > >  Joe Stein
>> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
>> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
>> > > > > > >> > >> > > > > > >  http://www.stealth.ly
>> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
>> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > > > > > > ********************************************/
>> > > > > > >> > >> > > > > > >
>> > > > > > >> > >> > > > > >
>> > > > > > >> > >> > > > >
>> > > > > > >> > >> > > >
>> > > > > > >> > >> > >
>> > > > > > >> > >> >
>> > > > > > >> > >>
>> > > > > > >> > >>
>> > > > > > >> > >>
>> > > > > > >> > >> --
>> > > > > > >> > >> -- Guozhang
>> > > > > > >> > >>
>> > > > > > >> > >
>> > > > > > >> > >
>> > > > > > >> >
>> > > > > > >>
>> > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> >
>> >
>> > --
>> > Jeff Holoman
>> > Systems Engineer
>> >
>>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Joe Stein <jo...@stealth.ly>.
Hey, I just sent out a google hangout invite to all pmc, committers and
everyone I found working on a KIP. If I missed anyone in the invite please
let me know and can update it, np.

We should do this every Tuesday @ 2pm Eastern Time. Maybe we can get INFRA
help to make a google account so we can manage better?

To discuss
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
in progress and related JIRA that are interdependent and common work.

~ Joe Stein

On Tue, Feb 24, 2015 at 2:59 PM, Jay Kreps <ja...@gmail.com> wrote:

> Let's stay on Google hangouts that will also record and make the sessions
> available on youtube.
>
> -Jay
>
> On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <jh...@cloudera.com>
> wrote:
>
> > Jay / Joe
> >
> > We're happy to send out a Webex for this purpose. We could record the
> > sessions if there is interest and publish them out.
> >
> > Thanks
> >
> > Jeff
> >
> > On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Let's try to get the technical hang-ups sorted out, though. I really
> > think
> > > there is some benefit to live discussion vs writing. I am hopeful that
> if
> > > we post instructions and give ourselves a few attempts we can get it
> > > working.
> > >
> > > Tuesday at that time would work for me...any objections?
> > >
> > > -Jay
> > >
> > > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <jo...@stealth.ly>
> wrote:
> > >
> > > > Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am PT
> ????
> > > >
> > > > I don't mind google hangout but there is always some issue or
> whatever
> > so
> > > > we know the apache irc channel works. We can start there and see how
> it
> > > > goes? We can pull transcripts too and associate to tickets if need be
> > > makes
> > > > it helpful for things.
> > > >
> > > > ~ Joestein
> > > >
> > > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > We'd talked about doing a Google Hangout to chat about this. What
> > about
> > > > > generalizing that a little further...I actually think it would be
> > good
> > > > for
> > > > > everyone spending a reasonable chunk of their week on Kafka stuff
> to
> > > > maybe
> > > > > sync up once a week. I think we could use time to talk through
> design
> > > > > stuff, make sure we are on top of code reviews, talk through any
> > tricky
> > > > > issues, etc.
> > > > >
> > > > > We can make it publicly available so that any one can follow along
> > who
> > > > > likes.
> > > > >
> > > > > Any interest in doing this? If so I'll try to set it up starting
> next
> > > > week.
> > > > >
> > > > > -Jay
> > > > >
> > > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > I've updated KIP page, fixed / aligned document structure. Also I
> > > added
> > > > > > some
> > > > > > very initial proposal for AdminClient so we have something to
> start
> > > > from
> > > > > > while
> > > > > > discussing the KIP.
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii Biletskyi
> > > > > >
> > > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> > > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >
> > > > > > > Jay,
> > > > > > >
> > > > > > > Re error messages: you are right, in most cases client will
> have
> > > > enough
> > > > > > > context to show descriptive error message. My concern is that
> we
> > > will
> > > > > > have
> > > > > > > to
> > > > > > > add lots of new error codes for each possible error. Of course,
> > we
> > > > > could
> > > > > > > reuse
> > > > > > > some of existing like UknownTopicOrPartitionCode, but we will
> > also
> > > > need
> > > > > > to
> > > > > > > add smth like: TopicAlreadyExistsCode, TopicConfigInvalid (both
> > for
> > > > > topic
> > > > > > > name and config, and probably user would like to know what
> > exactly
> > > > > > > is wrong in his config), InvalidReplicaAssignment,
> InternalError
> > > > (e.g.
> > > > > > > zookeeper failure) etc.
> > > > > > > And this is only for TopicCommand, we will also need to add
> > similar
> > > > > stuff
> > > > > > > for
> > > > > > > ReassignPartitions, PreferredReplica. So we'll end up with a
> > large
> > > > list
> > > > > > of
> > > > > > > error codes, used only in Admin protocol.
> > > > > > > Having said that, I agree my proposal is not consistent with
> > other
> > > > > cases.
> > > > > > > Maybe we can find better solution or something in-between.
> > > > > > >
> > > > > > > Re Hangout chat: I think it is a great idea. This way we can
> move
> > > on
> > > > > > > faster.
> > > > > > > Let's agree somehow on date/time so people can join. Will work
> > for
> > > me
> > > > > > this
> > > > > > > and
> > > > > > > next week almost anytime if agreed in advance.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Andrii
> > > > > > >
> > > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <
> jay.kreps@gmail.com>
> > > > > wrote:
> > > > > > >
> > > > > > >> Hey Andrii,
> > > > > > >>
> > > > > > >> Generally we can do good error handling without needing custom
> > > > > > server-side
> > > > > > >> messages. I.e. generally the client has the context to know
> that
> > > if
> > > > it
> > > > > > got
> > > > > > >> an error that the topic doesn't exist to say "Topic X doesn't
> > > exist"
> > > > > > >> rather
> > > > > > >> than "error code 14" (or whatever). Maybe there are specific
> > cases
> > > > > where
> > > > > > >> this is hard? If we want to add server-side error messages we
> > > really
> > > > > do
> > > > > > >> need to do this in a consistent way across the protocol.
> > > > > > >>
> > > > > > >> I still have a bunch of open questions here from my previous
> > > list. I
> > > > > > will
> > > > > > >> be out for the next few days for Strata though. Maybe we could
> > do
> > > a
> > > > > > Google
> > > > > > >> Hangout chat on any open issues some time towards the end of
> > next
> > > > week
> > > > > > for
> > > > > > >> anyone interested in this ticket? I have a feeling that might
> > > > progress
> > > > > > >> things a little faster than email--I think we could talk
> through
> > > > those
> > > > > > >> issues I brought up fairly quickly...
> > > > > > >>
> > > > > > >> -Jay
> > > > > > >>
> > > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> > > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > > > > > >>
> > > > > > >> > Hi all,
> > > > > > >> >
> > > > > > >> > I'm trying to address some of the issues which were
> mentioned
> > > > > earlier
> > > > > > >> about
> > > > > > >> > Admin RQ/RP format. One of those was about batching
> > operations.
> > > > What
> > > > > > if
> > > > > > >> we
> > > > > > >> > follow TopicCommand approach and let people specify
> topic-name
> > > by
> > > > > > >> regexp -
> > > > > > >> > would that cover most of the use cases?
> > > > > > >> >
> > > > > > >> > Secondly, is what information should we generally provide in
> > > Admin
> > > > > > >> > responses.
> > > > > > >> > I realize that Admin commands don't imply they will be used
> > only
> > > > in
> > > > > > CLI
> > > > > > >> > but,
> > > > > > >> > it seems to me, CLI is a very important client of this
> > feature.
> > > In
> > > > > > this
> > > > > > >> > case,
> > > > > > >> > seems logical, we would like to provide users with rich
> > > experience
> > > > > in
> > > > > > >> terms
> > > > > > >> > of
> > > > > > >> > getting results / errors of the executed commands. Usually
> we
> > > > supply
> > > > > > >> with
> > > > > > >> > responses only errorCode, which looks very limiting, in case
> > of
> > > > CLI
> > > > > we
> > > > > > >> may
> > > > > > >> > want to print human readable error description.
> > > > > > >> >
> > > > > > >> > So, taking into account previous item about batching, what
> do
> > > you
> > > > > > think
> > > > > > >> > about
> > > > > > >> > having smth like:
> > > > > > >> >
> > > > > > >> > ('create' doesn't support regexp)
> > > > > > >> > CreateTopicRequest => TopicName Partitions Replicas
> > > > > ReplicaAssignment
> > > > > > >> > [Config]
> > > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
> > > > > > >> >   ErrorCode => int16
> > > > > > >> >   ErrorDescription => string (empty if successful)
> > > > > > >> >
> > > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> > > ReplicaAssignment
> > > > > > >> > [AddedConfig] [DeletedConfig]
> > > > > > >> > AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
> > > > > > >> > CommandErrorCode CommandErrorDescription
> > > > > > >> >   CommandErrorCode => int16
> > > > > > >> >   CommandErrorDescription => string (nonempty in case of
> fatal
> > > > > error,
> > > > > > >> e.g.
> > > > > > >> > we couldn't get topics by regexp)
> > > > > > >> >
> > > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> > > > > > >> > DescribeTopicResponse -> [TopicName TopicDescription
> ErrorCode
> > > > > > >> > ErrorDescription] CommandErrorCode CommandErrorDescription
> > > > > > >> >
> > > > > > >> > Also, any thoughts about our discussion regarding re-routing
> > > > > facility?
> > > > > > >> In
> > > > > > >> > my
> > > > > > >> > understanding, it is like between augmenting
> > > TopicMetadataRequest
> > > > > > >> > (to include at least controllerId) and implementing new
> > generic
> > > > > > >> re-routing
> > > > > > >> > facility so sending messages to controller will be handled
> by
> > > it.
> > > > > > >> >
> > > > > > >> > Thanks,
> > > > > > >> > Andrii Biletskyi
> > > > > > >> >
> > > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> > > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >> >
> > > > > > >> > > @Guozhang:
> > > > > > >> > > Thanks for your comments, I've answered some of those. The
> > > main
> > > > > > thing
> > > > > > >> is
> > > > > > >> > > having merged request for create-alter-delete-describe - I
> > > have
> > > > > some
> > > > > > >> > > concerns about this approach.
> > > > > > >> > >
> > > > > > >> > > @*Jay*:
> > > > > > >> > > I see that introduced ClusterMetadaRequest is also one of
> > the
> > > > > > >> concerns.
> > > > > > >> > We
> > > > > > >> > > can solve it if we implement re-routing facility. But I
> > agree
> > > > with
> > > > > > >> > > Guozhang - it will make clients' internals a little bit
> > easier
> > > > but
> > > > > > >> this
> > > > > > >> > > seems to be a complex logic to implement and support then.
> > > > > > Especially
> > > > > > >> for
> > > > > > >> > > Fetch and Produce (even if we add re-routing later for
> these
> > > > > > >> requests).
> > > > > > >> > > Also people will tend to avoid this re-routing facility
> and
> > > hold
> > > > > > local
> > > > > > >> > > cluster cache to ensure their high-priority requests
> (which
> > > some
> > > > > of
> > > > > > >> the
> > > > > > >> > > admin requests are) not sent to some busy broker where
> they
> > > wait
> > > > > to
> > > > > > be
> > > > > > >> > > routed to the correct one.
> > > > > > >> > > As pointed out by Jun here (
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > > > > >> > )
> > > > > > >> > > to solve the issue we might introduce a message type to
> get
> > > > > cluster
> > > > > > >> > state.
> > > > > > >> > > But I agree we can just update TopicMetadataResponse to
> > > include
> > > > > > >> > > controllerId (and probably smth else).
> > > > > > >> > > What are you thougths?
> > > > > > >> > >
> > > > > > >> > > Thanks,
> > > > > > >> > > Andrii
> > > > > > >> > >
> > > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> > > > > wangguoz@gmail.com>
> > > > > > >> > wrote:
> > > > > > >> > >
> > > > > > >> > >> I think for the topics commands we can actually merge
> > > > > > >> > >> create/alter/delete/describe as one request type since
> > their
> > > > > > formats
> > > > > > >> are
> > > > > > >> > >> very much similar, and keep list-topics and others like
> > > > > > >> > >> partition-reassignment / preferred-leader-election as
> > > separate
> > > > > > >> request
> > > > > > >> > >> types, I also left some other comments on the RB (
> > > > > > >> > >> https://reviews.apache.org/r/29301/).
> > > > > > >> > >>
> > > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> > > > jay.kreps@gmail.com>
> > > > > > >> wrote:
> > > > > > >> > >>
> > > > > > >> > >> > Yeah I totally agree that we don't want to just have
> one
> > > "do
> > > > > > admin
> > > > > > >> > >> stuff"
> > > > > > >> > >> > command that has the union of all parameters.
> > > > > > >> > >> >
> > > > > > >> > >> > What I am saying is that command line tools are one
> > client
> > > of
> > > > > the
> > > > > > >> > >> > administrative apis, but these will be used in a number
> > of
> > > > > > >> scenarios
> > > > > > >> > so
> > > > > > >> > >> > they should make logical sense even in the absence of
> the
> > > > > command
> > > > > > >> line
> > > > > > >> > >> > tool. Hence comments like trying to clarify the
> > > relationship
> > > > > > >> between
> > > > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds of
> things
> > > > > really
> > > > > > >> need
> > > > > > >> > >> to be
> > > > > > >> > >> > thought through.
> > > > > > >> > >> >
> > > > > > >> > >> > Hope that makes sense.
> > > > > > >> > >> >
> > > > > > >> > >> > -Jay
> > > > > > >> > >> >
> > > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> > > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >> > >> >
> > > > > > >> > >> > > Jay,
> > > > > > >> > >> > >
> > > > > > >> > >> > > Thanks for answering. You understood correctly, most
> of
> > > my
> > > > > > >> comments
> > > > > > >> > >> were
> > > > > > >> > >> > > related to your point 1) - about "well thought-out"
> > apis.
> > > > > Also,
> > > > > > >> yes,
> > > > > > >> > >> as I
> > > > > > >> > >> > > understood we would like to introduce a single
> unified
> > > CLI
> > > > > tool
> > > > > > >> with
> > > > > > >> > >> > > centralized server-side request handling for lots of
> > > > existing
> > > > > > >> ones
> > > > > > >> > >> (incl.
> > > > > > >> > >> > > TopicCommand, CommitOffsetChecker,
> ReassignPartitions,
> > > smth
> > > > > > else
> > > > > > >> if
> > > > > > >> > >> added
> > > > > > >> > >> > > in future). In our previous discussion (
> > > > > > >> > >> > > https://issues.apache.org/jira/browse/KAFKA-1694)
> > people
> > > > > said
> > > > > > >> > they'd
> > > > > > >> > >> > > rather
> > > > > > >> > >> > > have a separate message for each command, so, yes,
> this
> > > > way I
> > > > > > >> came
> > > > > > >> > to
> > > > > > >> > >> 1-1
> > > > > > >> > >> > > mapping between commands in the tool and protocol
> > > > additions.
> > > > > > But
> > > > > > >> I
> > > > > > >> > >> might
> > > > > > >> > >> > be
> > > > > > >> > >> > > wrong.
> > > > > > >> > >> > > At the end I just try to start discussion how at
> least
> > > > > > generally
> > > > > > >> > this
> > > > > > >> > >> > > protocol should look like.
> > > > > > >> > >> > >
> > > > > > >> > >> > > Thanks,
> > > > > > >> > >> > > Andrii
> > > > > > >> > >> > >
> > > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
> > > > > > jay.kreps@gmail.com
> > > > > > >> >
> > > > > > >> > >> wrote:
> > > > > > >> > >> > >
> > > > > > >> > >> > > > Hey Andrii,
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > To answer your earlier question we just really
> can't
> > be
> > > > > > adding
> > > > > > >> any
> > > > > > >> > >> more
> > > > > > >> > >> > > > scala protocol objects. These things are super hard
> > to
> > > > > > maintain
> > > > > > >> > >> because
> > > > > > >> > >> > > > they hand code the byte parsing and don't have good
> > > > > > versioning
> > > > > > >> > >> support.
> > > > > > >> > >> > > > Since we are already planning on converting we
> > > definitely
> > > > > > don't
> > > > > > >> > >> want to
> > > > > > >> > >> > > add
> > > > > > >> > >> > > > a ton more of these--they are total tech debt.
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > What does it mean that the changes are isolated
> from
> > > the
> > > > > > >> current
> > > > > > >> > >> code
> > > > > > >> > >> > > base?
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > I actually didn't understand the remaining
> comments,
> > > > which
> > > > > of
> > > > > > >> the
> > > > > > >> > >> > points
> > > > > > >> > >> > > > are you responding to?
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > Maybe one sticking point here is that it seems like
> > you
> > > > > want
> > > > > > to
> > > > > > >> > make
> > > > > > >> > >> > some
> > > > > > >> > >> > > > kind of tool, and you have made a 1-1 mapping
> between
> > > > > > commands
> > > > > > >> you
> > > > > > >> > >> > > imagine
> > > > > > >> > >> > > > in the tool and protocol additions. I want to make
> > sure
> > > > we
> > > > > > >> don't
> > > > > > >> > do
> > > > > > >> > >> > that.
> > > > > > >> > >> > > > The protocol needs to be really really well thought
> > out
> > > > > > against
> > > > > > >> > many
> > > > > > >> > >> > use
> > > > > > >> > >> > > > cases so it should make perfect logical sense in
> the
> > > > > absence
> > > > > > of
> > > > > > >> > >> knowing
> > > > > > >> > >> > > the
> > > > > > >> > >> > > > command line tool, right?
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > -Jay
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi
> <
> > > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > > Hey Jay,
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > > > I would like to continue this discussion as it
> seem
> > > > there
> > > > > > is
> > > > > > >> no
> > > > > > >> > >> > > progress
> > > > > > >> > >> > > > > here.
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > > > First of all, could you please explain what did
> you
> > > > mean
> > > > > in
> > > > > > >> 2?
> > > > > > >> > How
> > > > > > >> > >> > > > exactly
> > > > > > >> > >> > > > > are we going to migrate to the new java protocol
> > > > > > definitions.
> > > > > > >> > And
> > > > > > >> > >> why
> > > > > > >> > >> > > > it's
> > > > > > >> > >> > > > > a blocker for centralized CLI?
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > > > I agree with you, this feature includes lots of
> > > stuff,
> > > > > but
> > > > > > >> > >> thankfully
> > > > > > >> > >> > > > > almost all changes are isolated from the current
> > code
> > > > > base,
> > > > > > >> > >> > > > > so the main thing, I think, we need to agree is
> > RQ/RP
> > > > > > format.
> > > > > > >> > >> > > > > So how can we start discussion about the concrete
> > > > > messages
> > > > > > >> > format?
> > > > > > >> > >> > > > > Can we take (
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > >
> > > > > > >> > >> > >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > > > > >> > >> > > > > )
> > > > > > >> > >> > > > > as starting point?
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > > > We had some doubts earlier whether it worth
> > > introducing
> > > > > one
> > > > > > >> > >> generic
> > > > > > >> > >> > > Admin
> > > > > > >> > >> > > > > Request for all commands (
> > > > > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > > >> > >> > > > > )
> > > > > > >> > >> > > > > but then everybody agreed it would be better to
> > have
> > > > > > separate
> > > > > > >> > >> message
> > > > > > >> > >> > > for
> > > > > > >> > >> > > > > each admin command. The Request part is really
> > > dictated
> > > > > > from
> > > > > > >> the
> > > > > > >> > >> > > command
> > > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so the
> > proposed
> > > > > > version
> > > > > > >> > >> should
> > > > > > >> > >> > be
> > > > > > >> > >> > > > > fine (let's put aside for now remarks about
> > Optional
> > > > > type,
> > > > > > >> > >> batching,
> > > > > > >> > >> > > > > configs normalization - I agree with all of
> them).
> > > > > > >> > >> > > > > So the second part is Response. I see there are
> two
> > > > cases
> > > > > > >> here.
> > > > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b)
> "Get"
> > > > > > requests -
> > > > > > >> > >> > > > > List/Describe...
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > > > a) should only hold request result (regardless
> what
> > > we
> > > > > > decide
> > > > > > >> > >> about
> > > > > > >> > >> > > > > blocking/non-blocking commands execution).
> > > > > > >> > >> > > > > Usually we provide error code in response but
> since
> > > we
> > > > > will
> > > > > > >> use
> > > > > > >> > >> this
> > > > > > >> > >> > in
> > > > > > >> > >> > > > > interactive shell we need some human readable
> error
> > > > > > >> description
> > > > > > >> > -
> > > > > > >> > >> so
> > > > > > >> > >> > I
> > > > > > >> > >> > > > > added errorDesription field where you can at
> least
> > > > leave
> > > > > > >> > >> > > > > exception.getMessage.
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > > > b) in addition to previous item message should
> hold
> > > > > command
> > > > > > >> > >> specific
> > > > > > >> > >> > > > > response data. We can discuss in detail each of
> > them
> > > > but
> > > > > > >> let's
> > > > > > >> > for
> > > > > > >> > >> > now
> > > > > > >> > >> > > > > agree about the overall pattern.
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > > > Thanks,
> > > > > > >> > >> > > > > Andrii Biletskyi
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
> > > > > > >> jay.kreps@gmail.com
> > > > > > >> > >
> > > > > > >> > >> > > wrote:
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > > > > Hey Joe,
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > This is great. A few comments on KIP-4
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > 1. This is much needed functionality, but there
> > > are a
> > > > > lot
> > > > > > >> of
> > > > > > >> > >> the so
> > > > > > >> > >> > > > let's
> > > > > > >> > >> > > > > > really think these protocols through. We really
> > > want
> > > > to
> > > > > > >> end up
> > > > > > >> > >> > with a
> > > > > > >> > >> > > > set
> > > > > > >> > >> > > > > > of well thought-out, orthoganol apis. For this
> > > > reason I
> > > > > > >> think
> > > > > > >> > >> it is
> > > > > > >> > >> > > > > really
> > > > > > >> > >> > > > > > important to think through the end state even
> if
> > > that
> > > > > > >> includes
> > > > > > >> > >> APIs
> > > > > > >> > >> > > we
> > > > > > >> > >> > > > > > won't implement in the first phase.
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > 2. Let's please please please wait until we
> have
> > > > > switched
> > > > > > >> the
> > > > > > >> > >> > server
> > > > > > >> > >> > > > over
> > > > > > >> > >> > > > > > to the new java protocol definitions. If we add
> > > > upteen
> > > > > > >> more ad
> > > > > > >> > >> hoc
> > > > > > >> > >> > > > scala
> > > > > > >> > >> > > > > > objects that is just generating more work for
> the
> > > > > > >> conversion
> > > > > > >> > we
> > > > > > >> > >> > know
> > > > > > >> > >> > > we
> > > > > > >> > >> > > > > > have to do.
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > 3. This proposal introduces a new type of
> > optional
> > > > > > >> parameter.
> > > > > > >> > >> This
> > > > > > >> > >> > is
> > > > > > >> > >> > > > > > inconsistent with everything else in the
> protocol
> > > > where
> > > > > > we
> > > > > > >> use
> > > > > > >> > >> -1
> > > > > > >> > >> > or
> > > > > > >> > >> > > > some
> > > > > > >> > >> > > > > > other marker value. You could argue either way
> > but
> > > > > let's
> > > > > > >> stick
> > > > > > >> > >> with
> > > > > > >> > >> > > > that
> > > > > > >> > >> > > > > > for consistency. For clients that implemented
> the
> > > > > > protocol
> > > > > > >> in
> > > > > > >> > a
> > > > > > >> > >> > > better
> > > > > > >> > >> > > > > way
> > > > > > >> > >> > > > > > than our scala code these basic primitives are
> > hard
> > > > to
> > > > > > >> change.
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to duplicate
> > > > > > >> > TopicMetadataRequest
> > > > > > >> > >> > > which
> > > > > > >> > >> > > > > has
> > > > > > >> > >> > > > > > brokers, topics, and partitions. I think we
> > should
> > > > > rename
> > > > > > >> that
> > > > > > >> > >> > > request
> > > > > > >> > >> > > > > > ClusterMetadataRequest (or just
> MetadataRequest)
> > > and
> > > > > > >> include
> > > > > > >> > >> the id
> > > > > > >> > >> > > of
> > > > > > >> > >> > > > > the
> > > > > > >> > >> > > > > > controller. Or are there other things we could
> > add
> > > > > here?
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > 5. We have a tendency to try to make a lot of
> > > > requests
> > > > > > that
> > > > > > >> > can
> > > > > > >> > >> > only
> > > > > > >> > >> > > go
> > > > > > >> > >> > > > > to
> > > > > > >> > >> > > > > > particular nodes. This adds a lot of burden for
> > > > client
> > > > > > >> > >> > > implementations
> > > > > > >> > >> > > > > (it
> > > > > > >> > >> > > > > > sounds easy but each discovery can fail in many
> > > parts
> > > > > so
> > > > > > it
> > > > > > >> > >> ends up
> > > > > > >> > >> > > > > being a
> > > > > > >> > >> > > > > > full state machine to do right). I think we
> > should
> > > > > > consider
> > > > > > >> > >> making
> > > > > > >> > >> > > > admin
> > > > > > >> > >> > > > > > commands and ideally as many of the other apis
> as
> > > > > > possible
> > > > > > >> > >> > available
> > > > > > >> > >> > > on
> > > > > > >> > >> > > > > all
> > > > > > >> > >> > > > > > brokers and just redirect to the controller on
> > the
> > > > > broker
> > > > > > >> > side.
> > > > > > >> > >> > > Perhaps
> > > > > > >> > >> > > > > > there would be a general way to encapsulate
> this
> > > > > > re-routing
> > > > > > >> > >> > behavior.
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > 6. We should probably normalize the key value
> > pairs
> > > > > used
> > > > > > >> for
> > > > > > >> > >> > configs
> > > > > > >> > >> > > > > rather
> > > > > > >> > >> > > > > > than embedding a new formatting. So two strings
> > > > rather
> > > > > > than
> > > > > > >> > one
> > > > > > >> > >> > with
> > > > > > >> > >> > > an
> > > > > > >> > >> > > > > > internal equals sign.
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > 7. Is the postcondition of these APIs that the
> > > > command
> > > > > > has
> > > > > > >> > >> begun or
> > > > > > >> > >> > > > that
> > > > > > >> > >> > > > > > the command has been completed? It is a lot
> more
> > > > usable
> > > > > > if
> > > > > > >> the
> > > > > > >> > >> > > command
> > > > > > >> > >> > > > > has
> > > > > > >> > >> > > > > > been completed so you know that if you create a
> > > topic
> > > > > and
> > > > > > >> then
> > > > > > >> > >> > > publish
> > > > > > >> > >> > > > to
> > > > > > >> > >> > > > > > it you won't get an exception about there being
> > no
> > > > such
> > > > > > >> topic.
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > 8. Describe topic and list topics duplicate a
> lot
> > > of
> > > > > > stuff
> > > > > > >> in
> > > > > > >> > >> the
> > > > > > >> > >> > > > > metadata
> > > > > > >> > >> > > > > > request. Is there a reason to give back topics
> > > marked
> > > > > for
> > > > > > >> > >> > deletion? I
> > > > > > >> > >> > > > > feel
> > > > > > >> > >> > > > > > like if we just make the post-condition of the
> > > delete
> > > > > > >> command
> > > > > > >> > be
> > > > > > >> > >> > that
> > > > > > >> > >> > > > the
> > > > > > >> > >> > > > > > topic is deleted that will get rid of the need
> > for
> > > > this
> > > > > > >> right?
> > > > > > >> > >> And
> > > > > > >> > >> > it
> > > > > > >> > >> > > > > will
> > > > > > >> > >> > > > > > be much more intuitive.
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > 9. Should we consider batching these requests?
> We
> > > > have
> > > > > > >> > generally
> > > > > > >> > >> > > tried
> > > > > > >> > >> > > > to
> > > > > > >> > >> > > > > > allow multiple operations to be batched. My
> > > suspicion
> > > > > is
> > > > > > >> that
> > > > > > >> > >> > without
> > > > > > >> > >> > > > > this
> > > > > > >> > >> > > > > > we will get a lot of code that does something
> > like
> > > > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
> > > > > > >> > >> > > > > >       adminClient.describeTopic(topic)
> > > > > > >> > >> > > > > > this code will work great when you test on 5
> > topics
> > > > but
> > > > > > >> not do
> > > > > > >> > >> as
> > > > > > >> > >> > > well
> > > > > > >> > >> > > > if
> > > > > > >> > >> > > > > > you have 50k.
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > 10. I think we should also discuss how we want
> to
> > > > > expose
> > > > > > a
> > > > > > >> > >> > > programmatic
> > > > > > >> > >> > > > > JVM
> > > > > > >> > >> > > > > > client api for these operations. Currently
> people
> > > > rely
> > > > > on
> > > > > > >> > >> > AdminUtils
> > > > > > >> > >> > > > > which
> > > > > > >> > >> > > > > > is totally sketchy. I think we probably need
> > > another
> > > > > > client
> > > > > > >> > >> under
> > > > > > >> > >> > > > > clients/
> > > > > > >> > >> > > > > > that exposes administrative functionality. We
> > will
> > > > need
> > > > > > >> this
> > > > > > >> > >> just
> > > > > > >> > >> > to
> > > > > > >> > >> > > > > > properly test the new apis, I suspect. We
> should
> > > > figure
> > > > > > out
> > > > > > >> > that
> > > > > > >> > >> > API.
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > 11. The other information that would be really
> > > useful
> > > > > to
> > > > > > >> get
> > > > > > >> > >> would
> > > > > > >> > >> > be
> > > > > > >> > >> > > > > > information about partitions--how much data is
> in
> > > the
> > > > > > >> > partition,
> > > > > > >> > >> > what
> > > > > > >> > >> > > > are
> > > > > > >> > >> > > > > > the segment offsets, what is the log-end offset
> > > (i.e.
> > > > > > last
> > > > > > >> > >> offset),
> > > > > > >> > >> > > > what
> > > > > > >> > >> > > > > is
> > > > > > >> > >> > > > > > the compaction point, etc. I think that done
> > right
> > > > this
> > > > > > >> would
> > > > > > >> > be
> > > > > > >> > >> > the
> > > > > > >> > >> > > > > > successor to the very awkward OffsetRequest we
> > have
> > > > > > today.
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > -Jay
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
> > > > > > >> > >> joe.stein@stealth.ly>
> > > > > > >> > >> > > > > wrote:
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > > > > Hi, created a KIP
> > > > > > >> > >> > > > > > >
> > > > > > >> > >> > > > > > >
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > >
> > > > > > >> > >> > >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > >> > >> > > > > > >
> > > > > > >> > >> > > > > > > JIRA
> > > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > > >> > >> > > > > > >
> > > > > > >> > >> > > > > > > /*******************************************
> > > > > > >> > >> > > > > > >  Joe Stein
> > > > > > >> > >> > > > > > >  Founder, Principal Consultant
> > > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
> > > > > > >> > >> > > > > > >  http://www.stealth.ly
> > > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> > > > > > >> > >> > http://www.twitter.com/allthingshadoop
> > > > > > >> > >> > > >
> > > > > > >> > >> > > > > > > ********************************************/
> > > > > > >> > >> > > > > > >
> > > > > > >> > >> > > > > >
> > > > > > >> > >> > > > >
> > > > > > >> > >> > > >
> > > > > > >> > >> > >
> > > > > > >> > >> >
> > > > > > >> > >>
> > > > > > >> > >>
> > > > > > >> > >>
> > > > > > >> > >> --
> > > > > > >> > >> -- Guozhang
> > > > > > >> > >>
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > Jeff Holoman
> > Systems Engineer
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
Let's stay on Google hangouts that will also record and make the sessions
available on youtube.

-Jay

On Tue, Feb 24, 2015 at 11:49 AM, Jeff Holoman <jh...@cloudera.com>
wrote:

> Jay / Joe
>
> We're happy to send out a Webex for this purpose. We could record the
> sessions if there is interest and publish them out.
>
> Thanks
>
> Jeff
>
> On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Let's try to get the technical hang-ups sorted out, though. I really
> think
> > there is some benefit to live discussion vs writing. I am hopeful that if
> > we post instructions and give ourselves a few attempts we can get it
> > working.
> >
> > Tuesday at that time would work for me...any objections?
> >
> > -Jay
> >
> > On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <jo...@stealth.ly> wrote:
> >
> > > Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am PT ????
> > >
> > > I don't mind google hangout but there is always some issue or whatever
> so
> > > we know the apache irc channel works. We can start there and see how it
> > > goes? We can pull transcripts too and associate to tickets if need be
> > makes
> > > it helpful for things.
> > >
> > > ~ Joestein
> > >
> > > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > We'd talked about doing a Google Hangout to chat about this. What
> about
> > > > generalizing that a little further...I actually think it would be
> good
> > > for
> > > > everyone spending a reasonable chunk of their week on Kafka stuff to
> > > maybe
> > > > sync up once a week. I think we could use time to talk through design
> > > > stuff, make sure we are on top of code reviews, talk through any
> tricky
> > > > issues, etc.
> > > >
> > > > We can make it publicly available so that any one can follow along
> who
> > > > likes.
> > > >
> > > > Any interest in doing this? If so I'll try to set it up starting next
> > > week.
> > > >
> > > > -Jay
> > > >
> > > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > I've updated KIP page, fixed / aligned document structure. Also I
> > added
> > > > > some
> > > > > very initial proposal for AdminClient so we have something to start
> > > from
> > > > > while
> > > > > discussing the KIP.
> > > > >
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> > > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >
> > > > > > Jay,
> > > > > >
> > > > > > Re error messages: you are right, in most cases client will have
> > > enough
> > > > > > context to show descriptive error message. My concern is that we
> > will
> > > > > have
> > > > > > to
> > > > > > add lots of new error codes for each possible error. Of course,
> we
> > > > could
> > > > > > reuse
> > > > > > some of existing like UknownTopicOrPartitionCode, but we will
> also
> > > need
> > > > > to
> > > > > > add smth like: TopicAlreadyExistsCode, TopicConfigInvalid (both
> for
> > > > topic
> > > > > > name and config, and probably user would like to know what
> exactly
> > > > > > is wrong in his config), InvalidReplicaAssignment, InternalError
> > > (e.g.
> > > > > > zookeeper failure) etc.
> > > > > > And this is only for TopicCommand, we will also need to add
> similar
> > > > stuff
> > > > > > for
> > > > > > ReassignPartitions, PreferredReplica. So we'll end up with a
> large
> > > list
> > > > > of
> > > > > > error codes, used only in Admin protocol.
> > > > > > Having said that, I agree my proposal is not consistent with
> other
> > > > cases.
> > > > > > Maybe we can find better solution or something in-between.
> > > > > >
> > > > > > Re Hangout chat: I think it is a great idea. This way we can move
> > on
> > > > > > faster.
> > > > > > Let's agree somehow on date/time so people can join. Will work
> for
> > me
> > > > > this
> > > > > > and
> > > > > > next week almost anytime if agreed in advance.
> > > > > >
> > > > > > Thanks,
> > > > > > Andrii
> > > > > >
> > > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <ja...@gmail.com>
> > > > wrote:
> > > > > >
> > > > > >> Hey Andrii,
> > > > > >>
> > > > > >> Generally we can do good error handling without needing custom
> > > > > server-side
> > > > > >> messages. I.e. generally the client has the context to know that
> > if
> > > it
> > > > > got
> > > > > >> an error that the topic doesn't exist to say "Topic X doesn't
> > exist"
> > > > > >> rather
> > > > > >> than "error code 14" (or whatever). Maybe there are specific
> cases
> > > > where
> > > > > >> this is hard? If we want to add server-side error messages we
> > really
> > > > do
> > > > > >> need to do this in a consistent way across the protocol.
> > > > > >>
> > > > > >> I still have a bunch of open questions here from my previous
> > list. I
> > > > > will
> > > > > >> be out for the next few days for Strata though. Maybe we could
> do
> > a
> > > > > Google
> > > > > >> Hangout chat on any open issues some time towards the end of
> next
> > > week
> > > > > for
> > > > > >> anyone interested in this ticket? I have a feeling that might
> > > progress
> > > > > >> things a little faster than email--I think we could talk through
> > > those
> > > > > >> issues I brought up fairly quickly...
> > > > > >>
> > > > > >> -Jay
> > > > > >>
> > > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> > > > > >> andrii.biletskyi@stealth.ly> wrote:
> > > > > >>
> > > > > >> > Hi all,
> > > > > >> >
> > > > > >> > I'm trying to address some of the issues which were mentioned
> > > > earlier
> > > > > >> about
> > > > > >> > Admin RQ/RP format. One of those was about batching
> operations.
> > > What
> > > > > if
> > > > > >> we
> > > > > >> > follow TopicCommand approach and let people specify topic-name
> > by
> > > > > >> regexp -
> > > > > >> > would that cover most of the use cases?
> > > > > >> >
> > > > > >> > Secondly, is what information should we generally provide in
> > Admin
> > > > > >> > responses.
> > > > > >> > I realize that Admin commands don't imply they will be used
> only
> > > in
> > > > > CLI
> > > > > >> > but,
> > > > > >> > it seems to me, CLI is a very important client of this
> feature.
> > In
> > > > > this
> > > > > >> > case,
> > > > > >> > seems logical, we would like to provide users with rich
> > experience
> > > > in
> > > > > >> terms
> > > > > >> > of
> > > > > >> > getting results / errors of the executed commands. Usually we
> > > supply
> > > > > >> with
> > > > > >> > responses only errorCode, which looks very limiting, in case
> of
> > > CLI
> > > > we
> > > > > >> may
> > > > > >> > want to print human readable error description.
> > > > > >> >
> > > > > >> > So, taking into account previous item about batching, what do
> > you
> > > > > think
> > > > > >> > about
> > > > > >> > having smth like:
> > > > > >> >
> > > > > >> > ('create' doesn't support regexp)
> > > > > >> > CreateTopicRequest => TopicName Partitions Replicas
> > > > ReplicaAssignment
> > > > > >> > [Config]
> > > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
> > > > > >> >   ErrorCode => int16
> > > > > >> >   ErrorDescription => string (empty if successful)
> > > > > >> >
> > > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> > ReplicaAssignment
> > > > > >> > [AddedConfig] [DeletedConfig]
> > > > > >> > AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
> > > > > >> > CommandErrorCode CommandErrorDescription
> > > > > >> >   CommandErrorCode => int16
> > > > > >> >   CommandErrorDescription => string (nonempty in case of fatal
> > > > error,
> > > > > >> e.g.
> > > > > >> > we couldn't get topics by regexp)
> > > > > >> >
> > > > > >> > DescribeTopicRequest -> TopicNameRegexp
> > > > > >> > DescribeTopicResponse -> [TopicName TopicDescription ErrorCode
> > > > > >> > ErrorDescription] CommandErrorCode CommandErrorDescription
> > > > > >> >
> > > > > >> > Also, any thoughts about our discussion regarding re-routing
> > > > facility?
> > > > > >> In
> > > > > >> > my
> > > > > >> > understanding, it is like between augmenting
> > TopicMetadataRequest
> > > > > >> > (to include at least controllerId) and implementing new
> generic
> > > > > >> re-routing
> > > > > >> > facility so sending messages to controller will be handled by
> > it.
> > > > > >> >
> > > > > >> > Thanks,
> > > > > >> > Andrii Biletskyi
> > > > > >> >
> > > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> > > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > > >> >
> > > > > >> > > @Guozhang:
> > > > > >> > > Thanks for your comments, I've answered some of those. The
> > main
> > > > > thing
> > > > > >> is
> > > > > >> > > having merged request for create-alter-delete-describe - I
> > have
> > > > some
> > > > > >> > > concerns about this approach.
> > > > > >> > >
> > > > > >> > > @*Jay*:
> > > > > >> > > I see that introduced ClusterMetadaRequest is also one of
> the
> > > > > >> concerns.
> > > > > >> > We
> > > > > >> > > can solve it if we implement re-routing facility. But I
> agree
> > > with
> > > > > >> > > Guozhang - it will make clients' internals a little bit
> easier
> > > but
> > > > > >> this
> > > > > >> > > seems to be a complex logic to implement and support then.
> > > > > Especially
> > > > > >> for
> > > > > >> > > Fetch and Produce (even if we add re-routing later for these
> > > > > >> requests).
> > > > > >> > > Also people will tend to avoid this re-routing facility and
> > hold
> > > > > local
> > > > > >> > > cluster cache to ensure their high-priority requests (which
> > some
> > > > of
> > > > > >> the
> > > > > >> > > admin requests are) not sent to some busy broker where they
> > wait
> > > > to
> > > > > be
> > > > > >> > > routed to the correct one.
> > > > > >> > > As pointed out by Jun here (
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > > > >> > )
> > > > > >> > > to solve the issue we might introduce a message type to get
> > > > cluster
> > > > > >> > state.
> > > > > >> > > But I agree we can just update TopicMetadataResponse to
> > include
> > > > > >> > > controllerId (and probably smth else).
> > > > > >> > > What are you thougths?
> > > > > >> > >
> > > > > >> > > Thanks,
> > > > > >> > > Andrii
> > > > > >> > >
> > > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > > >> > wrote:
> > > > > >> > >
> > > > > >> > >> I think for the topics commands we can actually merge
> > > > > >> > >> create/alter/delete/describe as one request type since
> their
> > > > > formats
> > > > > >> are
> > > > > >> > >> very much similar, and keep list-topics and others like
> > > > > >> > >> partition-reassignment / preferred-leader-election as
> > separate
> > > > > >> request
> > > > > >> > >> types, I also left some other comments on the RB (
> > > > > >> > >> https://reviews.apache.org/r/29301/).
> > > > > >> > >>
> > > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> > > jay.kreps@gmail.com>
> > > > > >> wrote:
> > > > > >> > >>
> > > > > >> > >> > Yeah I totally agree that we don't want to just have one
> > "do
> > > > > admin
> > > > > >> > >> stuff"
> > > > > >> > >> > command that has the union of all parameters.
> > > > > >> > >> >
> > > > > >> > >> > What I am saying is that command line tools are one
> client
> > of
> > > > the
> > > > > >> > >> > administrative apis, but these will be used in a number
> of
> > > > > >> scenarios
> > > > > >> > so
> > > > > >> > >> > they should make logical sense even in the absence of the
> > > > command
> > > > > >> line
> > > > > >> > >> > tool. Hence comments like trying to clarify the
> > relationship
> > > > > >> between
> > > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds of things
> > > > really
> > > > > >> need
> > > > > >> > >> to be
> > > > > >> > >> > thought through.
> > > > > >> > >> >
> > > > > >> > >> > Hope that makes sense.
> > > > > >> > >> >
> > > > > >> > >> > -Jay
> > > > > >> > >> >
> > > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> > > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > > >> > >> >
> > > > > >> > >> > > Jay,
> > > > > >> > >> > >
> > > > > >> > >> > > Thanks for answering. You understood correctly, most of
> > my
> > > > > >> comments
> > > > > >> > >> were
> > > > > >> > >> > > related to your point 1) - about "well thought-out"
> apis.
> > > > Also,
> > > > > >> yes,
> > > > > >> > >> as I
> > > > > >> > >> > > understood we would like to introduce a single unified
> > CLI
> > > > tool
> > > > > >> with
> > > > > >> > >> > > centralized server-side request handling for lots of
> > > existing
> > > > > >> ones
> > > > > >> > >> (incl.
> > > > > >> > >> > > TopicCommand, CommitOffsetChecker, ReassignPartitions,
> > smth
> > > > > else
> > > > > >> if
> > > > > >> > >> added
> > > > > >> > >> > > in future). In our previous discussion (
> > > > > >> > >> > > https://issues.apache.org/jira/browse/KAFKA-1694)
> people
> > > > said
> > > > > >> > they'd
> > > > > >> > >> > > rather
> > > > > >> > >> > > have a separate message for each command, so, yes, this
> > > way I
> > > > > >> came
> > > > > >> > to
> > > > > >> > >> 1-1
> > > > > >> > >> > > mapping between commands in the tool and protocol
> > > additions.
> > > > > But
> > > > > >> I
> > > > > >> > >> might
> > > > > >> > >> > be
> > > > > >> > >> > > wrong.
> > > > > >> > >> > > At the end I just try to start discussion how at least
> > > > > generally
> > > > > >> > this
> > > > > >> > >> > > protocol should look like.
> > > > > >> > >> > >
> > > > > >> > >> > > Thanks,
> > > > > >> > >> > > Andrii
> > > > > >> > >> > >
> > > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
> > > > > jay.kreps@gmail.com
> > > > > >> >
> > > > > >> > >> wrote:
> > > > > >> > >> > >
> > > > > >> > >> > > > Hey Andrii,
> > > > > >> > >> > > >
> > > > > >> > >> > > > To answer your earlier question we just really can't
> be
> > > > > adding
> > > > > >> any
> > > > > >> > >> more
> > > > > >> > >> > > > scala protocol objects. These things are super hard
> to
> > > > > maintain
> > > > > >> > >> because
> > > > > >> > >> > > > they hand code the byte parsing and don't have good
> > > > > versioning
> > > > > >> > >> support.
> > > > > >> > >> > > > Since we are already planning on converting we
> > definitely
> > > > > don't
> > > > > >> > >> want to
> > > > > >> > >> > > add
> > > > > >> > >> > > > a ton more of these--they are total tech debt.
> > > > > >> > >> > > >
> > > > > >> > >> > > > What does it mean that the changes are isolated from
> > the
> > > > > >> current
> > > > > >> > >> code
> > > > > >> > >> > > base?
> > > > > >> > >> > > >
> > > > > >> > >> > > > I actually didn't understand the remaining comments,
> > > which
> > > > of
> > > > > >> the
> > > > > >> > >> > points
> > > > > >> > >> > > > are you responding to?
> > > > > >> > >> > > >
> > > > > >> > >> > > > Maybe one sticking point here is that it seems like
> you
> > > > want
> > > > > to
> > > > > >> > make
> > > > > >> > >> > some
> > > > > >> > >> > > > kind of tool, and you have made a 1-1 mapping between
> > > > > commands
> > > > > >> you
> > > > > >> > >> > > imagine
> > > > > >> > >> > > > in the tool and protocol additions. I want to make
> sure
> > > we
> > > > > >> don't
> > > > > >> > do
> > > > > >> > >> > that.
> > > > > >> > >> > > > The protocol needs to be really really well thought
> out
> > > > > against
> > > > > >> > many
> > > > > >> > >> > use
> > > > > >> > >> > > > cases so it should make perfect logical sense in the
> > > > absence
> > > > > of
> > > > > >> > >> knowing
> > > > > >> > >> > > the
> > > > > >> > >> > > > command line tool, right?
> > > > > >> > >> > > >
> > > > > >> > >> > > > -Jay
> > > > > >> > >> > > >
> > > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
> > > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> > > > > >> > >> > > >
> > > > > >> > >> > > > > Hey Jay,
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > I would like to continue this discussion as it seem
> > > there
> > > > > is
> > > > > >> no
> > > > > >> > >> > > progress
> > > > > >> > >> > > > > here.
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > First of all, could you please explain what did you
> > > mean
> > > > in
> > > > > >> 2?
> > > > > >> > How
> > > > > >> > >> > > > exactly
> > > > > >> > >> > > > > are we going to migrate to the new java protocol
> > > > > definitions.
> > > > > >> > And
> > > > > >> > >> why
> > > > > >> > >> > > > it's
> > > > > >> > >> > > > > a blocker for centralized CLI?
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > I agree with you, this feature includes lots of
> > stuff,
> > > > but
> > > > > >> > >> thankfully
> > > > > >> > >> > > > > almost all changes are isolated from the current
> code
> > > > base,
> > > > > >> > >> > > > > so the main thing, I think, we need to agree is
> RQ/RP
> > > > > format.
> > > > > >> > >> > > > > So how can we start discussion about the concrete
> > > > messages
> > > > > >> > format?
> > > > > >> > >> > > > > Can we take (
> > > > > >> > >> > > > >
> > > > > >> > >> > > > >
> > > > > >> > >> > > >
> > > > > >> > >> > >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > > > >> > >> > > > > )
> > > > > >> > >> > > > > as starting point?
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > We had some doubts earlier whether it worth
> > introducing
> > > > one
> > > > > >> > >> generic
> > > > > >> > >> > > Admin
> > > > > >> > >> > > > > Request for all commands (
> > > > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > >> > >> > > > > )
> > > > > >> > >> > > > > but then everybody agreed it would be better to
> have
> > > > > separate
> > > > > >> > >> message
> > > > > >> > >> > > for
> > > > > >> > >> > > > > each admin command. The Request part is really
> > dictated
> > > > > from
> > > > > >> the
> > > > > >> > >> > > command
> > > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so the
> proposed
> > > > > version
> > > > > >> > >> should
> > > > > >> > >> > be
> > > > > >> > >> > > > > fine (let's put aside for now remarks about
> Optional
> > > > type,
> > > > > >> > >> batching,
> > > > > >> > >> > > > > configs normalization - I agree with all of them).
> > > > > >> > >> > > > > So the second part is Response. I see there are two
> > > cases
> > > > > >> here.
> > > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b) "Get"
> > > > > requests -
> > > > > >> > >> > > > > List/Describe...
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > a) should only hold request result (regardless what
> > we
> > > > > decide
> > > > > >> > >> about
> > > > > >> > >> > > > > blocking/non-blocking commands execution).
> > > > > >> > >> > > > > Usually we provide error code in response but since
> > we
> > > > will
> > > > > >> use
> > > > > >> > >> this
> > > > > >> > >> > in
> > > > > >> > >> > > > > interactive shell we need some human readable error
> > > > > >> description
> > > > > >> > -
> > > > > >> > >> so
> > > > > >> > >> > I
> > > > > >> > >> > > > > added errorDesription field where you can at least
> > > leave
> > > > > >> > >> > > > > exception.getMessage.
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > b) in addition to previous item message should hold
> > > > command
> > > > > >> > >> specific
> > > > > >> > >> > > > > response data. We can discuss in detail each of
> them
> > > but
> > > > > >> let's
> > > > > >> > for
> > > > > >> > >> > now
> > > > > >> > >> > > > > agree about the overall pattern.
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > Thanks,
> > > > > >> > >> > > > > Andrii Biletskyi
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
> > > > > >> jay.kreps@gmail.com
> > > > > >> > >
> > > > > >> > >> > > wrote:
> > > > > >> > >> > > > >
> > > > > >> > >> > > > > > Hey Joe,
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > This is great. A few comments on KIP-4
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > 1. This is much needed functionality, but there
> > are a
> > > > lot
> > > > > >> of
> > > > > >> > >> the so
> > > > > >> > >> > > > let's
> > > > > >> > >> > > > > > really think these protocols through. We really
> > want
> > > to
> > > > > >> end up
> > > > > >> > >> > with a
> > > > > >> > >> > > > set
> > > > > >> > >> > > > > > of well thought-out, orthoganol apis. For this
> > > reason I
> > > > > >> think
> > > > > >> > >> it is
> > > > > >> > >> > > > > really
> > > > > >> > >> > > > > > important to think through the end state even if
> > that
> > > > > >> includes
> > > > > >> > >> APIs
> > > > > >> > >> > > we
> > > > > >> > >> > > > > > won't implement in the first phase.
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > 2. Let's please please please wait until we have
> > > > switched
> > > > > >> the
> > > > > >> > >> > server
> > > > > >> > >> > > > over
> > > > > >> > >> > > > > > to the new java protocol definitions. If we add
> > > upteen
> > > > > >> more ad
> > > > > >> > >> hoc
> > > > > >> > >> > > > scala
> > > > > >> > >> > > > > > objects that is just generating more work for the
> > > > > >> conversion
> > > > > >> > we
> > > > > >> > >> > know
> > > > > >> > >> > > we
> > > > > >> > >> > > > > > have to do.
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > 3. This proposal introduces a new type of
> optional
> > > > > >> parameter.
> > > > > >> > >> This
> > > > > >> > >> > is
> > > > > >> > >> > > > > > inconsistent with everything else in the protocol
> > > where
> > > > > we
> > > > > >> use
> > > > > >> > >> -1
> > > > > >> > >> > or
> > > > > >> > >> > > > some
> > > > > >> > >> > > > > > other marker value. You could argue either way
> but
> > > > let's
> > > > > >> stick
> > > > > >> > >> with
> > > > > >> > >> > > > that
> > > > > >> > >> > > > > > for consistency. For clients that implemented the
> > > > > protocol
> > > > > >> in
> > > > > >> > a
> > > > > >> > >> > > better
> > > > > >> > >> > > > > way
> > > > > >> > >> > > > > > than our scala code these basic primitives are
> hard
> > > to
> > > > > >> change.
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > 4. ClusterMetadata: This seems to duplicate
> > > > > >> > TopicMetadataRequest
> > > > > >> > >> > > which
> > > > > >> > >> > > > > has
> > > > > >> > >> > > > > > brokers, topics, and partitions. I think we
> should
> > > > rename
> > > > > >> that
> > > > > >> > >> > > request
> > > > > >> > >> > > > > > ClusterMetadataRequest (or just MetadataRequest)
> > and
> > > > > >> include
> > > > > >> > >> the id
> > > > > >> > >> > > of
> > > > > >> > >> > > > > the
> > > > > >> > >> > > > > > controller. Or are there other things we could
> add
> > > > here?
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > 5. We have a tendency to try to make a lot of
> > > requests
> > > > > that
> > > > > >> > can
> > > > > >> > >> > only
> > > > > >> > >> > > go
> > > > > >> > >> > > > > to
> > > > > >> > >> > > > > > particular nodes. This adds a lot of burden for
> > > client
> > > > > >> > >> > > implementations
> > > > > >> > >> > > > > (it
> > > > > >> > >> > > > > > sounds easy but each discovery can fail in many
> > parts
> > > > so
> > > > > it
> > > > > >> > >> ends up
> > > > > >> > >> > > > > being a
> > > > > >> > >> > > > > > full state machine to do right). I think we
> should
> > > > > consider
> > > > > >> > >> making
> > > > > >> > >> > > > admin
> > > > > >> > >> > > > > > commands and ideally as many of the other apis as
> > > > > possible
> > > > > >> > >> > available
> > > > > >> > >> > > on
> > > > > >> > >> > > > > all
> > > > > >> > >> > > > > > brokers and just redirect to the controller on
> the
> > > > broker
> > > > > >> > side.
> > > > > >> > >> > > Perhaps
> > > > > >> > >> > > > > > there would be a general way to encapsulate this
> > > > > re-routing
> > > > > >> > >> > behavior.
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > 6. We should probably normalize the key value
> pairs
> > > > used
> > > > > >> for
> > > > > >> > >> > configs
> > > > > >> > >> > > > > rather
> > > > > >> > >> > > > > > than embedding a new formatting. So two strings
> > > rather
> > > > > than
> > > > > >> > one
> > > > > >> > >> > with
> > > > > >> > >> > > an
> > > > > >> > >> > > > > > internal equals sign.
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > 7. Is the postcondition of these APIs that the
> > > command
> > > > > has
> > > > > >> > >> begun or
> > > > > >> > >> > > > that
> > > > > >> > >> > > > > > the command has been completed? It is a lot more
> > > usable
> > > > > if
> > > > > >> the
> > > > > >> > >> > > command
> > > > > >> > >> > > > > has
> > > > > >> > >> > > > > > been completed so you know that if you create a
> > topic
> > > > and
> > > > > >> then
> > > > > >> > >> > > publish
> > > > > >> > >> > > > to
> > > > > >> > >> > > > > > it you won't get an exception about there being
> no
> > > such
> > > > > >> topic.
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > 8. Describe topic and list topics duplicate a lot
> > of
> > > > > stuff
> > > > > >> in
> > > > > >> > >> the
> > > > > >> > >> > > > > metadata
> > > > > >> > >> > > > > > request. Is there a reason to give back topics
> > marked
> > > > for
> > > > > >> > >> > deletion? I
> > > > > >> > >> > > > > feel
> > > > > >> > >> > > > > > like if we just make the post-condition of the
> > delete
> > > > > >> command
> > > > > >> > be
> > > > > >> > >> > that
> > > > > >> > >> > > > the
> > > > > >> > >> > > > > > topic is deleted that will get rid of the need
> for
> > > this
> > > > > >> right?
> > > > > >> > >> And
> > > > > >> > >> > it
> > > > > >> > >> > > > > will
> > > > > >> > >> > > > > > be much more intuitive.
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > 9. Should we consider batching these requests? We
> > > have
> > > > > >> > generally
> > > > > >> > >> > > tried
> > > > > >> > >> > > > to
> > > > > >> > >> > > > > > allow multiple operations to be batched. My
> > suspicion
> > > > is
> > > > > >> that
> > > > > >> > >> > without
> > > > > >> > >> > > > > this
> > > > > >> > >> > > > > > we will get a lot of code that does something
> like
> > > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
> > > > > >> > >> > > > > >       adminClient.describeTopic(topic)
> > > > > >> > >> > > > > > this code will work great when you test on 5
> topics
> > > but
> > > > > >> not do
> > > > > >> > >> as
> > > > > >> > >> > > well
> > > > > >> > >> > > > if
> > > > > >> > >> > > > > > you have 50k.
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > 10. I think we should also discuss how we want to
> > > > expose
> > > > > a
> > > > > >> > >> > > programmatic
> > > > > >> > >> > > > > JVM
> > > > > >> > >> > > > > > client api for these operations. Currently people
> > > rely
> > > > on
> > > > > >> > >> > AdminUtils
> > > > > >> > >> > > > > which
> > > > > >> > >> > > > > > is totally sketchy. I think we probably need
> > another
> > > > > client
> > > > > >> > >> under
> > > > > >> > >> > > > > clients/
> > > > > >> > >> > > > > > that exposes administrative functionality. We
> will
> > > need
> > > > > >> this
> > > > > >> > >> just
> > > > > >> > >> > to
> > > > > >> > >> > > > > > properly test the new apis, I suspect. We should
> > > figure
> > > > > out
> > > > > >> > that
> > > > > >> > >> > API.
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > 11. The other information that would be really
> > useful
> > > > to
> > > > > >> get
> > > > > >> > >> would
> > > > > >> > >> > be
> > > > > >> > >> > > > > > information about partitions--how much data is in
> > the
> > > > > >> > partition,
> > > > > >> > >> > what
> > > > > >> > >> > > > are
> > > > > >> > >> > > > > > the segment offsets, what is the log-end offset
> > (i.e.
> > > > > last
> > > > > >> > >> offset),
> > > > > >> > >> > > > what
> > > > > >> > >> > > > > is
> > > > > >> > >> > > > > > the compaction point, etc. I think that done
> right
> > > this
> > > > > >> would
> > > > > >> > be
> > > > > >> > >> > the
> > > > > >> > >> > > > > > successor to the very awkward OffsetRequest we
> have
> > > > > today.
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > -Jay
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
> > > > > >> > >> joe.stein@stealth.ly>
> > > > > >> > >> > > > > wrote:
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > > > > Hi, created a KIP
> > > > > >> > >> > > > > > >
> > > > > >> > >> > > > > > >
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > >
> > > > > >> > >> > > >
> > > > > >> > >> > >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > >> > >> > > > > > >
> > > > > >> > >> > > > > > > JIRA
> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > >> > >> > > > > > >
> > > > > >> > >> > > > > > > /*******************************************
> > > > > >> > >> > > > > > >  Joe Stein
> > > > > >> > >> > > > > > >  Founder, Principal Consultant
> > > > > >> > >> > > > > > >  Big Data Open Source Security LLC
> > > > > >> > >> > > > > > >  http://www.stealth.ly
> > > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> > > > > >> > >> > http://www.twitter.com/allthingshadoop
> > > > > >> > >> > > >
> > > > > >> > >> > > > > > > ********************************************/
> > > > > >> > >> > > > > > >
> > > > > >> > >> > > > > >
> > > > > >> > >> > > > >
> > > > > >> > >> > > >
> > > > > >> > >> > >
> > > > > >> > >> >
> > > > > >> > >>
> > > > > >> > >>
> > > > > >> > >>
> > > > > >> > >> --
> > > > > >> > >> -- Guozhang
> > > > > >> > >>
> > > > > >> > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
>
> --
> Jeff Holoman
> Systems Engineer
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jeff Holoman <jh...@cloudera.com>.
Jay / Joe

We're happy to send out a Webex for this purpose. We could record the
sessions if there is interest and publish them out.

Thanks

Jeff

On Tue, Feb 24, 2015 at 11:28 AM, Jay Kreps <ja...@gmail.com> wrote:

> Let's try to get the technical hang-ups sorted out, though. I really think
> there is some benefit to live discussion vs writing. I am hopeful that if
> we post instructions and give ourselves a few attempts we can get it
> working.
>
> Tuesday at that time would work for me...any objections?
>
> -Jay
>
> On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <jo...@stealth.ly> wrote:
>
> > Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am PT ????
> >
> > I don't mind google hangout but there is always some issue or whatever so
> > we know the apache irc channel works. We can start there and see how it
> > goes? We can pull transcripts too and associate to tickets if need be
> makes
> > it helpful for things.
> >
> > ~ Joestein
> >
> > On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > We'd talked about doing a Google Hangout to chat about this. What about
> > > generalizing that a little further...I actually think it would be good
> > for
> > > everyone spending a reasonable chunk of their week on Kafka stuff to
> > maybe
> > > sync up once a week. I think we could use time to talk through design
> > > stuff, make sure we are on top of code reviews, talk through any tricky
> > > issues, etc.
> > >
> > > We can make it publicly available so that any one can follow along who
> > > likes.
> > >
> > > Any interest in doing this? If so I'll try to set it up starting next
> > week.
> > >
> > > -Jay
> > >
> > > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Hi all,
> > > >
> > > > I've updated KIP page, fixed / aligned document structure. Also I
> added
> > > > some
> > > > very initial proposal for AdminClient so we have something to start
> > from
> > > > while
> > > > discussing the KIP.
> > > >
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Jay,
> > > > >
> > > > > Re error messages: you are right, in most cases client will have
> > enough
> > > > > context to show descriptive error message. My concern is that we
> will
> > > > have
> > > > > to
> > > > > add lots of new error codes for each possible error. Of course, we
> > > could
> > > > > reuse
> > > > > some of existing like UknownTopicOrPartitionCode, but we will also
> > need
> > > > to
> > > > > add smth like: TopicAlreadyExistsCode, TopicConfigInvalid (both for
> > > topic
> > > > > name and config, and probably user would like to know what exactly
> > > > > is wrong in his config), InvalidReplicaAssignment, InternalError
> > (e.g.
> > > > > zookeeper failure) etc.
> > > > > And this is only for TopicCommand, we will also need to add similar
> > > stuff
> > > > > for
> > > > > ReassignPartitions, PreferredReplica. So we'll end up with a large
> > list
> > > > of
> > > > > error codes, used only in Admin protocol.
> > > > > Having said that, I agree my proposal is not consistent with other
> > > cases.
> > > > > Maybe we can find better solution or something in-between.
> > > > >
> > > > > Re Hangout chat: I think it is a great idea. This way we can move
> on
> > > > > faster.
> > > > > Let's agree somehow on date/time so people can join. Will work for
> me
> > > > this
> > > > > and
> > > > > next week almost anytime if agreed in advance.
> > > > >
> > > > > Thanks,
> > > > > Andrii
> > > > >
> > > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > >> Hey Andrii,
> > > > >>
> > > > >> Generally we can do good error handling without needing custom
> > > > server-side
> > > > >> messages. I.e. generally the client has the context to know that
> if
> > it
> > > > got
> > > > >> an error that the topic doesn't exist to say "Topic X doesn't
> exist"
> > > > >> rather
> > > > >> than "error code 14" (or whatever). Maybe there are specific cases
> > > where
> > > > >> this is hard? If we want to add server-side error messages we
> really
> > > do
> > > > >> need to do this in a consistent way across the protocol.
> > > > >>
> > > > >> I still have a bunch of open questions here from my previous
> list. I
> > > > will
> > > > >> be out for the next few days for Strata though. Maybe we could do
> a
> > > > Google
> > > > >> Hangout chat on any open issues some time towards the end of next
> > week
> > > > for
> > > > >> anyone interested in this ticket? I have a feeling that might
> > progress
> > > > >> things a little faster than email--I think we could talk through
> > those
> > > > >> issues I brought up fairly quickly...
> > > > >>
> > > > >> -Jay
> > > > >>
> > > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> > > > >> andrii.biletskyi@stealth.ly> wrote:
> > > > >>
> > > > >> > Hi all,
> > > > >> >
> > > > >> > I'm trying to address some of the issues which were mentioned
> > > earlier
> > > > >> about
> > > > >> > Admin RQ/RP format. One of those was about batching operations.
> > What
> > > > if
> > > > >> we
> > > > >> > follow TopicCommand approach and let people specify topic-name
> by
> > > > >> regexp -
> > > > >> > would that cover most of the use cases?
> > > > >> >
> > > > >> > Secondly, is what information should we generally provide in
> Admin
> > > > >> > responses.
> > > > >> > I realize that Admin commands don't imply they will be used only
> > in
> > > > CLI
> > > > >> > but,
> > > > >> > it seems to me, CLI is a very important client of this feature.
> In
> > > > this
> > > > >> > case,
> > > > >> > seems logical, we would like to provide users with rich
> experience
> > > in
> > > > >> terms
> > > > >> > of
> > > > >> > getting results / errors of the executed commands. Usually we
> > supply
> > > > >> with
> > > > >> > responses only errorCode, which looks very limiting, in case of
> > CLI
> > > we
> > > > >> may
> > > > >> > want to print human readable error description.
> > > > >> >
> > > > >> > So, taking into account previous item about batching, what do
> you
> > > > think
> > > > >> > about
> > > > >> > having smth like:
> > > > >> >
> > > > >> > ('create' doesn't support regexp)
> > > > >> > CreateTopicRequest => TopicName Partitions Replicas
> > > ReplicaAssignment
> > > > >> > [Config]
> > > > >> > CreateTopicResponse => ErrorCode ErrorDescription
> > > > >> >   ErrorCode => int16
> > > > >> >   ErrorDescription => string (empty if successful)
> > > > >> >
> > > > >> > AlterTopicRequest -> TopicNameRegexp Partitions
> ReplicaAssignment
> > > > >> > [AddedConfig] [DeletedConfig]
> > > > >> > AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
> > > > >> > CommandErrorCode CommandErrorDescription
> > > > >> >   CommandErrorCode => int16
> > > > >> >   CommandErrorDescription => string (nonempty in case of fatal
> > > error,
> > > > >> e.g.
> > > > >> > we couldn't get topics by regexp)
> > > > >> >
> > > > >> > DescribeTopicRequest -> TopicNameRegexp
> > > > >> > DescribeTopicResponse -> [TopicName TopicDescription ErrorCode
> > > > >> > ErrorDescription] CommandErrorCode CommandErrorDescription
> > > > >> >
> > > > >> > Also, any thoughts about our discussion regarding re-routing
> > > facility?
> > > > >> In
> > > > >> > my
> > > > >> > understanding, it is like between augmenting
> TopicMetadataRequest
> > > > >> > (to include at least controllerId) and implementing new generic
> > > > >> re-routing
> > > > >> > facility so sending messages to controller will be handled by
> it.
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Andrii Biletskyi
> > > > >> >
> > > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> > > > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > >> >
> > > > >> > > @Guozhang:
> > > > >> > > Thanks for your comments, I've answered some of those. The
> main
> > > > thing
> > > > >> is
> > > > >> > > having merged request for create-alter-delete-describe - I
> have
> > > some
> > > > >> > > concerns about this approach.
> > > > >> > >
> > > > >> > > @*Jay*:
> > > > >> > > I see that introduced ClusterMetadaRequest is also one of the
> > > > >> concerns.
> > > > >> > We
> > > > >> > > can solve it if we implement re-routing facility. But I agree
> > with
> > > > >> > > Guozhang - it will make clients' internals a little bit easier
> > but
> > > > >> this
> > > > >> > > seems to be a complex logic to implement and support then.
> > > > Especially
> > > > >> for
> > > > >> > > Fetch and Produce (even if we add re-routing later for these
> > > > >> requests).
> > > > >> > > Also people will tend to avoid this re-routing facility and
> hold
> > > > local
> > > > >> > > cluster cache to ensure their high-priority requests (which
> some
> > > of
> > > > >> the
> > > > >> > > admin requests are) not sent to some busy broker where they
> wait
> > > to
> > > > be
> > > > >> > > routed to the correct one.
> > > > >> > > As pointed out by Jun here (
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > > >> > )
> > > > >> > > to solve the issue we might introduce a message type to get
> > > cluster
> > > > >> > state.
> > > > >> > > But I agree we can just update TopicMetadataResponse to
> include
> > > > >> > > controllerId (and probably smth else).
> > > > >> > > What are you thougths?
> > > > >> > >
> > > > >> > > Thanks,
> > > > >> > > Andrii
> > > > >> > >
> > > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> > > wangguoz@gmail.com>
> > > > >> > wrote:
> > > > >> > >
> > > > >> > >> I think for the topics commands we can actually merge
> > > > >> > >> create/alter/delete/describe as one request type since their
> > > > formats
> > > > >> are
> > > > >> > >> very much similar, and keep list-topics and others like
> > > > >> > >> partition-reassignment / preferred-leader-election as
> separate
> > > > >> request
> > > > >> > >> types, I also left some other comments on the RB (
> > > > >> > >> https://reviews.apache.org/r/29301/).
> > > > >> > >>
> > > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> > jay.kreps@gmail.com>
> > > > >> wrote:
> > > > >> > >>
> > > > >> > >> > Yeah I totally agree that we don't want to just have one
> "do
> > > > admin
> > > > >> > >> stuff"
> > > > >> > >> > command that has the union of all parameters.
> > > > >> > >> >
> > > > >> > >> > What I am saying is that command line tools are one client
> of
> > > the
> > > > >> > >> > administrative apis, but these will be used in a number of
> > > > >> scenarios
> > > > >> > so
> > > > >> > >> > they should make logical sense even in the absence of the
> > > command
> > > > >> line
> > > > >> > >> > tool. Hence comments like trying to clarify the
> relationship
> > > > >> between
> > > > >> > >> > ClusterMetadata and TopicMetadata...these kinds of things
> > > really
> > > > >> need
> > > > >> > >> to be
> > > > >> > >> > thought through.
> > > > >> > >> >
> > > > >> > >> > Hope that makes sense.
> > > > >> > >> >
> > > > >> > >> > -Jay
> > > > >> > >> >
> > > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> > > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > >> >
> > > > >> > >> > > Jay,
> > > > >> > >> > >
> > > > >> > >> > > Thanks for answering. You understood correctly, most of
> my
> > > > >> comments
> > > > >> > >> were
> > > > >> > >> > > related to your point 1) - about "well thought-out" apis.
> > > Also,
> > > > >> yes,
> > > > >> > >> as I
> > > > >> > >> > > understood we would like to introduce a single unified
> CLI
> > > tool
> > > > >> with
> > > > >> > >> > > centralized server-side request handling for lots of
> > existing
> > > > >> ones
> > > > >> > >> (incl.
> > > > >> > >> > > TopicCommand, CommitOffsetChecker, ReassignPartitions,
> smth
> > > > else
> > > > >> if
> > > > >> > >> added
> > > > >> > >> > > in future). In our previous discussion (
> > > > >> > >> > > https://issues.apache.org/jira/browse/KAFKA-1694) people
> > > said
> > > > >> > they'd
> > > > >> > >> > > rather
> > > > >> > >> > > have a separate message for each command, so, yes, this
> > way I
> > > > >> came
> > > > >> > to
> > > > >> > >> 1-1
> > > > >> > >> > > mapping between commands in the tool and protocol
> > additions.
> > > > But
> > > > >> I
> > > > >> > >> might
> > > > >> > >> > be
> > > > >> > >> > > wrong.
> > > > >> > >> > > At the end I just try to start discussion how at least
> > > > generally
> > > > >> > this
> > > > >> > >> > > protocol should look like.
> > > > >> > >> > >
> > > > >> > >> > > Thanks,
> > > > >> > >> > > Andrii
> > > > >> > >> > >
> > > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
> > > > jay.kreps@gmail.com
> > > > >> >
> > > > >> > >> wrote:
> > > > >> > >> > >
> > > > >> > >> > > > Hey Andrii,
> > > > >> > >> > > >
> > > > >> > >> > > > To answer your earlier question we just really can't be
> > > > adding
> > > > >> any
> > > > >> > >> more
> > > > >> > >> > > > scala protocol objects. These things are super hard to
> > > > maintain
> > > > >> > >> because
> > > > >> > >> > > > they hand code the byte parsing and don't have good
> > > > versioning
> > > > >> > >> support.
> > > > >> > >> > > > Since we are already planning on converting we
> definitely
> > > > don't
> > > > >> > >> want to
> > > > >> > >> > > add
> > > > >> > >> > > > a ton more of these--they are total tech debt.
> > > > >> > >> > > >
> > > > >> > >> > > > What does it mean that the changes are isolated from
> the
> > > > >> current
> > > > >> > >> code
> > > > >> > >> > > base?
> > > > >> > >> > > >
> > > > >> > >> > > > I actually didn't understand the remaining comments,
> > which
> > > of
> > > > >> the
> > > > >> > >> > points
> > > > >> > >> > > > are you responding to?
> > > > >> > >> > > >
> > > > >> > >> > > > Maybe one sticking point here is that it seems like you
> > > want
> > > > to
> > > > >> > make
> > > > >> > >> > some
> > > > >> > >> > > > kind of tool, and you have made a 1-1 mapping between
> > > > commands
> > > > >> you
> > > > >> > >> > > imagine
> > > > >> > >> > > > in the tool and protocol additions. I want to make sure
> > we
> > > > >> don't
> > > > >> > do
> > > > >> > >> > that.
> > > > >> > >> > > > The protocol needs to be really really well thought out
> > > > against
> > > > >> > many
> > > > >> > >> > use
> > > > >> > >> > > > cases so it should make perfect logical sense in the
> > > absence
> > > > of
> > > > >> > >> knowing
> > > > >> > >> > > the
> > > > >> > >> > > > command line tool, right?
> > > > >> > >> > > >
> > > > >> > >> > > > -Jay
> > > > >> > >> > > >
> > > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
> > > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> > > > >> > >> > > >
> > > > >> > >> > > > > Hey Jay,
> > > > >> > >> > > > >
> > > > >> > >> > > > > I would like to continue this discussion as it seem
> > there
> > > > is
> > > > >> no
> > > > >> > >> > > progress
> > > > >> > >> > > > > here.
> > > > >> > >> > > > >
> > > > >> > >> > > > > First of all, could you please explain what did you
> > mean
> > > in
> > > > >> 2?
> > > > >> > How
> > > > >> > >> > > > exactly
> > > > >> > >> > > > > are we going to migrate to the new java protocol
> > > > definitions.
> > > > >> > And
> > > > >> > >> why
> > > > >> > >> > > > it's
> > > > >> > >> > > > > a blocker for centralized CLI?
> > > > >> > >> > > > >
> > > > >> > >> > > > > I agree with you, this feature includes lots of
> stuff,
> > > but
> > > > >> > >> thankfully
> > > > >> > >> > > > > almost all changes are isolated from the current code
> > > base,
> > > > >> > >> > > > > so the main thing, I think, we need to agree is RQ/RP
> > > > format.
> > > > >> > >> > > > > So how can we start discussion about the concrete
> > > messages
> > > > >> > format?
> > > > >> > >> > > > > Can we take (
> > > > >> > >> > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > > >> > >> > > > > )
> > > > >> > >> > > > > as starting point?
> > > > >> > >> > > > >
> > > > >> > >> > > > > We had some doubts earlier whether it worth
> introducing
> > > one
> > > > >> > >> generic
> > > > >> > >> > > Admin
> > > > >> > >> > > > > Request for all commands (
> > > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > >> > >> > > > > )
> > > > >> > >> > > > > but then everybody agreed it would be better to have
> > > > separate
> > > > >> > >> message
> > > > >> > >> > > for
> > > > >> > >> > > > > each admin command. The Request part is really
> dictated
> > > > from
> > > > >> the
> > > > >> > >> > > command
> > > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so the proposed
> > > > version
> > > > >> > >> should
> > > > >> > >> > be
> > > > >> > >> > > > > fine (let's put aside for now remarks about Optional
> > > type,
> > > > >> > >> batching,
> > > > >> > >> > > > > configs normalization - I agree with all of them).
> > > > >> > >> > > > > So the second part is Response. I see there are two
> > cases
> > > > >> here.
> > > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b) "Get"
> > > > requests -
> > > > >> > >> > > > > List/Describe...
> > > > >> > >> > > > >
> > > > >> > >> > > > > a) should only hold request result (regardless what
> we
> > > > decide
> > > > >> > >> about
> > > > >> > >> > > > > blocking/non-blocking commands execution).
> > > > >> > >> > > > > Usually we provide error code in response but since
> we
> > > will
> > > > >> use
> > > > >> > >> this
> > > > >> > >> > in
> > > > >> > >> > > > > interactive shell we need some human readable error
> > > > >> description
> > > > >> > -
> > > > >> > >> so
> > > > >> > >> > I
> > > > >> > >> > > > > added errorDesription field where you can at least
> > leave
> > > > >> > >> > > > > exception.getMessage.
> > > > >> > >> > > > >
> > > > >> > >> > > > > b) in addition to previous item message should hold
> > > command
> > > > >> > >> specific
> > > > >> > >> > > > > response data. We can discuss in detail each of them
> > but
> > > > >> let's
> > > > >> > for
> > > > >> > >> > now
> > > > >> > >> > > > > agree about the overall pattern.
> > > > >> > >> > > > >
> > > > >> > >> > > > > Thanks,
> > > > >> > >> > > > > Andrii Biletskyi
> > > > >> > >> > > > >
> > > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
> > > > >> jay.kreps@gmail.com
> > > > >> > >
> > > > >> > >> > > wrote:
> > > > >> > >> > > > >
> > > > >> > >> > > > > > Hey Joe,
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > This is great. A few comments on KIP-4
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 1. This is much needed functionality, but there
> are a
> > > lot
> > > > >> of
> > > > >> > >> the so
> > > > >> > >> > > > let's
> > > > >> > >> > > > > > really think these protocols through. We really
> want
> > to
> > > > >> end up
> > > > >> > >> > with a
> > > > >> > >> > > > set
> > > > >> > >> > > > > > of well thought-out, orthoganol apis. For this
> > reason I
> > > > >> think
> > > > >> > >> it is
> > > > >> > >> > > > > really
> > > > >> > >> > > > > > important to think through the end state even if
> that
> > > > >> includes
> > > > >> > >> APIs
> > > > >> > >> > > we
> > > > >> > >> > > > > > won't implement in the first phase.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 2. Let's please please please wait until we have
> > > switched
> > > > >> the
> > > > >> > >> > server
> > > > >> > >> > > > over
> > > > >> > >> > > > > > to the new java protocol definitions. If we add
> > upteen
> > > > >> more ad
> > > > >> > >> hoc
> > > > >> > >> > > > scala
> > > > >> > >> > > > > > objects that is just generating more work for the
> > > > >> conversion
> > > > >> > we
> > > > >> > >> > know
> > > > >> > >> > > we
> > > > >> > >> > > > > > have to do.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 3. This proposal introduces a new type of optional
> > > > >> parameter.
> > > > >> > >> This
> > > > >> > >> > is
> > > > >> > >> > > > > > inconsistent with everything else in the protocol
> > where
> > > > we
> > > > >> use
> > > > >> > >> -1
> > > > >> > >> > or
> > > > >> > >> > > > some
> > > > >> > >> > > > > > other marker value. You could argue either way but
> > > let's
> > > > >> stick
> > > > >> > >> with
> > > > >> > >> > > > that
> > > > >> > >> > > > > > for consistency. For clients that implemented the
> > > > protocol
> > > > >> in
> > > > >> > a
> > > > >> > >> > > better
> > > > >> > >> > > > > way
> > > > >> > >> > > > > > than our scala code these basic primitives are hard
> > to
> > > > >> change.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 4. ClusterMetadata: This seems to duplicate
> > > > >> > TopicMetadataRequest
> > > > >> > >> > > which
> > > > >> > >> > > > > has
> > > > >> > >> > > > > > brokers, topics, and partitions. I think we should
> > > rename
> > > > >> that
> > > > >> > >> > > request
> > > > >> > >> > > > > > ClusterMetadataRequest (or just MetadataRequest)
> and
> > > > >> include
> > > > >> > >> the id
> > > > >> > >> > > of
> > > > >> > >> > > > > the
> > > > >> > >> > > > > > controller. Or are there other things we could add
> > > here?
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 5. We have a tendency to try to make a lot of
> > requests
> > > > that
> > > > >> > can
> > > > >> > >> > only
> > > > >> > >> > > go
> > > > >> > >> > > > > to
> > > > >> > >> > > > > > particular nodes. This adds a lot of burden for
> > client
> > > > >> > >> > > implementations
> > > > >> > >> > > > > (it
> > > > >> > >> > > > > > sounds easy but each discovery can fail in many
> parts
> > > so
> > > > it
> > > > >> > >> ends up
> > > > >> > >> > > > > being a
> > > > >> > >> > > > > > full state machine to do right). I think we should
> > > > consider
> > > > >> > >> making
> > > > >> > >> > > > admin
> > > > >> > >> > > > > > commands and ideally as many of the other apis as
> > > > possible
> > > > >> > >> > available
> > > > >> > >> > > on
> > > > >> > >> > > > > all
> > > > >> > >> > > > > > brokers and just redirect to the controller on the
> > > broker
> > > > >> > side.
> > > > >> > >> > > Perhaps
> > > > >> > >> > > > > > there would be a general way to encapsulate this
> > > > re-routing
> > > > >> > >> > behavior.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 6. We should probably normalize the key value pairs
> > > used
> > > > >> for
> > > > >> > >> > configs
> > > > >> > >> > > > > rather
> > > > >> > >> > > > > > than embedding a new formatting. So two strings
> > rather
> > > > than
> > > > >> > one
> > > > >> > >> > with
> > > > >> > >> > > an
> > > > >> > >> > > > > > internal equals sign.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 7. Is the postcondition of these APIs that the
> > command
> > > > has
> > > > >> > >> begun or
> > > > >> > >> > > > that
> > > > >> > >> > > > > > the command has been completed? It is a lot more
> > usable
> > > > if
> > > > >> the
> > > > >> > >> > > command
> > > > >> > >> > > > > has
> > > > >> > >> > > > > > been completed so you know that if you create a
> topic
> > > and
> > > > >> then
> > > > >> > >> > > publish
> > > > >> > >> > > > to
> > > > >> > >> > > > > > it you won't get an exception about there being no
> > such
> > > > >> topic.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 8. Describe topic and list topics duplicate a lot
> of
> > > > stuff
> > > > >> in
> > > > >> > >> the
> > > > >> > >> > > > > metadata
> > > > >> > >> > > > > > request. Is there a reason to give back topics
> marked
> > > for
> > > > >> > >> > deletion? I
> > > > >> > >> > > > > feel
> > > > >> > >> > > > > > like if we just make the post-condition of the
> delete
> > > > >> command
> > > > >> > be
> > > > >> > >> > that
> > > > >> > >> > > > the
> > > > >> > >> > > > > > topic is deleted that will get rid of the need for
> > this
> > > > >> right?
> > > > >> > >> And
> > > > >> > >> > it
> > > > >> > >> > > > > will
> > > > >> > >> > > > > > be much more intuitive.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 9. Should we consider batching these requests? We
> > have
> > > > >> > generally
> > > > >> > >> > > tried
> > > > >> > >> > > > to
> > > > >> > >> > > > > > allow multiple operations to be batched. My
> suspicion
> > > is
> > > > >> that
> > > > >> > >> > without
> > > > >> > >> > > > > this
> > > > >> > >> > > > > > we will get a lot of code that does something like
> > > > >> > >> > > > > >    for(topic: adminClient.listTopics())
> > > > >> > >> > > > > >       adminClient.describeTopic(topic)
> > > > >> > >> > > > > > this code will work great when you test on 5 topics
> > but
> > > > >> not do
> > > > >> > >> as
> > > > >> > >> > > well
> > > > >> > >> > > > if
> > > > >> > >> > > > > > you have 50k.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 10. I think we should also discuss how we want to
> > > expose
> > > > a
> > > > >> > >> > > programmatic
> > > > >> > >> > > > > JVM
> > > > >> > >> > > > > > client api for these operations. Currently people
> > rely
> > > on
> > > > >> > >> > AdminUtils
> > > > >> > >> > > > > which
> > > > >> > >> > > > > > is totally sketchy. I think we probably need
> another
> > > > client
> > > > >> > >> under
> > > > >> > >> > > > > clients/
> > > > >> > >> > > > > > that exposes administrative functionality. We will
> > need
> > > > >> this
> > > > >> > >> just
> > > > >> > >> > to
> > > > >> > >> > > > > > properly test the new apis, I suspect. We should
> > figure
> > > > out
> > > > >> > that
> > > > >> > >> > API.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > 11. The other information that would be really
> useful
> > > to
> > > > >> get
> > > > >> > >> would
> > > > >> > >> > be
> > > > >> > >> > > > > > information about partitions--how much data is in
> the
> > > > >> > partition,
> > > > >> > >> > what
> > > > >> > >> > > > are
> > > > >> > >> > > > > > the segment offsets, what is the log-end offset
> (i.e.
> > > > last
> > > > >> > >> offset),
> > > > >> > >> > > > what
> > > > >> > >> > > > > is
> > > > >> > >> > > > > > the compaction point, etc. I think that done right
> > this
> > > > >> would
> > > > >> > be
> > > > >> > >> > the
> > > > >> > >> > > > > > successor to the very awkward OffsetRequest we have
> > > > today.
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > -Jay
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
> > > > >> > >> joe.stein@stealth.ly>
> > > > >> > >> > > > > wrote:
> > > > >> > >> > > > > >
> > > > >> > >> > > > > > > Hi, created a KIP
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > JIRA
> > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > > > /*******************************************
> > > > >> > >> > > > > > >  Joe Stein
> > > > >> > >> > > > > > >  Founder, Principal Consultant
> > > > >> > >> > > > > > >  Big Data Open Source Security LLC
> > > > >> > >> > > > > > >  http://www.stealth.ly
> > > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> > > > >> > >> > http://www.twitter.com/allthingshadoop
> > > > >> > >> > > >
> > > > >> > >> > > > > > > ********************************************/
> > > > >> > >> > > > > > >
> > > > >> > >> > > > > >
> > > > >> > >> > > > >
> > > > >> > >> > > >
> > > > >> > >> > >
> > > > >> > >> >
> > > > >> > >>
> > > > >> > >>
> > > > >> > >>
> > > > >> > >> --
> > > > >> > >> -- Guozhang
> > > > >> > >>
> > > > >> > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>



-- 
Jeff Holoman
Systems Engineer

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
Let's try to get the technical hang-ups sorted out, though. I really think
there is some benefit to live discussion vs writing. I am hopeful that if
we post instructions and give ourselves a few attempts we can get it
working.

Tuesday at that time would work for me...any objections?

-Jay

On Tue, Feb 24, 2015 at 8:18 AM, Joe Stein <jo...@stealth.ly> wrote:

> Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am PT ????
>
> I don't mind google hangout but there is always some issue or whatever so
> we know the apache irc channel works. We can start there and see how it
> goes? We can pull transcripts too and associate to tickets if need be makes
> it helpful for things.
>
> ~ Joestein
>
> On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > We'd talked about doing a Google Hangout to chat about this. What about
> > generalizing that a little further...I actually think it would be good
> for
> > everyone spending a reasonable chunk of their week on Kafka stuff to
> maybe
> > sync up once a week. I think we could use time to talk through design
> > stuff, make sure we are on top of code reviews, talk through any tricky
> > issues, etc.
> >
> > We can make it publicly available so that any one can follow along who
> > likes.
> >
> > Any interest in doing this? If so I'll try to set it up starting next
> week.
> >
> > -Jay
> >
> > On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Hi all,
> > >
> > > I've updated KIP page, fixed / aligned document structure. Also I added
> > > some
> > > very initial proposal for AdminClient so we have something to start
> from
> > > while
> > > discussing the KIP.
> > >
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Jay,
> > > >
> > > > Re error messages: you are right, in most cases client will have
> enough
> > > > context to show descriptive error message. My concern is that we will
> > > have
> > > > to
> > > > add lots of new error codes for each possible error. Of course, we
> > could
> > > > reuse
> > > > some of existing like UknownTopicOrPartitionCode, but we will also
> need
> > > to
> > > > add smth like: TopicAlreadyExistsCode, TopicConfigInvalid (both for
> > topic
> > > > name and config, and probably user would like to know what exactly
> > > > is wrong in his config), InvalidReplicaAssignment, InternalError
> (e.g.
> > > > zookeeper failure) etc.
> > > > And this is only for TopicCommand, we will also need to add similar
> > stuff
> > > > for
> > > > ReassignPartitions, PreferredReplica. So we'll end up with a large
> list
> > > of
> > > > error codes, used only in Admin protocol.
> > > > Having said that, I agree my proposal is not consistent with other
> > cases.
> > > > Maybe we can find better solution or something in-between.
> > > >
> > > > Re Hangout chat: I think it is a great idea. This way we can move on
> > > > faster.
> > > > Let's agree somehow on date/time so people can join. Will work for me
> > > this
> > > > and
> > > > next week almost anytime if agreed in advance.
> > > >
> > > > Thanks,
> > > > Andrii
> > > >
> > > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > >> Hey Andrii,
> > > >>
> > > >> Generally we can do good error handling without needing custom
> > > server-side
> > > >> messages. I.e. generally the client has the context to know that if
> it
> > > got
> > > >> an error that the topic doesn't exist to say "Topic X doesn't exist"
> > > >> rather
> > > >> than "error code 14" (or whatever). Maybe there are specific cases
> > where
> > > >> this is hard? If we want to add server-side error messages we really
> > do
> > > >> need to do this in a consistent way across the protocol.
> > > >>
> > > >> I still have a bunch of open questions here from my previous list. I
> > > will
> > > >> be out for the next few days for Strata though. Maybe we could do a
> > > Google
> > > >> Hangout chat on any open issues some time towards the end of next
> week
> > > for
> > > >> anyone interested in this ticket? I have a feeling that might
> progress
> > > >> things a little faster than email--I think we could talk through
> those
> > > >> issues I brought up fairly quickly...
> > > >>
> > > >> -Jay
> > > >>
> > > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> > > >> andrii.biletskyi@stealth.ly> wrote:
> > > >>
> > > >> > Hi all,
> > > >> >
> > > >> > I'm trying to address some of the issues which were mentioned
> > earlier
> > > >> about
> > > >> > Admin RQ/RP format. One of those was about batching operations.
> What
> > > if
> > > >> we
> > > >> > follow TopicCommand approach and let people specify topic-name by
> > > >> regexp -
> > > >> > would that cover most of the use cases?
> > > >> >
> > > >> > Secondly, is what information should we generally provide in Admin
> > > >> > responses.
> > > >> > I realize that Admin commands don't imply they will be used only
> in
> > > CLI
> > > >> > but,
> > > >> > it seems to me, CLI is a very important client of this feature. In
> > > this
> > > >> > case,
> > > >> > seems logical, we would like to provide users with rich experience
> > in
> > > >> terms
> > > >> > of
> > > >> > getting results / errors of the executed commands. Usually we
> supply
> > > >> with
> > > >> > responses only errorCode, which looks very limiting, in case of
> CLI
> > we
> > > >> may
> > > >> > want to print human readable error description.
> > > >> >
> > > >> > So, taking into account previous item about batching, what do you
> > > think
> > > >> > about
> > > >> > having smth like:
> > > >> >
> > > >> > ('create' doesn't support regexp)
> > > >> > CreateTopicRequest => TopicName Partitions Replicas
> > ReplicaAssignment
> > > >> > [Config]
> > > >> > CreateTopicResponse => ErrorCode ErrorDescription
> > > >> >   ErrorCode => int16
> > > >> >   ErrorDescription => string (empty if successful)
> > > >> >
> > > >> > AlterTopicRequest -> TopicNameRegexp Partitions ReplicaAssignment
> > > >> > [AddedConfig] [DeletedConfig]
> > > >> > AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
> > > >> > CommandErrorCode CommandErrorDescription
> > > >> >   CommandErrorCode => int16
> > > >> >   CommandErrorDescription => string (nonempty in case of fatal
> > error,
> > > >> e.g.
> > > >> > we couldn't get topics by regexp)
> > > >> >
> > > >> > DescribeTopicRequest -> TopicNameRegexp
> > > >> > DescribeTopicResponse -> [TopicName TopicDescription ErrorCode
> > > >> > ErrorDescription] CommandErrorCode CommandErrorDescription
> > > >> >
> > > >> > Also, any thoughts about our discussion regarding re-routing
> > facility?
> > > >> In
> > > >> > my
> > > >> > understanding, it is like between augmenting TopicMetadataRequest
> > > >> > (to include at least controllerId) and implementing new generic
> > > >> re-routing
> > > >> > facility so sending messages to controller will be handled by it.
> > > >> >
> > > >> > Thanks,
> > > >> > Andrii Biletskyi
> > > >> >
> > > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> > > >> > andrii.biletskyi@stealth.ly> wrote:
> > > >> >
> > > >> > > @Guozhang:
> > > >> > > Thanks for your comments, I've answered some of those. The main
> > > thing
> > > >> is
> > > >> > > having merged request for create-alter-delete-describe - I have
> > some
> > > >> > > concerns about this approach.
> > > >> > >
> > > >> > > @*Jay*:
> > > >> > > I see that introduced ClusterMetadaRequest is also one of the
> > > >> concerns.
> > > >> > We
> > > >> > > can solve it if we implement re-routing facility. But I agree
> with
> > > >> > > Guozhang - it will make clients' internals a little bit easier
> but
> > > >> this
> > > >> > > seems to be a complex logic to implement and support then.
> > > Especially
> > > >> for
> > > >> > > Fetch and Produce (even if we add re-routing later for these
> > > >> requests).
> > > >> > > Also people will tend to avoid this re-routing facility and hold
> > > local
> > > >> > > cluster cache to ensure their high-priority requests (which some
> > of
> > > >> the
> > > >> > > admin requests are) not sent to some busy broker where they wait
> > to
> > > be
> > > >> > > routed to the correct one.
> > > >> > > As pointed out by Jun here (
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > > >> > )
> > > >> > > to solve the issue we might introduce a message type to get
> > cluster
> > > >> > state.
> > > >> > > But I agree we can just update TopicMetadataResponse to include
> > > >> > > controllerId (and probably smth else).
> > > >> > > What are you thougths?
> > > >> > >
> > > >> > > Thanks,
> > > >> > > Andrii
> > > >> > >
> > > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> > wangguoz@gmail.com>
> > > >> > wrote:
> > > >> > >
> > > >> > >> I think for the topics commands we can actually merge
> > > >> > >> create/alter/delete/describe as one request type since their
> > > formats
> > > >> are
> > > >> > >> very much similar, and keep list-topics and others like
> > > >> > >> partition-reassignment / preferred-leader-election as separate
> > > >> request
> > > >> > >> types, I also left some other comments on the RB (
> > > >> > >> https://reviews.apache.org/r/29301/).
> > > >> > >>
> > > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <
> jay.kreps@gmail.com>
> > > >> wrote:
> > > >> > >>
> > > >> > >> > Yeah I totally agree that we don't want to just have one "do
> > > admin
> > > >> > >> stuff"
> > > >> > >> > command that has the union of all parameters.
> > > >> > >> >
> > > >> > >> > What I am saying is that command line tools are one client of
> > the
> > > >> > >> > administrative apis, but these will be used in a number of
> > > >> scenarios
> > > >> > so
> > > >> > >> > they should make logical sense even in the absence of the
> > command
> > > >> line
> > > >> > >> > tool. Hence comments like trying to clarify the relationship
> > > >> between
> > > >> > >> > ClusterMetadata and TopicMetadata...these kinds of things
> > really
> > > >> need
> > > >> > >> to be
> > > >> > >> > thought through.
> > > >> > >> >
> > > >> > >> > Hope that makes sense.
> > > >> > >> >
> > > >> > >> > -Jay
> > > >> > >> >
> > > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> > > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > > >> > >> >
> > > >> > >> > > Jay,
> > > >> > >> > >
> > > >> > >> > > Thanks for answering. You understood correctly, most of my
> > > >> comments
> > > >> > >> were
> > > >> > >> > > related to your point 1) - about "well thought-out" apis.
> > Also,
> > > >> yes,
> > > >> > >> as I
> > > >> > >> > > understood we would like to introduce a single unified CLI
> > tool
> > > >> with
> > > >> > >> > > centralized server-side request handling for lots of
> existing
> > > >> ones
> > > >> > >> (incl.
> > > >> > >> > > TopicCommand, CommitOffsetChecker, ReassignPartitions, smth
> > > else
> > > >> if
> > > >> > >> added
> > > >> > >> > > in future). In our previous discussion (
> > > >> > >> > > https://issues.apache.org/jira/browse/KAFKA-1694) people
> > said
> > > >> > they'd
> > > >> > >> > > rather
> > > >> > >> > > have a separate message for each command, so, yes, this
> way I
> > > >> came
> > > >> > to
> > > >> > >> 1-1
> > > >> > >> > > mapping between commands in the tool and protocol
> additions.
> > > But
> > > >> I
> > > >> > >> might
> > > >> > >> > be
> > > >> > >> > > wrong.
> > > >> > >> > > At the end I just try to start discussion how at least
> > > generally
> > > >> > this
> > > >> > >> > > protocol should look like.
> > > >> > >> > >
> > > >> > >> > > Thanks,
> > > >> > >> > > Andrii
> > > >> > >> > >
> > > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
> > > jay.kreps@gmail.com
> > > >> >
> > > >> > >> wrote:
> > > >> > >> > >
> > > >> > >> > > > Hey Andrii,
> > > >> > >> > > >
> > > >> > >> > > > To answer your earlier question we just really can't be
> > > adding
> > > >> any
> > > >> > >> more
> > > >> > >> > > > scala protocol objects. These things are super hard to
> > > maintain
> > > >> > >> because
> > > >> > >> > > > they hand code the byte parsing and don't have good
> > > versioning
> > > >> > >> support.
> > > >> > >> > > > Since we are already planning on converting we definitely
> > > don't
> > > >> > >> want to
> > > >> > >> > > add
> > > >> > >> > > > a ton more of these--they are total tech debt.
> > > >> > >> > > >
> > > >> > >> > > > What does it mean that the changes are isolated from the
> > > >> current
> > > >> > >> code
> > > >> > >> > > base?
> > > >> > >> > > >
> > > >> > >> > > > I actually didn't understand the remaining comments,
> which
> > of
> > > >> the
> > > >> > >> > points
> > > >> > >> > > > are you responding to?
> > > >> > >> > > >
> > > >> > >> > > > Maybe one sticking point here is that it seems like you
> > want
> > > to
> > > >> > make
> > > >> > >> > some
> > > >> > >> > > > kind of tool, and you have made a 1-1 mapping between
> > > commands
> > > >> you
> > > >> > >> > > imagine
> > > >> > >> > > > in the tool and protocol additions. I want to make sure
> we
> > > >> don't
> > > >> > do
> > > >> > >> > that.
> > > >> > >> > > > The protocol needs to be really really well thought out
> > > against
> > > >> > many
> > > >> > >> > use
> > > >> > >> > > > cases so it should make perfect logical sense in the
> > absence
> > > of
> > > >> > >> knowing
> > > >> > >> > > the
> > > >> > >> > > > command line tool, right?
> > > >> > >> > > >
> > > >> > >> > > > -Jay
> > > >> > >> > > >
> > > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
> > > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >> > >> > > >
> > > >> > >> > > > > Hey Jay,
> > > >> > >> > > > >
> > > >> > >> > > > > I would like to continue this discussion as it seem
> there
> > > is
> > > >> no
> > > >> > >> > > progress
> > > >> > >> > > > > here.
> > > >> > >> > > > >
> > > >> > >> > > > > First of all, could you please explain what did you
> mean
> > in
> > > >> 2?
> > > >> > How
> > > >> > >> > > > exactly
> > > >> > >> > > > > are we going to migrate to the new java protocol
> > > definitions.
> > > >> > And
> > > >> > >> why
> > > >> > >> > > > it's
> > > >> > >> > > > > a blocker for centralized CLI?
> > > >> > >> > > > >
> > > >> > >> > > > > I agree with you, this feature includes lots of stuff,
> > but
> > > >> > >> thankfully
> > > >> > >> > > > > almost all changes are isolated from the current code
> > base,
> > > >> > >> > > > > so the main thing, I think, we need to agree is RQ/RP
> > > format.
> > > >> > >> > > > > So how can we start discussion about the concrete
> > messages
> > > >> > format?
> > > >> > >> > > > > Can we take (
> > > >> > >> > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > >> > >> > > > > )
> > > >> > >> > > > > as starting point?
> > > >> > >> > > > >
> > > >> > >> > > > > We had some doubts earlier whether it worth introducing
> > one
> > > >> > >> generic
> > > >> > >> > > Admin
> > > >> > >> > > > > Request for all commands (
> > > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > >> > >> > > > > )
> > > >> > >> > > > > but then everybody agreed it would be better to have
> > > separate
> > > >> > >> message
> > > >> > >> > > for
> > > >> > >> > > > > each admin command. The Request part is really dictated
> > > from
> > > >> the
> > > >> > >> > > command
> > > >> > >> > > > > (e.g. TopicCommand) arguments itself, so the proposed
> > > version
> > > >> > >> should
> > > >> > >> > be
> > > >> > >> > > > > fine (let's put aside for now remarks about Optional
> > type,
> > > >> > >> batching,
> > > >> > >> > > > > configs normalization - I agree with all of them).
> > > >> > >> > > > > So the second part is Response. I see there are two
> cases
> > > >> here.
> > > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b) "Get"
> > > requests -
> > > >> > >> > > > > List/Describe...
> > > >> > >> > > > >
> > > >> > >> > > > > a) should only hold request result (regardless what we
> > > decide
> > > >> > >> about
> > > >> > >> > > > > blocking/non-blocking commands execution).
> > > >> > >> > > > > Usually we provide error code in response but since we
> > will
> > > >> use
> > > >> > >> this
> > > >> > >> > in
> > > >> > >> > > > > interactive shell we need some human readable error
> > > >> description
> > > >> > -
> > > >> > >> so
> > > >> > >> > I
> > > >> > >> > > > > added errorDesription field where you can at least
> leave
> > > >> > >> > > > > exception.getMessage.
> > > >> > >> > > > >
> > > >> > >> > > > > b) in addition to previous item message should hold
> > command
> > > >> > >> specific
> > > >> > >> > > > > response data. We can discuss in detail each of them
> but
> > > >> let's
> > > >> > for
> > > >> > >> > now
> > > >> > >> > > > > agree about the overall pattern.
> > > >> > >> > > > >
> > > >> > >> > > > > Thanks,
> > > >> > >> > > > > Andrii Biletskyi
> > > >> > >> > > > >
> > > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
> > > >> jay.kreps@gmail.com
> > > >> > >
> > > >> > >> > > wrote:
> > > >> > >> > > > >
> > > >> > >> > > > > > Hey Joe,
> > > >> > >> > > > > >
> > > >> > >> > > > > > This is great. A few comments on KIP-4
> > > >> > >> > > > > >
> > > >> > >> > > > > > 1. This is much needed functionality, but there are a
> > lot
> > > >> of
> > > >> > >> the so
> > > >> > >> > > > let's
> > > >> > >> > > > > > really think these protocols through. We really want
> to
> > > >> end up
> > > >> > >> > with a
> > > >> > >> > > > set
> > > >> > >> > > > > > of well thought-out, orthoganol apis. For this
> reason I
> > > >> think
> > > >> > >> it is
> > > >> > >> > > > > really
> > > >> > >> > > > > > important to think through the end state even if that
> > > >> includes
> > > >> > >> APIs
> > > >> > >> > > we
> > > >> > >> > > > > > won't implement in the first phase.
> > > >> > >> > > > > >
> > > >> > >> > > > > > 2. Let's please please please wait until we have
> > switched
> > > >> the
> > > >> > >> > server
> > > >> > >> > > > over
> > > >> > >> > > > > > to the new java protocol definitions. If we add
> upteen
> > > >> more ad
> > > >> > >> hoc
> > > >> > >> > > > scala
> > > >> > >> > > > > > objects that is just generating more work for the
> > > >> conversion
> > > >> > we
> > > >> > >> > know
> > > >> > >> > > we
> > > >> > >> > > > > > have to do.
> > > >> > >> > > > > >
> > > >> > >> > > > > > 3. This proposal introduces a new type of optional
> > > >> parameter.
> > > >> > >> This
> > > >> > >> > is
> > > >> > >> > > > > > inconsistent with everything else in the protocol
> where
> > > we
> > > >> use
> > > >> > >> -1
> > > >> > >> > or
> > > >> > >> > > > some
> > > >> > >> > > > > > other marker value. You could argue either way but
> > let's
> > > >> stick
> > > >> > >> with
> > > >> > >> > > > that
> > > >> > >> > > > > > for consistency. For clients that implemented the
> > > protocol
> > > >> in
> > > >> > a
> > > >> > >> > > better
> > > >> > >> > > > > way
> > > >> > >> > > > > > than our scala code these basic primitives are hard
> to
> > > >> change.
> > > >> > >> > > > > >
> > > >> > >> > > > > > 4. ClusterMetadata: This seems to duplicate
> > > >> > TopicMetadataRequest
> > > >> > >> > > which
> > > >> > >> > > > > has
> > > >> > >> > > > > > brokers, topics, and partitions. I think we should
> > rename
> > > >> that
> > > >> > >> > > request
> > > >> > >> > > > > > ClusterMetadataRequest (or just MetadataRequest) and
> > > >> include
> > > >> > >> the id
> > > >> > >> > > of
> > > >> > >> > > > > the
> > > >> > >> > > > > > controller. Or are there other things we could add
> > here?
> > > >> > >> > > > > >
> > > >> > >> > > > > > 5. We have a tendency to try to make a lot of
> requests
> > > that
> > > >> > can
> > > >> > >> > only
> > > >> > >> > > go
> > > >> > >> > > > > to
> > > >> > >> > > > > > particular nodes. This adds a lot of burden for
> client
> > > >> > >> > > implementations
> > > >> > >> > > > > (it
> > > >> > >> > > > > > sounds easy but each discovery can fail in many parts
> > so
> > > it
> > > >> > >> ends up
> > > >> > >> > > > > being a
> > > >> > >> > > > > > full state machine to do right). I think we should
> > > consider
> > > >> > >> making
> > > >> > >> > > > admin
> > > >> > >> > > > > > commands and ideally as many of the other apis as
> > > possible
> > > >> > >> > available
> > > >> > >> > > on
> > > >> > >> > > > > all
> > > >> > >> > > > > > brokers and just redirect to the controller on the
> > broker
> > > >> > side.
> > > >> > >> > > Perhaps
> > > >> > >> > > > > > there would be a general way to encapsulate this
> > > re-routing
> > > >> > >> > behavior.
> > > >> > >> > > > > >
> > > >> > >> > > > > > 6. We should probably normalize the key value pairs
> > used
> > > >> for
> > > >> > >> > configs
> > > >> > >> > > > > rather
> > > >> > >> > > > > > than embedding a new formatting. So two strings
> rather
> > > than
> > > >> > one
> > > >> > >> > with
> > > >> > >> > > an
> > > >> > >> > > > > > internal equals sign.
> > > >> > >> > > > > >
> > > >> > >> > > > > > 7. Is the postcondition of these APIs that the
> command
> > > has
> > > >> > >> begun or
> > > >> > >> > > > that
> > > >> > >> > > > > > the command has been completed? It is a lot more
> usable
> > > if
> > > >> the
> > > >> > >> > > command
> > > >> > >> > > > > has
> > > >> > >> > > > > > been completed so you know that if you create a topic
> > and
> > > >> then
> > > >> > >> > > publish
> > > >> > >> > > > to
> > > >> > >> > > > > > it you won't get an exception about there being no
> such
> > > >> topic.
> > > >> > >> > > > > >
> > > >> > >> > > > > > 8. Describe topic and list topics duplicate a lot of
> > > stuff
> > > >> in
> > > >> > >> the
> > > >> > >> > > > > metadata
> > > >> > >> > > > > > request. Is there a reason to give back topics marked
> > for
> > > >> > >> > deletion? I
> > > >> > >> > > > > feel
> > > >> > >> > > > > > like if we just make the post-condition of the delete
> > > >> command
> > > >> > be
> > > >> > >> > that
> > > >> > >> > > > the
> > > >> > >> > > > > > topic is deleted that will get rid of the need for
> this
> > > >> right?
> > > >> > >> And
> > > >> > >> > it
> > > >> > >> > > > > will
> > > >> > >> > > > > > be much more intuitive.
> > > >> > >> > > > > >
> > > >> > >> > > > > > 9. Should we consider batching these requests? We
> have
> > > >> > generally
> > > >> > >> > > tried
> > > >> > >> > > > to
> > > >> > >> > > > > > allow multiple operations to be batched. My suspicion
> > is
> > > >> that
> > > >> > >> > without
> > > >> > >> > > > > this
> > > >> > >> > > > > > we will get a lot of code that does something like
> > > >> > >> > > > > >    for(topic: adminClient.listTopics())
> > > >> > >> > > > > >       adminClient.describeTopic(topic)
> > > >> > >> > > > > > this code will work great when you test on 5 topics
> but
> > > >> not do
> > > >> > >> as
> > > >> > >> > > well
> > > >> > >> > > > if
> > > >> > >> > > > > > you have 50k.
> > > >> > >> > > > > >
> > > >> > >> > > > > > 10. I think we should also discuss how we want to
> > expose
> > > a
> > > >> > >> > > programmatic
> > > >> > >> > > > > JVM
> > > >> > >> > > > > > client api for these operations. Currently people
> rely
> > on
> > > >> > >> > AdminUtils
> > > >> > >> > > > > which
> > > >> > >> > > > > > is totally sketchy. I think we probably need another
> > > client
> > > >> > >> under
> > > >> > >> > > > > clients/
> > > >> > >> > > > > > that exposes administrative functionality. We will
> need
> > > >> this
> > > >> > >> just
> > > >> > >> > to
> > > >> > >> > > > > > properly test the new apis, I suspect. We should
> figure
> > > out
> > > >> > that
> > > >> > >> > API.
> > > >> > >> > > > > >
> > > >> > >> > > > > > 11. The other information that would be really useful
> > to
> > > >> get
> > > >> > >> would
> > > >> > >> > be
> > > >> > >> > > > > > information about partitions--how much data is in the
> > > >> > partition,
> > > >> > >> > what
> > > >> > >> > > > are
> > > >> > >> > > > > > the segment offsets, what is the log-end offset (i.e.
> > > last
> > > >> > >> offset),
> > > >> > >> > > > what
> > > >> > >> > > > > is
> > > >> > >> > > > > > the compaction point, etc. I think that done right
> this
> > > >> would
> > > >> > be
> > > >> > >> > the
> > > >> > >> > > > > > successor to the very awkward OffsetRequest we have
> > > today.
> > > >> > >> > > > > >
> > > >> > >> > > > > > -Jay
> > > >> > >> > > > > >
> > > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
> > > >> > >> joe.stein@stealth.ly>
> > > >> > >> > > > > wrote:
> > > >> > >> > > > > >
> > > >> > >> > > > > > > Hi, created a KIP
> > > >> > >> > > > > > >
> > > >> > >> > > > > > >
> > > >> > >> > > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >> >
> > > >>
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > JIRA
> > https://issues.apache.org/jira/browse/KAFKA-1694
> > > >> > >> > > > > > >
> > > >> > >> > > > > > > /*******************************************
> > > >> > >> > > > > > >  Joe Stein
> > > >> > >> > > > > > >  Founder, Principal Consultant
> > > >> > >> > > > > > >  Big Data Open Source Security LLC
> > > >> > >> > > > > > >  http://www.stealth.ly
> > > >> > >> > > > > > >  Twitter: @allthingshadoop <
> > > >> > >> > http://www.twitter.com/allthingshadoop
> > > >> > >> > > >
> > > >> > >> > > > > > > ********************************************/
> > > >> > >> > > > > > >
> > > >> > >> > > > > >
> > > >> > >> > > > >
> > > >> > >> > > >
> > > >> > >> > >
> > > >> > >> >
> > > >> > >>
> > > >> > >>
> > > >> > >>
> > > >> > >> --
> > > >> > >> -- Guozhang
> > > >> > >>
> > > >> > >
> > > >> > >
> > > >> >
> > > >>
> > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Joe Stein <jo...@stealth.ly>.
Weekly would be great maybe like every Tuesday ~ 1pm ET / 10am PT ????

I don't mind google hangout but there is always some issue or whatever so
we know the apache irc channel works. We can start there and see how it
goes? We can pull transcripts too and associate to tickets if need be makes
it helpful for things.

~ Joestein

On Tue, Feb 24, 2015 at 11:10 AM, Jay Kreps <ja...@gmail.com> wrote:

> We'd talked about doing a Google Hangout to chat about this. What about
> generalizing that a little further...I actually think it would be good for
> everyone spending a reasonable chunk of their week on Kafka stuff to maybe
> sync up once a week. I think we could use time to talk through design
> stuff, make sure we are on top of code reviews, talk through any tricky
> issues, etc.
>
> We can make it publicly available so that any one can follow along who
> likes.
>
> Any interest in doing this? If so I'll try to set it up starting next week.
>
> -Jay
>
> On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Hi all,
> >
> > I've updated KIP page, fixed / aligned document structure. Also I added
> > some
> > very initial proposal for AdminClient so we have something to start from
> > while
> > discussing the KIP.
> >
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >
> > Thanks,
> > Andrii Biletskyi
> >
> > On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Jay,
> > >
> > > Re error messages: you are right, in most cases client will have enough
> > > context to show descriptive error message. My concern is that we will
> > have
> > > to
> > > add lots of new error codes for each possible error. Of course, we
> could
> > > reuse
> > > some of existing like UknownTopicOrPartitionCode, but we will also need
> > to
> > > add smth like: TopicAlreadyExistsCode, TopicConfigInvalid (both for
> topic
> > > name and config, and probably user would like to know what exactly
> > > is wrong in his config), InvalidReplicaAssignment, InternalError (e.g.
> > > zookeeper failure) etc.
> > > And this is only for TopicCommand, we will also need to add similar
> stuff
> > > for
> > > ReassignPartitions, PreferredReplica. So we'll end up with a large list
> > of
> > > error codes, used only in Admin protocol.
> > > Having said that, I agree my proposal is not consistent with other
> cases.
> > > Maybe we can find better solution or something in-between.
> > >
> > > Re Hangout chat: I think it is a great idea. This way we can move on
> > > faster.
> > > Let's agree somehow on date/time so people can join. Will work for me
> > this
> > > and
> > > next week almost anytime if agreed in advance.
> > >
> > > Thanks,
> > > Andrii
> > >
> > > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > >> Hey Andrii,
> > >>
> > >> Generally we can do good error handling without needing custom
> > server-side
> > >> messages. I.e. generally the client has the context to know that if it
> > got
> > >> an error that the topic doesn't exist to say "Topic X doesn't exist"
> > >> rather
> > >> than "error code 14" (or whatever). Maybe there are specific cases
> where
> > >> this is hard? If we want to add server-side error messages we really
> do
> > >> need to do this in a consistent way across the protocol.
> > >>
> > >> I still have a bunch of open questions here from my previous list. I
> > will
> > >> be out for the next few days for Strata though. Maybe we could do a
> > Google
> > >> Hangout chat on any open issues some time towards the end of next week
> > for
> > >> anyone interested in this ticket? I have a feeling that might progress
> > >> things a little faster than email--I think we could talk through those
> > >> issues I brought up fairly quickly...
> > >>
> > >> -Jay
> > >>
> > >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> > >> andrii.biletskyi@stealth.ly> wrote:
> > >>
> > >> > Hi all,
> > >> >
> > >> > I'm trying to address some of the issues which were mentioned
> earlier
> > >> about
> > >> > Admin RQ/RP format. One of those was about batching operations. What
> > if
> > >> we
> > >> > follow TopicCommand approach and let people specify topic-name by
> > >> regexp -
> > >> > would that cover most of the use cases?
> > >> >
> > >> > Secondly, is what information should we generally provide in Admin
> > >> > responses.
> > >> > I realize that Admin commands don't imply they will be used only in
> > CLI
> > >> > but,
> > >> > it seems to me, CLI is a very important client of this feature. In
> > this
> > >> > case,
> > >> > seems logical, we would like to provide users with rich experience
> in
> > >> terms
> > >> > of
> > >> > getting results / errors of the executed commands. Usually we supply
> > >> with
> > >> > responses only errorCode, which looks very limiting, in case of CLI
> we
> > >> may
> > >> > want to print human readable error description.
> > >> >
> > >> > So, taking into account previous item about batching, what do you
> > think
> > >> > about
> > >> > having smth like:
> > >> >
> > >> > ('create' doesn't support regexp)
> > >> > CreateTopicRequest => TopicName Partitions Replicas
> ReplicaAssignment
> > >> > [Config]
> > >> > CreateTopicResponse => ErrorCode ErrorDescription
> > >> >   ErrorCode => int16
> > >> >   ErrorDescription => string (empty if successful)
> > >> >
> > >> > AlterTopicRequest -> TopicNameRegexp Partitions ReplicaAssignment
> > >> > [AddedConfig] [DeletedConfig]
> > >> > AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
> > >> > CommandErrorCode CommandErrorDescription
> > >> >   CommandErrorCode => int16
> > >> >   CommandErrorDescription => string (nonempty in case of fatal
> error,
> > >> e.g.
> > >> > we couldn't get topics by regexp)
> > >> >
> > >> > DescribeTopicRequest -> TopicNameRegexp
> > >> > DescribeTopicResponse -> [TopicName TopicDescription ErrorCode
> > >> > ErrorDescription] CommandErrorCode CommandErrorDescription
> > >> >
> > >> > Also, any thoughts about our discussion regarding re-routing
> facility?
> > >> In
> > >> > my
> > >> > understanding, it is like between augmenting TopicMetadataRequest
> > >> > (to include at least controllerId) and implementing new generic
> > >> re-routing
> > >> > facility so sending messages to controller will be handled by it.
> > >> >
> > >> > Thanks,
> > >> > Andrii Biletskyi
> > >> >
> > >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> > >> > andrii.biletskyi@stealth.ly> wrote:
> > >> >
> > >> > > @Guozhang:
> > >> > > Thanks for your comments, I've answered some of those. The main
> > thing
> > >> is
> > >> > > having merged request for create-alter-delete-describe - I have
> some
> > >> > > concerns about this approach.
> > >> > >
> > >> > > @*Jay*:
> > >> > > I see that introduced ClusterMetadaRequest is also one of the
> > >> concerns.
> > >> > We
> > >> > > can solve it if we implement re-routing facility. But I agree with
> > >> > > Guozhang - it will make clients' internals a little bit easier but
> > >> this
> > >> > > seems to be a complex logic to implement and support then.
> > Especially
> > >> for
> > >> > > Fetch and Produce (even if we add re-routing later for these
> > >> requests).
> > >> > > Also people will tend to avoid this re-routing facility and hold
> > local
> > >> > > cluster cache to ensure their high-priority requests (which some
> of
> > >> the
> > >> > > admin requests are) not sent to some busy broker where they wait
> to
> > be
> > >> > > routed to the correct one.
> > >> > > As pointed out by Jun here (
> > >> > >
> > >> >
> > >>
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > >> > )
> > >> > > to solve the issue we might introduce a message type to get
> cluster
> > >> > state.
> > >> > > But I agree we can just update TopicMetadataResponse to include
> > >> > > controllerId (and probably smth else).
> > >> > > What are you thougths?
> > >> > >
> > >> > > Thanks,
> > >> > > Andrii
> > >> > >
> > >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <
> wangguoz@gmail.com>
> > >> > wrote:
> > >> > >
> > >> > >> I think for the topics commands we can actually merge
> > >> > >> create/alter/delete/describe as one request type since their
> > formats
> > >> are
> > >> > >> very much similar, and keep list-topics and others like
> > >> > >> partition-reassignment / preferred-leader-election as separate
> > >> request
> > >> > >> types, I also left some other comments on the RB (
> > >> > >> https://reviews.apache.org/r/29301/).
> > >> > >>
> > >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <ja...@gmail.com>
> > >> wrote:
> > >> > >>
> > >> > >> > Yeah I totally agree that we don't want to just have one "do
> > admin
> > >> > >> stuff"
> > >> > >> > command that has the union of all parameters.
> > >> > >> >
> > >> > >> > What I am saying is that command line tools are one client of
> the
> > >> > >> > administrative apis, but these will be used in a number of
> > >> scenarios
> > >> > so
> > >> > >> > they should make logical sense even in the absence of the
> command
> > >> line
> > >> > >> > tool. Hence comments like trying to clarify the relationship
> > >> between
> > >> > >> > ClusterMetadata and TopicMetadata...these kinds of things
> really
> > >> need
> > >> > >> to be
> > >> > >> > thought through.
> > >> > >> >
> > >> > >> > Hope that makes sense.
> > >> > >> >
> > >> > >> > -Jay
> > >> > >> >
> > >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> > >> > >> > andrii.biletskyi@stealth.ly> wrote:
> > >> > >> >
> > >> > >> > > Jay,
> > >> > >> > >
> > >> > >> > > Thanks for answering. You understood correctly, most of my
> > >> comments
> > >> > >> were
> > >> > >> > > related to your point 1) - about "well thought-out" apis.
> Also,
> > >> yes,
> > >> > >> as I
> > >> > >> > > understood we would like to introduce a single unified CLI
> tool
> > >> with
> > >> > >> > > centralized server-side request handling for lots of existing
> > >> ones
> > >> > >> (incl.
> > >> > >> > > TopicCommand, CommitOffsetChecker, ReassignPartitions, smth
> > else
> > >> if
> > >> > >> added
> > >> > >> > > in future). In our previous discussion (
> > >> > >> > > https://issues.apache.org/jira/browse/KAFKA-1694) people
> said
> > >> > they'd
> > >> > >> > > rather
> > >> > >> > > have a separate message for each command, so, yes, this way I
> > >> came
> > >> > to
> > >> > >> 1-1
> > >> > >> > > mapping between commands in the tool and protocol additions.
> > But
> > >> I
> > >> > >> might
> > >> > >> > be
> > >> > >> > > wrong.
> > >> > >> > > At the end I just try to start discussion how at least
> > generally
> > >> > this
> > >> > >> > > protocol should look like.
> > >> > >> > >
> > >> > >> > > Thanks,
> > >> > >> > > Andrii
> > >> > >> > >
> > >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
> > jay.kreps@gmail.com
> > >> >
> > >> > >> wrote:
> > >> > >> > >
> > >> > >> > > > Hey Andrii,
> > >> > >> > > >
> > >> > >> > > > To answer your earlier question we just really can't be
> > adding
> > >> any
> > >> > >> more
> > >> > >> > > > scala protocol objects. These things are super hard to
> > maintain
> > >> > >> because
> > >> > >> > > > they hand code the byte parsing and don't have good
> > versioning
> > >> > >> support.
> > >> > >> > > > Since we are already planning on converting we definitely
> > don't
> > >> > >> want to
> > >> > >> > > add
> > >> > >> > > > a ton more of these--they are total tech debt.
> > >> > >> > > >
> > >> > >> > > > What does it mean that the changes are isolated from the
> > >> current
> > >> > >> code
> > >> > >> > > base?
> > >> > >> > > >
> > >> > >> > > > I actually didn't understand the remaining comments, which
> of
> > >> the
> > >> > >> > points
> > >> > >> > > > are you responding to?
> > >> > >> > > >
> > >> > >> > > > Maybe one sticking point here is that it seems like you
> want
> > to
> > >> > make
> > >> > >> > some
> > >> > >> > > > kind of tool, and you have made a 1-1 mapping between
> > commands
> > >> you
> > >> > >> > > imagine
> > >> > >> > > > in the tool and protocol additions. I want to make sure we
> > >> don't
> > >> > do
> > >> > >> > that.
> > >> > >> > > > The protocol needs to be really really well thought out
> > against
> > >> > many
> > >> > >> > use
> > >> > >> > > > cases so it should make perfect logical sense in the
> absence
> > of
> > >> > >> knowing
> > >> > >> > > the
> > >> > >> > > > command line tool, right?
> > >> > >> > > >
> > >> > >> > > > -Jay
> > >> > >> > > >
> > >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
> > >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > >> > > >
> > >> > >> > > > > Hey Jay,
> > >> > >> > > > >
> > >> > >> > > > > I would like to continue this discussion as it seem there
> > is
> > >> no
> > >> > >> > > progress
> > >> > >> > > > > here.
> > >> > >> > > > >
> > >> > >> > > > > First of all, could you please explain what did you mean
> in
> > >> 2?
> > >> > How
> > >> > >> > > > exactly
> > >> > >> > > > > are we going to migrate to the new java protocol
> > definitions.
> > >> > And
> > >> > >> why
> > >> > >> > > > it's
> > >> > >> > > > > a blocker for centralized CLI?
> > >> > >> > > > >
> > >> > >> > > > > I agree with you, this feature includes lots of stuff,
> but
> > >> > >> thankfully
> > >> > >> > > > > almost all changes are isolated from the current code
> base,
> > >> > >> > > > > so the main thing, I think, we need to agree is RQ/RP
> > format.
> > >> > >> > > > > So how can we start discussion about the concrete
> messages
> > >> > format?
> > >> > >> > > > > Can we take (
> > >> > >> > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > >> > >> > > > > )
> > >> > >> > > > > as starting point?
> > >> > >> > > > >
> > >> > >> > > > > We had some doubts earlier whether it worth introducing
> one
> > >> > >> generic
> > >> > >> > > Admin
> > >> > >> > > > > Request for all commands (
> > >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > >> > >> > > > > )
> > >> > >> > > > > but then everybody agreed it would be better to have
> > separate
> > >> > >> message
> > >> > >> > > for
> > >> > >> > > > > each admin command. The Request part is really dictated
> > from
> > >> the
> > >> > >> > > command
> > >> > >> > > > > (e.g. TopicCommand) arguments itself, so the proposed
> > version
> > >> > >> should
> > >> > >> > be
> > >> > >> > > > > fine (let's put aside for now remarks about Optional
> type,
> > >> > >> batching,
> > >> > >> > > > > configs normalization - I agree with all of them).
> > >> > >> > > > > So the second part is Response. I see there are two cases
> > >> here.
> > >> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b) "Get"
> > requests -
> > >> > >> > > > > List/Describe...
> > >> > >> > > > >
> > >> > >> > > > > a) should only hold request result (regardless what we
> > decide
> > >> > >> about
> > >> > >> > > > > blocking/non-blocking commands execution).
> > >> > >> > > > > Usually we provide error code in response but since we
> will
> > >> use
> > >> > >> this
> > >> > >> > in
> > >> > >> > > > > interactive shell we need some human readable error
> > >> description
> > >> > -
> > >> > >> so
> > >> > >> > I
> > >> > >> > > > > added errorDesription field where you can at least leave
> > >> > >> > > > > exception.getMessage.
> > >> > >> > > > >
> > >> > >> > > > > b) in addition to previous item message should hold
> command
> > >> > >> specific
> > >> > >> > > > > response data. We can discuss in detail each of them but
> > >> let's
> > >> > for
> > >> > >> > now
> > >> > >> > > > > agree about the overall pattern.
> > >> > >> > > > >
> > >> > >> > > > > Thanks,
> > >> > >> > > > > Andrii Biletskyi
> > >> > >> > > > >
> > >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
> > >> jay.kreps@gmail.com
> > >> > >
> > >> > >> > > wrote:
> > >> > >> > > > >
> > >> > >> > > > > > Hey Joe,
> > >> > >> > > > > >
> > >> > >> > > > > > This is great. A few comments on KIP-4
> > >> > >> > > > > >
> > >> > >> > > > > > 1. This is much needed functionality, but there are a
> lot
> > >> of
> > >> > >> the so
> > >> > >> > > > let's
> > >> > >> > > > > > really think these protocols through. We really want to
> > >> end up
> > >> > >> > with a
> > >> > >> > > > set
> > >> > >> > > > > > of well thought-out, orthoganol apis. For this reason I
> > >> think
> > >> > >> it is
> > >> > >> > > > > really
> > >> > >> > > > > > important to think through the end state even if that
> > >> includes
> > >> > >> APIs
> > >> > >> > > we
> > >> > >> > > > > > won't implement in the first phase.
> > >> > >> > > > > >
> > >> > >> > > > > > 2. Let's please please please wait until we have
> switched
> > >> the
> > >> > >> > server
> > >> > >> > > > over
> > >> > >> > > > > > to the new java protocol definitions. If we add upteen
> > >> more ad
> > >> > >> hoc
> > >> > >> > > > scala
> > >> > >> > > > > > objects that is just generating more work for the
> > >> conversion
> > >> > we
> > >> > >> > know
> > >> > >> > > we
> > >> > >> > > > > > have to do.
> > >> > >> > > > > >
> > >> > >> > > > > > 3. This proposal introduces a new type of optional
> > >> parameter.
> > >> > >> This
> > >> > >> > is
> > >> > >> > > > > > inconsistent with everything else in the protocol where
> > we
> > >> use
> > >> > >> -1
> > >> > >> > or
> > >> > >> > > > some
> > >> > >> > > > > > other marker value. You could argue either way but
> let's
> > >> stick
> > >> > >> with
> > >> > >> > > > that
> > >> > >> > > > > > for consistency. For clients that implemented the
> > protocol
> > >> in
> > >> > a
> > >> > >> > > better
> > >> > >> > > > > way
> > >> > >> > > > > > than our scala code these basic primitives are hard to
> > >> change.
> > >> > >> > > > > >
> > >> > >> > > > > > 4. ClusterMetadata: This seems to duplicate
> > >> > TopicMetadataRequest
> > >> > >> > > which
> > >> > >> > > > > has
> > >> > >> > > > > > brokers, topics, and partitions. I think we should
> rename
> > >> that
> > >> > >> > > request
> > >> > >> > > > > > ClusterMetadataRequest (or just MetadataRequest) and
> > >> include
> > >> > >> the id
> > >> > >> > > of
> > >> > >> > > > > the
> > >> > >> > > > > > controller. Or are there other things we could add
> here?
> > >> > >> > > > > >
> > >> > >> > > > > > 5. We have a tendency to try to make a lot of requests
> > that
> > >> > can
> > >> > >> > only
> > >> > >> > > go
> > >> > >> > > > > to
> > >> > >> > > > > > particular nodes. This adds a lot of burden for client
> > >> > >> > > implementations
> > >> > >> > > > > (it
> > >> > >> > > > > > sounds easy but each discovery can fail in many parts
> so
> > it
> > >> > >> ends up
> > >> > >> > > > > being a
> > >> > >> > > > > > full state machine to do right). I think we should
> > consider
> > >> > >> making
> > >> > >> > > > admin
> > >> > >> > > > > > commands and ideally as many of the other apis as
> > possible
> > >> > >> > available
> > >> > >> > > on
> > >> > >> > > > > all
> > >> > >> > > > > > brokers and just redirect to the controller on the
> broker
> > >> > side.
> > >> > >> > > Perhaps
> > >> > >> > > > > > there would be a general way to encapsulate this
> > re-routing
> > >> > >> > behavior.
> > >> > >> > > > > >
> > >> > >> > > > > > 6. We should probably normalize the key value pairs
> used
> > >> for
> > >> > >> > configs
> > >> > >> > > > > rather
> > >> > >> > > > > > than embedding a new formatting. So two strings rather
> > than
> > >> > one
> > >> > >> > with
> > >> > >> > > an
> > >> > >> > > > > > internal equals sign.
> > >> > >> > > > > >
> > >> > >> > > > > > 7. Is the postcondition of these APIs that the command
> > has
> > >> > >> begun or
> > >> > >> > > > that
> > >> > >> > > > > > the command has been completed? It is a lot more usable
> > if
> > >> the
> > >> > >> > > command
> > >> > >> > > > > has
> > >> > >> > > > > > been completed so you know that if you create a topic
> and
> > >> then
> > >> > >> > > publish
> > >> > >> > > > to
> > >> > >> > > > > > it you won't get an exception about there being no such
> > >> topic.
> > >> > >> > > > > >
> > >> > >> > > > > > 8. Describe topic and list topics duplicate a lot of
> > stuff
> > >> in
> > >> > >> the
> > >> > >> > > > > metadata
> > >> > >> > > > > > request. Is there a reason to give back topics marked
> for
> > >> > >> > deletion? I
> > >> > >> > > > > feel
> > >> > >> > > > > > like if we just make the post-condition of the delete
> > >> command
> > >> > be
> > >> > >> > that
> > >> > >> > > > the
> > >> > >> > > > > > topic is deleted that will get rid of the need for this
> > >> right?
> > >> > >> And
> > >> > >> > it
> > >> > >> > > > > will
> > >> > >> > > > > > be much more intuitive.
> > >> > >> > > > > >
> > >> > >> > > > > > 9. Should we consider batching these requests? We have
> > >> > generally
> > >> > >> > > tried
> > >> > >> > > > to
> > >> > >> > > > > > allow multiple operations to be batched. My suspicion
> is
> > >> that
> > >> > >> > without
> > >> > >> > > > > this
> > >> > >> > > > > > we will get a lot of code that does something like
> > >> > >> > > > > >    for(topic: adminClient.listTopics())
> > >> > >> > > > > >       adminClient.describeTopic(topic)
> > >> > >> > > > > > this code will work great when you test on 5 topics but
> > >> not do
> > >> > >> as
> > >> > >> > > well
> > >> > >> > > > if
> > >> > >> > > > > > you have 50k.
> > >> > >> > > > > >
> > >> > >> > > > > > 10. I think we should also discuss how we want to
> expose
> > a
> > >> > >> > > programmatic
> > >> > >> > > > > JVM
> > >> > >> > > > > > client api for these operations. Currently people rely
> on
> > >> > >> > AdminUtils
> > >> > >> > > > > which
> > >> > >> > > > > > is totally sketchy. I think we probably need another
> > client
> > >> > >> under
> > >> > >> > > > > clients/
> > >> > >> > > > > > that exposes administrative functionality. We will need
> > >> this
> > >> > >> just
> > >> > >> > to
> > >> > >> > > > > > properly test the new apis, I suspect. We should figure
> > out
> > >> > that
> > >> > >> > API.
> > >> > >> > > > > >
> > >> > >> > > > > > 11. The other information that would be really useful
> to
> > >> get
> > >> > >> would
> > >> > >> > be
> > >> > >> > > > > > information about partitions--how much data is in the
> > >> > partition,
> > >> > >> > what
> > >> > >> > > > are
> > >> > >> > > > > > the segment offsets, what is the log-end offset (i.e.
> > last
> > >> > >> offset),
> > >> > >> > > > what
> > >> > >> > > > > is
> > >> > >> > > > > > the compaction point, etc. I think that done right this
> > >> would
> > >> > be
> > >> > >> > the
> > >> > >> > > > > > successor to the very awkward OffsetRequest we have
> > today.
> > >> > >> > > > > >
> > >> > >> > > > > > -Jay
> > >> > >> > > > > >
> > >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
> > >> > >> joe.stein@stealth.ly>
> > >> > >> > > > > wrote:
> > >> > >> > > > > >
> > >> > >> > > > > > > Hi, created a KIP
> > >> > >> > > > > > >
> > >> > >> > > > > > >
> > >> > >> > > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >> > >> > > > > > >
> > >> > >> > > > > > > JIRA
> https://issues.apache.org/jira/browse/KAFKA-1694
> > >> > >> > > > > > >
> > >> > >> > > > > > > /*******************************************
> > >> > >> > > > > > >  Joe Stein
> > >> > >> > > > > > >  Founder, Principal Consultant
> > >> > >> > > > > > >  Big Data Open Source Security LLC
> > >> > >> > > > > > >  http://www.stealth.ly
> > >> > >> > > > > > >  Twitter: @allthingshadoop <
> > >> > >> > http://www.twitter.com/allthingshadoop
> > >> > >> > > >
> > >> > >> > > > > > > ********************************************/
> > >> > >> > > > > > >
> > >> > >> > > > > >
> > >> > >> > > > >
> > >> > >> > > >
> > >> > >> > >
> > >> > >> >
> > >> > >>
> > >> > >>
> > >> > >>
> > >> > >> --
> > >> > >> -- Guozhang
> > >> > >>
> > >> > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
We'd talked about doing a Google Hangout to chat about this. What about
generalizing that a little further...I actually think it would be good for
everyone spending a reasonable chunk of their week on Kafka stuff to maybe
sync up once a week. I think we could use time to talk through design
stuff, make sure we are on top of code reviews, talk through any tricky
issues, etc.

We can make it publicly available so that any one can follow along who
likes.

Any interest in doing this? If so I'll try to set it up starting next week.

-Jay

On Tue, Feb 24, 2015 at 3:57 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Hi all,
>
> I've updated KIP page, fixed / aligned document structure. Also I added
> some
> very initial proposal for AdminClient so we have something to start from
> while
> discussing the KIP.
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>
> Thanks,
> Andrii Biletskyi
>
> On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Jay,
> >
> > Re error messages: you are right, in most cases client will have enough
> > context to show descriptive error message. My concern is that we will
> have
> > to
> > add lots of new error codes for each possible error. Of course, we could
> > reuse
> > some of existing like UknownTopicOrPartitionCode, but we will also need
> to
> > add smth like: TopicAlreadyExistsCode, TopicConfigInvalid (both for topic
> > name and config, and probably user would like to know what exactly
> > is wrong in his config), InvalidReplicaAssignment, InternalError (e.g.
> > zookeeper failure) etc.
> > And this is only for TopicCommand, we will also need to add similar stuff
> > for
> > ReassignPartitions, PreferredReplica. So we'll end up with a large list
> of
> > error codes, used only in Admin protocol.
> > Having said that, I agree my proposal is not consistent with other cases.
> > Maybe we can find better solution or something in-between.
> >
> > Re Hangout chat: I think it is a great idea. This way we can move on
> > faster.
> > Let's agree somehow on date/time so people can join. Will work for me
> this
> > and
> > next week almost anytime if agreed in advance.
> >
> > Thanks,
> > Andrii
> >
> > On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> >> Hey Andrii,
> >>
> >> Generally we can do good error handling without needing custom
> server-side
> >> messages. I.e. generally the client has the context to know that if it
> got
> >> an error that the topic doesn't exist to say "Topic X doesn't exist"
> >> rather
> >> than "error code 14" (or whatever). Maybe there are specific cases where
> >> this is hard? If we want to add server-side error messages we really do
> >> need to do this in a consistent way across the protocol.
> >>
> >> I still have a bunch of open questions here from my previous list. I
> will
> >> be out for the next few days for Strata though. Maybe we could do a
> Google
> >> Hangout chat on any open issues some time towards the end of next week
> for
> >> anyone interested in this ticket? I have a feeling that might progress
> >> things a little faster than email--I think we could talk through those
> >> issues I brought up fairly quickly...
> >>
> >> -Jay
> >>
> >> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> >> andrii.biletskyi@stealth.ly> wrote:
> >>
> >> > Hi all,
> >> >
> >> > I'm trying to address some of the issues which were mentioned earlier
> >> about
> >> > Admin RQ/RP format. One of those was about batching operations. What
> if
> >> we
> >> > follow TopicCommand approach and let people specify topic-name by
> >> regexp -
> >> > would that cover most of the use cases?
> >> >
> >> > Secondly, is what information should we generally provide in Admin
> >> > responses.
> >> > I realize that Admin commands don't imply they will be used only in
> CLI
> >> > but,
> >> > it seems to me, CLI is a very important client of this feature. In
> this
> >> > case,
> >> > seems logical, we would like to provide users with rich experience in
> >> terms
> >> > of
> >> > getting results / errors of the executed commands. Usually we supply
> >> with
> >> > responses only errorCode, which looks very limiting, in case of CLI we
> >> may
> >> > want to print human readable error description.
> >> >
> >> > So, taking into account previous item about batching, what do you
> think
> >> > about
> >> > having smth like:
> >> >
> >> > ('create' doesn't support regexp)
> >> > CreateTopicRequest => TopicName Partitions Replicas ReplicaAssignment
> >> > [Config]
> >> > CreateTopicResponse => ErrorCode ErrorDescription
> >> >   ErrorCode => int16
> >> >   ErrorDescription => string (empty if successful)
> >> >
> >> > AlterTopicRequest -> TopicNameRegexp Partitions ReplicaAssignment
> >> > [AddedConfig] [DeletedConfig]
> >> > AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
> >> > CommandErrorCode CommandErrorDescription
> >> >   CommandErrorCode => int16
> >> >   CommandErrorDescription => string (nonempty in case of fatal error,
> >> e.g.
> >> > we couldn't get topics by regexp)
> >> >
> >> > DescribeTopicRequest -> TopicNameRegexp
> >> > DescribeTopicResponse -> [TopicName TopicDescription ErrorCode
> >> > ErrorDescription] CommandErrorCode CommandErrorDescription
> >> >
> >> > Also, any thoughts about our discussion regarding re-routing facility?
> >> In
> >> > my
> >> > understanding, it is like between augmenting TopicMetadataRequest
> >> > (to include at least controllerId) and implementing new generic
> >> re-routing
> >> > facility so sending messages to controller will be handled by it.
> >> >
> >> > Thanks,
> >> > Andrii Biletskyi
> >> >
> >> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> >> > andrii.biletskyi@stealth.ly> wrote:
> >> >
> >> > > @Guozhang:
> >> > > Thanks for your comments, I've answered some of those. The main
> thing
> >> is
> >> > > having merged request for create-alter-delete-describe - I have some
> >> > > concerns about this approach.
> >> > >
> >> > > @*Jay*:
> >> > > I see that introduced ClusterMetadaRequest is also one of the
> >> concerns.
> >> > We
> >> > > can solve it if we implement re-routing facility. But I agree with
> >> > > Guozhang - it will make clients' internals a little bit easier but
> >> this
> >> > > seems to be a complex logic to implement and support then.
> Especially
> >> for
> >> > > Fetch and Produce (even if we add re-routing later for these
> >> requests).
> >> > > Also people will tend to avoid this re-routing facility and hold
> local
> >> > > cluster cache to ensure their high-priority requests (which some of
> >> the
> >> > > admin requests are) not sent to some busy broker where they wait to
> be
> >> > > routed to the correct one.
> >> > > As pointed out by Jun here (
> >> > >
> >> >
> >>
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> >> > )
> >> > > to solve the issue we might introduce a message type to get cluster
> >> > state.
> >> > > But I agree we can just update TopicMetadataResponse to include
> >> > > controllerId (and probably smth else).
> >> > > What are you thougths?
> >> > >
> >> > > Thanks,
> >> > > Andrii
> >> > >
> >> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <wa...@gmail.com>
> >> > wrote:
> >> > >
> >> > >> I think for the topics commands we can actually merge
> >> > >> create/alter/delete/describe as one request type since their
> formats
> >> are
> >> > >> very much similar, and keep list-topics and others like
> >> > >> partition-reassignment / preferred-leader-election as separate
> >> request
> >> > >> types, I also left some other comments on the RB (
> >> > >> https://reviews.apache.org/r/29301/).
> >> > >>
> >> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <ja...@gmail.com>
> >> wrote:
> >> > >>
> >> > >> > Yeah I totally agree that we don't want to just have one "do
> admin
> >> > >> stuff"
> >> > >> > command that has the union of all parameters.
> >> > >> >
> >> > >> > What I am saying is that command line tools are one client of the
> >> > >> > administrative apis, but these will be used in a number of
> >> scenarios
> >> > so
> >> > >> > they should make logical sense even in the absence of the command
> >> line
> >> > >> > tool. Hence comments like trying to clarify the relationship
> >> between
> >> > >> > ClusterMetadata and TopicMetadata...these kinds of things really
> >> need
> >> > >> to be
> >> > >> > thought through.
> >> > >> >
> >> > >> > Hope that makes sense.
> >> > >> >
> >> > >> > -Jay
> >> > >> >
> >> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> >> > >> > andrii.biletskyi@stealth.ly> wrote:
> >> > >> >
> >> > >> > > Jay,
> >> > >> > >
> >> > >> > > Thanks for answering. You understood correctly, most of my
> >> comments
> >> > >> were
> >> > >> > > related to your point 1) - about "well thought-out" apis. Also,
> >> yes,
> >> > >> as I
> >> > >> > > understood we would like to introduce a single unified CLI tool
> >> with
> >> > >> > > centralized server-side request handling for lots of existing
> >> ones
> >> > >> (incl.
> >> > >> > > TopicCommand, CommitOffsetChecker, ReassignPartitions, smth
> else
> >> if
> >> > >> added
> >> > >> > > in future). In our previous discussion (
> >> > >> > > https://issues.apache.org/jira/browse/KAFKA-1694) people said
> >> > they'd
> >> > >> > > rather
> >> > >> > > have a separate message for each command, so, yes, this way I
> >> came
> >> > to
> >> > >> 1-1
> >> > >> > > mapping between commands in the tool and protocol additions.
> But
> >> I
> >> > >> might
> >> > >> > be
> >> > >> > > wrong.
> >> > >> > > At the end I just try to start discussion how at least
> generally
> >> > this
> >> > >> > > protocol should look like.
> >> > >> > >
> >> > >> > > Thanks,
> >> > >> > > Andrii
> >> > >> > >
> >> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <
> jay.kreps@gmail.com
> >> >
> >> > >> wrote:
> >> > >> > >
> >> > >> > > > Hey Andrii,
> >> > >> > > >
> >> > >> > > > To answer your earlier question we just really can't be
> adding
> >> any
> >> > >> more
> >> > >> > > > scala protocol objects. These things are super hard to
> maintain
> >> > >> because
> >> > >> > > > they hand code the byte parsing and don't have good
> versioning
> >> > >> support.
> >> > >> > > > Since we are already planning on converting we definitely
> don't
> >> > >> want to
> >> > >> > > add
> >> > >> > > > a ton more of these--they are total tech debt.
> >> > >> > > >
> >> > >> > > > What does it mean that the changes are isolated from the
> >> current
> >> > >> code
> >> > >> > > base?
> >> > >> > > >
> >> > >> > > > I actually didn't understand the remaining comments, which of
> >> the
> >> > >> > points
> >> > >> > > > are you responding to?
> >> > >> > > >
> >> > >> > > > Maybe one sticking point here is that it seems like you want
> to
> >> > make
> >> > >> > some
> >> > >> > > > kind of tool, and you have made a 1-1 mapping between
> commands
> >> you
> >> > >> > > imagine
> >> > >> > > > in the tool and protocol additions. I want to make sure we
> >> don't
> >> > do
> >> > >> > that.
> >> > >> > > > The protocol needs to be really really well thought out
> against
> >> > many
> >> > >> > use
> >> > >> > > > cases so it should make perfect logical sense in the absence
> of
> >> > >> knowing
> >> > >> > > the
> >> > >> > > > command line tool, right?
> >> > >> > > >
> >> > >> > > > -Jay
> >> > >> > > >
> >> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
> >> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> >> > >> > > >
> >> > >> > > > > Hey Jay,
> >> > >> > > > >
> >> > >> > > > > I would like to continue this discussion as it seem there
> is
> >> no
> >> > >> > > progress
> >> > >> > > > > here.
> >> > >> > > > >
> >> > >> > > > > First of all, could you please explain what did you mean in
> >> 2?
> >> > How
> >> > >> > > > exactly
> >> > >> > > > > are we going to migrate to the new java protocol
> definitions.
> >> > And
> >> > >> why
> >> > >> > > > it's
> >> > >> > > > > a blocker for centralized CLI?
> >> > >> > > > >
> >> > >> > > > > I agree with you, this feature includes lots of stuff, but
> >> > >> thankfully
> >> > >> > > > > almost all changes are isolated from the current code base,
> >> > >> > > > > so the main thing, I think, we need to agree is RQ/RP
> format.
> >> > >> > > > > So how can we start discussion about the concrete messages
> >> > format?
> >> > >> > > > > Can we take (
> >> > >> > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> >> > >> > > > > )
> >> > >> > > > > as starting point?
> >> > >> > > > >
> >> > >> > > > > We had some doubts earlier whether it worth introducing one
> >> > >> generic
> >> > >> > > Admin
> >> > >> > > > > Request for all commands (
> >> > >> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> >> > >> > > > > )
> >> > >> > > > > but then everybody agreed it would be better to have
> separate
> >> > >> message
> >> > >> > > for
> >> > >> > > > > each admin command. The Request part is really dictated
> from
> >> the
> >> > >> > > command
> >> > >> > > > > (e.g. TopicCommand) arguments itself, so the proposed
> version
> >> > >> should
> >> > >> > be
> >> > >> > > > > fine (let's put aside for now remarks about Optional type,
> >> > >> batching,
> >> > >> > > > > configs normalization - I agree with all of them).
> >> > >> > > > > So the second part is Response. I see there are two cases
> >> here.
> >> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b) "Get"
> requests -
> >> > >> > > > > List/Describe...
> >> > >> > > > >
> >> > >> > > > > a) should only hold request result (regardless what we
> decide
> >> > >> about
> >> > >> > > > > blocking/non-blocking commands execution).
> >> > >> > > > > Usually we provide error code in response but since we will
> >> use
> >> > >> this
> >> > >> > in
> >> > >> > > > > interactive shell we need some human readable error
> >> description
> >> > -
> >> > >> so
> >> > >> > I
> >> > >> > > > > added errorDesription field where you can at least leave
> >> > >> > > > > exception.getMessage.
> >> > >> > > > >
> >> > >> > > > > b) in addition to previous item message should hold command
> >> > >> specific
> >> > >> > > > > response data. We can discuss in detail each of them but
> >> let's
> >> > for
> >> > >> > now
> >> > >> > > > > agree about the overall pattern.
> >> > >> > > > >
> >> > >> > > > > Thanks,
> >> > >> > > > > Andrii Biletskyi
> >> > >> > > > >
> >> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
> >> jay.kreps@gmail.com
> >> > >
> >> > >> > > wrote:
> >> > >> > > > >
> >> > >> > > > > > Hey Joe,
> >> > >> > > > > >
> >> > >> > > > > > This is great. A few comments on KIP-4
> >> > >> > > > > >
> >> > >> > > > > > 1. This is much needed functionality, but there are a lot
> >> of
> >> > >> the so
> >> > >> > > > let's
> >> > >> > > > > > really think these protocols through. We really want to
> >> end up
> >> > >> > with a
> >> > >> > > > set
> >> > >> > > > > > of well thought-out, orthoganol apis. For this reason I
> >> think
> >> > >> it is
> >> > >> > > > > really
> >> > >> > > > > > important to think through the end state even if that
> >> includes
> >> > >> APIs
> >> > >> > > we
> >> > >> > > > > > won't implement in the first phase.
> >> > >> > > > > >
> >> > >> > > > > > 2. Let's please please please wait until we have switched
> >> the
> >> > >> > server
> >> > >> > > > over
> >> > >> > > > > > to the new java protocol definitions. If we add upteen
> >> more ad
> >> > >> hoc
> >> > >> > > > scala
> >> > >> > > > > > objects that is just generating more work for the
> >> conversion
> >> > we
> >> > >> > know
> >> > >> > > we
> >> > >> > > > > > have to do.
> >> > >> > > > > >
> >> > >> > > > > > 3. This proposal introduces a new type of optional
> >> parameter.
> >> > >> This
> >> > >> > is
> >> > >> > > > > > inconsistent with everything else in the protocol where
> we
> >> use
> >> > >> -1
> >> > >> > or
> >> > >> > > > some
> >> > >> > > > > > other marker value. You could argue either way but let's
> >> stick
> >> > >> with
> >> > >> > > > that
> >> > >> > > > > > for consistency. For clients that implemented the
> protocol
> >> in
> >> > a
> >> > >> > > better
> >> > >> > > > > way
> >> > >> > > > > > than our scala code these basic primitives are hard to
> >> change.
> >> > >> > > > > >
> >> > >> > > > > > 4. ClusterMetadata: This seems to duplicate
> >> > TopicMetadataRequest
> >> > >> > > which
> >> > >> > > > > has
> >> > >> > > > > > brokers, topics, and partitions. I think we should rename
> >> that
> >> > >> > > request
> >> > >> > > > > > ClusterMetadataRequest (or just MetadataRequest) and
> >> include
> >> > >> the id
> >> > >> > > of
> >> > >> > > > > the
> >> > >> > > > > > controller. Or are there other things we could add here?
> >> > >> > > > > >
> >> > >> > > > > > 5. We have a tendency to try to make a lot of requests
> that
> >> > can
> >> > >> > only
> >> > >> > > go
> >> > >> > > > > to
> >> > >> > > > > > particular nodes. This adds a lot of burden for client
> >> > >> > > implementations
> >> > >> > > > > (it
> >> > >> > > > > > sounds easy but each discovery can fail in many parts so
> it
> >> > >> ends up
> >> > >> > > > > being a
> >> > >> > > > > > full state machine to do right). I think we should
> consider
> >> > >> making
> >> > >> > > > admin
> >> > >> > > > > > commands and ideally as many of the other apis as
> possible
> >> > >> > available
> >> > >> > > on
> >> > >> > > > > all
> >> > >> > > > > > brokers and just redirect to the controller on the broker
> >> > side.
> >> > >> > > Perhaps
> >> > >> > > > > > there would be a general way to encapsulate this
> re-routing
> >> > >> > behavior.
> >> > >> > > > > >
> >> > >> > > > > > 6. We should probably normalize the key value pairs used
> >> for
> >> > >> > configs
> >> > >> > > > > rather
> >> > >> > > > > > than embedding a new formatting. So two strings rather
> than
> >> > one
> >> > >> > with
> >> > >> > > an
> >> > >> > > > > > internal equals sign.
> >> > >> > > > > >
> >> > >> > > > > > 7. Is the postcondition of these APIs that the command
> has
> >> > >> begun or
> >> > >> > > > that
> >> > >> > > > > > the command has been completed? It is a lot more usable
> if
> >> the
> >> > >> > > command
> >> > >> > > > > has
> >> > >> > > > > > been completed so you know that if you create a topic and
> >> then
> >> > >> > > publish
> >> > >> > > > to
> >> > >> > > > > > it you won't get an exception about there being no such
> >> topic.
> >> > >> > > > > >
> >> > >> > > > > > 8. Describe topic and list topics duplicate a lot of
> stuff
> >> in
> >> > >> the
> >> > >> > > > > metadata
> >> > >> > > > > > request. Is there a reason to give back topics marked for
> >> > >> > deletion? I
> >> > >> > > > > feel
> >> > >> > > > > > like if we just make the post-condition of the delete
> >> command
> >> > be
> >> > >> > that
> >> > >> > > > the
> >> > >> > > > > > topic is deleted that will get rid of the need for this
> >> right?
> >> > >> And
> >> > >> > it
> >> > >> > > > > will
> >> > >> > > > > > be much more intuitive.
> >> > >> > > > > >
> >> > >> > > > > > 9. Should we consider batching these requests? We have
> >> > generally
> >> > >> > > tried
> >> > >> > > > to
> >> > >> > > > > > allow multiple operations to be batched. My suspicion is
> >> that
> >> > >> > without
> >> > >> > > > > this
> >> > >> > > > > > we will get a lot of code that does something like
> >> > >> > > > > >    for(topic: adminClient.listTopics())
> >> > >> > > > > >       adminClient.describeTopic(topic)
> >> > >> > > > > > this code will work great when you test on 5 topics but
> >> not do
> >> > >> as
> >> > >> > > well
> >> > >> > > > if
> >> > >> > > > > > you have 50k.
> >> > >> > > > > >
> >> > >> > > > > > 10. I think we should also discuss how we want to expose
> a
> >> > >> > > programmatic
> >> > >> > > > > JVM
> >> > >> > > > > > client api for these operations. Currently people rely on
> >> > >> > AdminUtils
> >> > >> > > > > which
> >> > >> > > > > > is totally sketchy. I think we probably need another
> client
> >> > >> under
> >> > >> > > > > clients/
> >> > >> > > > > > that exposes administrative functionality. We will need
> >> this
> >> > >> just
> >> > >> > to
> >> > >> > > > > > properly test the new apis, I suspect. We should figure
> out
> >> > that
> >> > >> > API.
> >> > >> > > > > >
> >> > >> > > > > > 11. The other information that would be really useful to
> >> get
> >> > >> would
> >> > >> > be
> >> > >> > > > > > information about partitions--how much data is in the
> >> > partition,
> >> > >> > what
> >> > >> > > > are
> >> > >> > > > > > the segment offsets, what is the log-end offset (i.e.
> last
> >> > >> offset),
> >> > >> > > > what
> >> > >> > > > > is
> >> > >> > > > > > the compaction point, etc. I think that done right this
> >> would
> >> > be
> >> > >> > the
> >> > >> > > > > > successor to the very awkward OffsetRequest we have
> today.
> >> > >> > > > > >
> >> > >> > > > > > -Jay
> >> > >> > > > > >
> >> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
> >> > >> joe.stein@stealth.ly>
> >> > >> > > > > wrote:
> >> > >> > > > > >
> >> > >> > > > > > > Hi, created a KIP
> >> > >> > > > > > >
> >> > >> > > > > > >
> >> > >> > > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> > >> > > > > > >
> >> > >> > > > > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> >> > >> > > > > > >
> >> > >> > > > > > > /*******************************************
> >> > >> > > > > > >  Joe Stein
> >> > >> > > > > > >  Founder, Principal Consultant
> >> > >> > > > > > >  Big Data Open Source Security LLC
> >> > >> > > > > > >  http://www.stealth.ly
> >> > >> > > > > > >  Twitter: @allthingshadoop <
> >> > >> > http://www.twitter.com/allthingshadoop
> >> > >> > > >
> >> > >> > > > > > > ********************************************/
> >> > >> > > > > > >
> >> > >> > > > > >
> >> > >> > > > >
> >> > >> > > >
> >> > >> > >
> >> > >> >
> >> > >>
> >> > >>
> >> > >>
> >> > >> --
> >> > >> -- Guozhang
> >> > >>
> >> > >
> >> > >
> >> >
> >>
> >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Hi all,

I've updated KIP page, fixed / aligned document structure. Also I added some
very initial proposal for AdminClient so we have something to start from
while
discussing the KIP.

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

Thanks,
Andrii Biletskyi

On Wed, Feb 18, 2015 at 9:01 PM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Jay,
>
> Re error messages: you are right, in most cases client will have enough
> context to show descriptive error message. My concern is that we will have
> to
> add lots of new error codes for each possible error. Of course, we could
> reuse
> some of existing like UknownTopicOrPartitionCode, but we will also need to
> add smth like: TopicAlreadyExistsCode, TopicConfigInvalid (both for topic
> name and config, and probably user would like to know what exactly
> is wrong in his config), InvalidReplicaAssignment, InternalError (e.g.
> zookeeper failure) etc.
> And this is only for TopicCommand, we will also need to add similar stuff
> for
> ReassignPartitions, PreferredReplica. So we'll end up with a large list of
> error codes, used only in Admin protocol.
> Having said that, I agree my proposal is not consistent with other cases.
> Maybe we can find better solution or something in-between.
>
> Re Hangout chat: I think it is a great idea. This way we can move on
> faster.
> Let's agree somehow on date/time so people can join. Will work for me this
> and
> next week almost anytime if agreed in advance.
>
> Thanks,
> Andrii
>
> On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <ja...@gmail.com> wrote:
>
>> Hey Andrii,
>>
>> Generally we can do good error handling without needing custom server-side
>> messages. I.e. generally the client has the context to know that if it got
>> an error that the topic doesn't exist to say "Topic X doesn't exist"
>> rather
>> than "error code 14" (or whatever). Maybe there are specific cases where
>> this is hard? If we want to add server-side error messages we really do
>> need to do this in a consistent way across the protocol.
>>
>> I still have a bunch of open questions here from my previous list. I will
>> be out for the next few days for Strata though. Maybe we could do a Google
>> Hangout chat on any open issues some time towards the end of next week for
>> anyone interested in this ticket? I have a feeling that might progress
>> things a little faster than email--I think we could talk through those
>> issues I brought up fairly quickly...
>>
>> -Jay
>>
>> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
>> andrii.biletskyi@stealth.ly> wrote:
>>
>> > Hi all,
>> >
>> > I'm trying to address some of the issues which were mentioned earlier
>> about
>> > Admin RQ/RP format. One of those was about batching operations. What if
>> we
>> > follow TopicCommand approach and let people specify topic-name by
>> regexp -
>> > would that cover most of the use cases?
>> >
>> > Secondly, is what information should we generally provide in Admin
>> > responses.
>> > I realize that Admin commands don't imply they will be used only in CLI
>> > but,
>> > it seems to me, CLI is a very important client of this feature. In this
>> > case,
>> > seems logical, we would like to provide users with rich experience in
>> terms
>> > of
>> > getting results / errors of the executed commands. Usually we supply
>> with
>> > responses only errorCode, which looks very limiting, in case of CLI we
>> may
>> > want to print human readable error description.
>> >
>> > So, taking into account previous item about batching, what do you think
>> > about
>> > having smth like:
>> >
>> > ('create' doesn't support regexp)
>> > CreateTopicRequest => TopicName Partitions Replicas ReplicaAssignment
>> > [Config]
>> > CreateTopicResponse => ErrorCode ErrorDescription
>> >   ErrorCode => int16
>> >   ErrorDescription => string (empty if successful)
>> >
>> > AlterTopicRequest -> TopicNameRegexp Partitions ReplicaAssignment
>> > [AddedConfig] [DeletedConfig]
>> > AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
>> > CommandErrorCode CommandErrorDescription
>> >   CommandErrorCode => int16
>> >   CommandErrorDescription => string (nonempty in case of fatal error,
>> e.g.
>> > we couldn't get topics by regexp)
>> >
>> > DescribeTopicRequest -> TopicNameRegexp
>> > DescribeTopicResponse -> [TopicName TopicDescription ErrorCode
>> > ErrorDescription] CommandErrorCode CommandErrorDescription
>> >
>> > Also, any thoughts about our discussion regarding re-routing facility?
>> In
>> > my
>> > understanding, it is like between augmenting TopicMetadataRequest
>> > (to include at least controllerId) and implementing new generic
>> re-routing
>> > facility so sending messages to controller will be handled by it.
>> >
>> > Thanks,
>> > Andrii Biletskyi
>> >
>> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
>> > andrii.biletskyi@stealth.ly> wrote:
>> >
>> > > @Guozhang:
>> > > Thanks for your comments, I've answered some of those. The main thing
>> is
>> > > having merged request for create-alter-delete-describe - I have some
>> > > concerns about this approach.
>> > >
>> > > @*Jay*:
>> > > I see that introduced ClusterMetadaRequest is also one of the
>> concerns.
>> > We
>> > > can solve it if we implement re-routing facility. But I agree with
>> > > Guozhang - it will make clients' internals a little bit easier but
>> this
>> > > seems to be a complex logic to implement and support then. Especially
>> for
>> > > Fetch and Produce (even if we add re-routing later for these
>> requests).
>> > > Also people will tend to avoid this re-routing facility and hold local
>> > > cluster cache to ensure their high-priority requests (which some of
>> the
>> > > admin requests are) not sent to some busy broker where they wait to be
>> > > routed to the correct one.
>> > > As pointed out by Jun here (
>> > >
>> >
>> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
>> > )
>> > > to solve the issue we might introduce a message type to get cluster
>> > state.
>> > > But I agree we can just update TopicMetadataResponse to include
>> > > controllerId (and probably smth else).
>> > > What are you thougths?
>> > >
>> > > Thanks,
>> > > Andrii
>> > >
>> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <wa...@gmail.com>
>> > wrote:
>> > >
>> > >> I think for the topics commands we can actually merge
>> > >> create/alter/delete/describe as one request type since their formats
>> are
>> > >> very much similar, and keep list-topics and others like
>> > >> partition-reassignment / preferred-leader-election as separate
>> request
>> > >> types, I also left some other comments on the RB (
>> > >> https://reviews.apache.org/r/29301/).
>> > >>
>> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <ja...@gmail.com>
>> wrote:
>> > >>
>> > >> > Yeah I totally agree that we don't want to just have one "do admin
>> > >> stuff"
>> > >> > command that has the union of all parameters.
>> > >> >
>> > >> > What I am saying is that command line tools are one client of the
>> > >> > administrative apis, but these will be used in a number of
>> scenarios
>> > so
>> > >> > they should make logical sense even in the absence of the command
>> line
>> > >> > tool. Hence comments like trying to clarify the relationship
>> between
>> > >> > ClusterMetadata and TopicMetadata...these kinds of things really
>> need
>> > >> to be
>> > >> > thought through.
>> > >> >
>> > >> > Hope that makes sense.
>> > >> >
>> > >> > -Jay
>> > >> >
>> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
>> > >> > andrii.biletskyi@stealth.ly> wrote:
>> > >> >
>> > >> > > Jay,
>> > >> > >
>> > >> > > Thanks for answering. You understood correctly, most of my
>> comments
>> > >> were
>> > >> > > related to your point 1) - about "well thought-out" apis. Also,
>> yes,
>> > >> as I
>> > >> > > understood we would like to introduce a single unified CLI tool
>> with
>> > >> > > centralized server-side request handling for lots of existing
>> ones
>> > >> (incl.
>> > >> > > TopicCommand, CommitOffsetChecker, ReassignPartitions, smth else
>> if
>> > >> added
>> > >> > > in future). In our previous discussion (
>> > >> > > https://issues.apache.org/jira/browse/KAFKA-1694) people said
>> > they'd
>> > >> > > rather
>> > >> > > have a separate message for each command, so, yes, this way I
>> came
>> > to
>> > >> 1-1
>> > >> > > mapping between commands in the tool and protocol additions. But
>> I
>> > >> might
>> > >> > be
>> > >> > > wrong.
>> > >> > > At the end I just try to start discussion how at least generally
>> > this
>> > >> > > protocol should look like.
>> > >> > >
>> > >> > > Thanks,
>> > >> > > Andrii
>> > >> > >
>> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <jay.kreps@gmail.com
>> >
>> > >> wrote:
>> > >> > >
>> > >> > > > Hey Andrii,
>> > >> > > >
>> > >> > > > To answer your earlier question we just really can't be adding
>> any
>> > >> more
>> > >> > > > scala protocol objects. These things are super hard to maintain
>> > >> because
>> > >> > > > they hand code the byte parsing and don't have good versioning
>> > >> support.
>> > >> > > > Since we are already planning on converting we definitely don't
>> > >> want to
>> > >> > > add
>> > >> > > > a ton more of these--they are total tech debt.
>> > >> > > >
>> > >> > > > What does it mean that the changes are isolated from the
>> current
>> > >> code
>> > >> > > base?
>> > >> > > >
>> > >> > > > I actually didn't understand the remaining comments, which of
>> the
>> > >> > points
>> > >> > > > are you responding to?
>> > >> > > >
>> > >> > > > Maybe one sticking point here is that it seems like you want to
>> > make
>> > >> > some
>> > >> > > > kind of tool, and you have made a 1-1 mapping between commands
>> you
>> > >> > > imagine
>> > >> > > > in the tool and protocol additions. I want to make sure we
>> don't
>> > do
>> > >> > that.
>> > >> > > > The protocol needs to be really really well thought out against
>> > many
>> > >> > use
>> > >> > > > cases so it should make perfect logical sense in the absence of
>> > >> knowing
>> > >> > > the
>> > >> > > > command line tool, right?
>> > >> > > >
>> > >> > > > -Jay
>> > >> > > >
>> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
>> > >> > > > andrii.biletskyi@stealth.ly> wrote:
>> > >> > > >
>> > >> > > > > Hey Jay,
>> > >> > > > >
>> > >> > > > > I would like to continue this discussion as it seem there is
>> no
>> > >> > > progress
>> > >> > > > > here.
>> > >> > > > >
>> > >> > > > > First of all, could you please explain what did you mean in
>> 2?
>> > How
>> > >> > > > exactly
>> > >> > > > > are we going to migrate to the new java protocol definitions.
>> > And
>> > >> why
>> > >> > > > it's
>> > >> > > > > a blocker for centralized CLI?
>> > >> > > > >
>> > >> > > > > I agree with you, this feature includes lots of stuff, but
>> > >> thankfully
>> > >> > > > > almost all changes are isolated from the current code base,
>> > >> > > > > so the main thing, I think, we need to agree is RQ/RP format.
>> > >> > > > > So how can we start discussion about the concrete messages
>> > format?
>> > >> > > > > Can we take (
>> > >> > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
>> > >> > > > > )
>> > >> > > > > as starting point?
>> > >> > > > >
>> > >> > > > > We had some doubts earlier whether it worth introducing one
>> > >> generic
>> > >> > > Admin
>> > >> > > > > Request for all commands (
>> > >> > > > https://issues.apache.org/jira/browse/KAFKA-1694
>> > >> > > > > )
>> > >> > > > > but then everybody agreed it would be better to have separate
>> > >> message
>> > >> > > for
>> > >> > > > > each admin command. The Request part is really dictated from
>> the
>> > >> > > command
>> > >> > > > > (e.g. TopicCommand) arguments itself, so the proposed version
>> > >> should
>> > >> > be
>> > >> > > > > fine (let's put aside for now remarks about Optional type,
>> > >> batching,
>> > >> > > > > configs normalization - I agree with all of them).
>> > >> > > > > So the second part is Response. I see there are two cases
>> here.
>> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b) "Get" requests -
>> > >> > > > > List/Describe...
>> > >> > > > >
>> > >> > > > > a) should only hold request result (regardless what we decide
>> > >> about
>> > >> > > > > blocking/non-blocking commands execution).
>> > >> > > > > Usually we provide error code in response but since we will
>> use
>> > >> this
>> > >> > in
>> > >> > > > > interactive shell we need some human readable error
>> description
>> > -
>> > >> so
>> > >> > I
>> > >> > > > > added errorDesription field where you can at least leave
>> > >> > > > > exception.getMessage.
>> > >> > > > >
>> > >> > > > > b) in addition to previous item message should hold command
>> > >> specific
>> > >> > > > > response data. We can discuss in detail each of them but
>> let's
>> > for
>> > >> > now
>> > >> > > > > agree about the overall pattern.
>> > >> > > > >
>> > >> > > > > Thanks,
>> > >> > > > > Andrii Biletskyi
>> > >> > > > >
>> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
>> jay.kreps@gmail.com
>> > >
>> > >> > > wrote:
>> > >> > > > >
>> > >> > > > > > Hey Joe,
>> > >> > > > > >
>> > >> > > > > > This is great. A few comments on KIP-4
>> > >> > > > > >
>> > >> > > > > > 1. This is much needed functionality, but there are a lot
>> of
>> > >> the so
>> > >> > > > let's
>> > >> > > > > > really think these protocols through. We really want to
>> end up
>> > >> > with a
>> > >> > > > set
>> > >> > > > > > of well thought-out, orthoganol apis. For this reason I
>> think
>> > >> it is
>> > >> > > > > really
>> > >> > > > > > important to think through the end state even if that
>> includes
>> > >> APIs
>> > >> > > we
>> > >> > > > > > won't implement in the first phase.
>> > >> > > > > >
>> > >> > > > > > 2. Let's please please please wait until we have switched
>> the
>> > >> > server
>> > >> > > > over
>> > >> > > > > > to the new java protocol definitions. If we add upteen
>> more ad
>> > >> hoc
>> > >> > > > scala
>> > >> > > > > > objects that is just generating more work for the
>> conversion
>> > we
>> > >> > know
>> > >> > > we
>> > >> > > > > > have to do.
>> > >> > > > > >
>> > >> > > > > > 3. This proposal introduces a new type of optional
>> parameter.
>> > >> This
>> > >> > is
>> > >> > > > > > inconsistent with everything else in the protocol where we
>> use
>> > >> -1
>> > >> > or
>> > >> > > > some
>> > >> > > > > > other marker value. You could argue either way but let's
>> stick
>> > >> with
>> > >> > > > that
>> > >> > > > > > for consistency. For clients that implemented the protocol
>> in
>> > a
>> > >> > > better
>> > >> > > > > way
>> > >> > > > > > than our scala code these basic primitives are hard to
>> change.
>> > >> > > > > >
>> > >> > > > > > 4. ClusterMetadata: This seems to duplicate
>> > TopicMetadataRequest
>> > >> > > which
>> > >> > > > > has
>> > >> > > > > > brokers, topics, and partitions. I think we should rename
>> that
>> > >> > > request
>> > >> > > > > > ClusterMetadataRequest (or just MetadataRequest) and
>> include
>> > >> the id
>> > >> > > of
>> > >> > > > > the
>> > >> > > > > > controller. Or are there other things we could add here?
>> > >> > > > > >
>> > >> > > > > > 5. We have a tendency to try to make a lot of requests that
>> > can
>> > >> > only
>> > >> > > go
>> > >> > > > > to
>> > >> > > > > > particular nodes. This adds a lot of burden for client
>> > >> > > implementations
>> > >> > > > > (it
>> > >> > > > > > sounds easy but each discovery can fail in many parts so it
>> > >> ends up
>> > >> > > > > being a
>> > >> > > > > > full state machine to do right). I think we should consider
>> > >> making
>> > >> > > > admin
>> > >> > > > > > commands and ideally as many of the other apis as possible
>> > >> > available
>> > >> > > on
>> > >> > > > > all
>> > >> > > > > > brokers and just redirect to the controller on the broker
>> > side.
>> > >> > > Perhaps
>> > >> > > > > > there would be a general way to encapsulate this re-routing
>> > >> > behavior.
>> > >> > > > > >
>> > >> > > > > > 6. We should probably normalize the key value pairs used
>> for
>> > >> > configs
>> > >> > > > > rather
>> > >> > > > > > than embedding a new formatting. So two strings rather than
>> > one
>> > >> > with
>> > >> > > an
>> > >> > > > > > internal equals sign.
>> > >> > > > > >
>> > >> > > > > > 7. Is the postcondition of these APIs that the command has
>> > >> begun or
>> > >> > > > that
>> > >> > > > > > the command has been completed? It is a lot more usable if
>> the
>> > >> > > command
>> > >> > > > > has
>> > >> > > > > > been completed so you know that if you create a topic and
>> then
>> > >> > > publish
>> > >> > > > to
>> > >> > > > > > it you won't get an exception about there being no such
>> topic.
>> > >> > > > > >
>> > >> > > > > > 8. Describe topic and list topics duplicate a lot of stuff
>> in
>> > >> the
>> > >> > > > > metadata
>> > >> > > > > > request. Is there a reason to give back topics marked for
>> > >> > deletion? I
>> > >> > > > > feel
>> > >> > > > > > like if we just make the post-condition of the delete
>> command
>> > be
>> > >> > that
>> > >> > > > the
>> > >> > > > > > topic is deleted that will get rid of the need for this
>> right?
>> > >> And
>> > >> > it
>> > >> > > > > will
>> > >> > > > > > be much more intuitive.
>> > >> > > > > >
>> > >> > > > > > 9. Should we consider batching these requests? We have
>> > generally
>> > >> > > tried
>> > >> > > > to
>> > >> > > > > > allow multiple operations to be batched. My suspicion is
>> that
>> > >> > without
>> > >> > > > > this
>> > >> > > > > > we will get a lot of code that does something like
>> > >> > > > > >    for(topic: adminClient.listTopics())
>> > >> > > > > >       adminClient.describeTopic(topic)
>> > >> > > > > > this code will work great when you test on 5 topics but
>> not do
>> > >> as
>> > >> > > well
>> > >> > > > if
>> > >> > > > > > you have 50k.
>> > >> > > > > >
>> > >> > > > > > 10. I think we should also discuss how we want to expose a
>> > >> > > programmatic
>> > >> > > > > JVM
>> > >> > > > > > client api for these operations. Currently people rely on
>> > >> > AdminUtils
>> > >> > > > > which
>> > >> > > > > > is totally sketchy. I think we probably need another client
>> > >> under
>> > >> > > > > clients/
>> > >> > > > > > that exposes administrative functionality. We will need
>> this
>> > >> just
>> > >> > to
>> > >> > > > > > properly test the new apis, I suspect. We should figure out
>> > that
>> > >> > API.
>> > >> > > > > >
>> > >> > > > > > 11. The other information that would be really useful to
>> get
>> > >> would
>> > >> > be
>> > >> > > > > > information about partitions--how much data is in the
>> > partition,
>> > >> > what
>> > >> > > > are
>> > >> > > > > > the segment offsets, what is the log-end offset (i.e. last
>> > >> offset),
>> > >> > > > what
>> > >> > > > > is
>> > >> > > > > > the compaction point, etc. I think that done right this
>> would
>> > be
>> > >> > the
>> > >> > > > > > successor to the very awkward OffsetRequest we have today.
>> > >> > > > > >
>> > >> > > > > > -Jay
>> > >> > > > > >
>> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
>> > >> joe.stein@stealth.ly>
>> > >> > > > > wrote:
>> > >> > > > > >
>> > >> > > > > > > Hi, created a KIP
>> > >> > > > > > >
>> > >> > > > > > >
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>> > >> > > > > > >
>> > >> > > > > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
>> > >> > > > > > >
>> > >> > > > > > > /*******************************************
>> > >> > > > > > >  Joe Stein
>> > >> > > > > > >  Founder, Principal Consultant
>> > >> > > > > > >  Big Data Open Source Security LLC
>> > >> > > > > > >  http://www.stealth.ly
>> > >> > > > > > >  Twitter: @allthingshadoop <
>> > >> > http://www.twitter.com/allthingshadoop
>> > >> > > >
>> > >> > > > > > > ********************************************/
>> > >> > > > > > >
>> > >> > > > > >
>> > >> > > > >
>> > >> > > >
>> > >> > >
>> > >> >
>> > >>
>> > >>
>> > >>
>> > >> --
>> > >> -- Guozhang
>> > >>
>> > >
>> > >
>> >
>>
>
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Jay,

Re error messages: you are right, in most cases client will have enough
context to show descriptive error message. My concern is that we will have
to
add lots of new error codes for each possible error. Of course, we could
reuse
some of existing like UknownTopicOrPartitionCode, but we will also need to
add smth like: TopicAlreadyExistsCode, TopicConfigInvalid (both for topic
name and config, and probably user would like to know what exactly
is wrong in his config), InvalidReplicaAssignment, InternalError (e.g.
zookeeper failure) etc.
And this is only for TopicCommand, we will also need to add similar stuff
for
ReassignPartitions, PreferredReplica. So we'll end up with a large list of
error codes, used only in Admin protocol.
Having said that, I agree my proposal is not consistent with other cases.
Maybe we can find better solution or something in-between.

Re Hangout chat: I think it is a great idea. This way we can move on faster.
Let's agree somehow on date/time so people can join. Will work for me this
and
next week almost anytime if agreed in advance.

Thanks,
Andrii

On Wed, Feb 18, 2015 at 7:09 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Andrii,
>
> Generally we can do good error handling without needing custom server-side
> messages. I.e. generally the client has the context to know that if it got
> an error that the topic doesn't exist to say "Topic X doesn't exist" rather
> than "error code 14" (or whatever). Maybe there are specific cases where
> this is hard? If we want to add server-side error messages we really do
> need to do this in a consistent way across the protocol.
>
> I still have a bunch of open questions here from my previous list. I will
> be out for the next few days for Strata though. Maybe we could do a Google
> Hangout chat on any open issues some time towards the end of next week for
> anyone interested in this ticket? I have a feeling that might progress
> things a little faster than email--I think we could talk through those
> issues I brought up fairly quickly...
>
> -Jay
>
> On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Hi all,
> >
> > I'm trying to address some of the issues which were mentioned earlier
> about
> > Admin RQ/RP format. One of those was about batching operations. What if
> we
> > follow TopicCommand approach and let people specify topic-name by regexp
> -
> > would that cover most of the use cases?
> >
> > Secondly, is what information should we generally provide in Admin
> > responses.
> > I realize that Admin commands don't imply they will be used only in CLI
> > but,
> > it seems to me, CLI is a very important client of this feature. In this
> > case,
> > seems logical, we would like to provide users with rich experience in
> terms
> > of
> > getting results / errors of the executed commands. Usually we supply with
> > responses only errorCode, which looks very limiting, in case of CLI we
> may
> > want to print human readable error description.
> >
> > So, taking into account previous item about batching, what do you think
> > about
> > having smth like:
> >
> > ('create' doesn't support regexp)
> > CreateTopicRequest => TopicName Partitions Replicas ReplicaAssignment
> > [Config]
> > CreateTopicResponse => ErrorCode ErrorDescription
> >   ErrorCode => int16
> >   ErrorDescription => string (empty if successful)
> >
> > AlterTopicRequest -> TopicNameRegexp Partitions ReplicaAssignment
> > [AddedConfig] [DeletedConfig]
> > AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
> > CommandErrorCode CommandErrorDescription
> >   CommandErrorCode => int16
> >   CommandErrorDescription => string (nonempty in case of fatal error,
> e.g.
> > we couldn't get topics by regexp)
> >
> > DescribeTopicRequest -> TopicNameRegexp
> > DescribeTopicResponse -> [TopicName TopicDescription ErrorCode
> > ErrorDescription] CommandErrorCode CommandErrorDescription
> >
> > Also, any thoughts about our discussion regarding re-routing facility? In
> > my
> > understanding, it is like between augmenting TopicMetadataRequest
> > (to include at least controllerId) and implementing new generic
> re-routing
> > facility so sending messages to controller will be handled by it.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> > On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > @Guozhang:
> > > Thanks for your comments, I've answered some of those. The main thing
> is
> > > having merged request for create-alter-delete-describe - I have some
> > > concerns about this approach.
> > >
> > > @*Jay*:
> > > I see that introduced ClusterMetadaRequest is also one of the concerns.
> > We
> > > can solve it if we implement re-routing facility. But I agree with
> > > Guozhang - it will make clients' internals a little bit easier but this
> > > seems to be a complex logic to implement and support then. Especially
> for
> > > Fetch and Produce (even if we add re-routing later for these requests).
> > > Also people will tend to avoid this re-routing facility and hold local
> > > cluster cache to ensure their high-priority requests (which some of
> the
> > > admin requests are) not sent to some busy broker where they wait to be
> > > routed to the correct one.
> > > As pointed out by Jun here (
> > >
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> > )
> > > to solve the issue we might introduce a message type to get cluster
> > state.
> > > But I agree we can just update TopicMetadataResponse to include
> > > controllerId (and probably smth else).
> > > What are you thougths?
> > >
> > > Thanks,
> > > Andrii
> > >
> > > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <wa...@gmail.com>
> > wrote:
> > >
> > >> I think for the topics commands we can actually merge
> > >> create/alter/delete/describe as one request type since their formats
> are
> > >> very much similar, and keep list-topics and others like
> > >> partition-reassignment / preferred-leader-election as separate request
> > >> types, I also left some other comments on the RB (
> > >> https://reviews.apache.org/r/29301/).
> > >>
> > >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >>
> > >> > Yeah I totally agree that we don't want to just have one "do admin
> > >> stuff"
> > >> > command that has the union of all parameters.
> > >> >
> > >> > What I am saying is that command line tools are one client of the
> > >> > administrative apis, but these will be used in a number of scenarios
> > so
> > >> > they should make logical sense even in the absence of the command
> line
> > >> > tool. Hence comments like trying to clarify the relationship between
> > >> > ClusterMetadata and TopicMetadata...these kinds of things really
> need
> > >> to be
> > >> > thought through.
> > >> >
> > >> > Hope that makes sense.
> > >> >
> > >> > -Jay
> > >> >
> > >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> > >> > andrii.biletskyi@stealth.ly> wrote:
> > >> >
> > >> > > Jay,
> > >> > >
> > >> > > Thanks for answering. You understood correctly, most of my
> comments
> > >> were
> > >> > > related to your point 1) - about "well thought-out" apis. Also,
> yes,
> > >> as I
> > >> > > understood we would like to introduce a single unified CLI tool
> with
> > >> > > centralized server-side request handling for lots of existing ones
> > >> (incl.
> > >> > > TopicCommand, CommitOffsetChecker, ReassignPartitions, smth else
> if
> > >> added
> > >> > > in future). In our previous discussion (
> > >> > > https://issues.apache.org/jira/browse/KAFKA-1694) people said
> > they'd
> > >> > > rather
> > >> > > have a separate message for each command, so, yes, this way I came
> > to
> > >> 1-1
> > >> > > mapping between commands in the tool and protocol additions. But I
> > >> might
> > >> > be
> > >> > > wrong.
> > >> > > At the end I just try to start discussion how at least generally
> > this
> > >> > > protocol should look like.
> > >> > >
> > >> > > Thanks,
> > >> > > Andrii
> > >> > >
> > >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <ja...@gmail.com>
> > >> wrote:
> > >> > >
> > >> > > > Hey Andrii,
> > >> > > >
> > >> > > > To answer your earlier question we just really can't be adding
> any
> > >> more
> > >> > > > scala protocol objects. These things are super hard to maintain
> > >> because
> > >> > > > they hand code the byte parsing and don't have good versioning
> > >> support.
> > >> > > > Since we are already planning on converting we definitely don't
> > >> want to
> > >> > > add
> > >> > > > a ton more of these--they are total tech debt.
> > >> > > >
> > >> > > > What does it mean that the changes are isolated from the current
> > >> code
> > >> > > base?
> > >> > > >
> > >> > > > I actually didn't understand the remaining comments, which of
> the
> > >> > points
> > >> > > > are you responding to?
> > >> > > >
> > >> > > > Maybe one sticking point here is that it seems like you want to
> > make
> > >> > some
> > >> > > > kind of tool, and you have made a 1-1 mapping between commands
> you
> > >> > > imagine
> > >> > > > in the tool and protocol additions. I want to make sure we don't
> > do
> > >> > that.
> > >> > > > The protocol needs to be really really well thought out against
> > many
> > >> > use
> > >> > > > cases so it should make perfect logical sense in the absence of
> > >> knowing
> > >> > > the
> > >> > > > command line tool, right?
> > >> > > >
> > >> > > > -Jay
> > >> > > >
> > >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
> > >> > > > andrii.biletskyi@stealth.ly> wrote:
> > >> > > >
> > >> > > > > Hey Jay,
> > >> > > > >
> > >> > > > > I would like to continue this discussion as it seem there is
> no
> > >> > > progress
> > >> > > > > here.
> > >> > > > >
> > >> > > > > First of all, could you please explain what did you mean in 2?
> > How
> > >> > > > exactly
> > >> > > > > are we going to migrate to the new java protocol definitions.
> > And
> > >> why
> > >> > > > it's
> > >> > > > > a blocker for centralized CLI?
> > >> > > > >
> > >> > > > > I agree with you, this feature includes lots of stuff, but
> > >> thankfully
> > >> > > > > almost all changes are isolated from the current code base,
> > >> > > > > so the main thing, I think, we need to agree is RQ/RP format.
> > >> > > > > So how can we start discussion about the concrete messages
> > format?
> > >> > > > > Can we take (
> > >> > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > >> > > > > )
> > >> > > > > as starting point?
> > >> > > > >
> > >> > > > > We had some doubts earlier whether it worth introducing one
> > >> generic
> > >> > > Admin
> > >> > > > > Request for all commands (
> > >> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > >> > > > > )
> > >> > > > > but then everybody agreed it would be better to have separate
> > >> message
> > >> > > for
> > >> > > > > each admin command. The Request part is really dictated from
> the
> > >> > > command
> > >> > > > > (e.g. TopicCommand) arguments itself, so the proposed version
> > >> should
> > >> > be
> > >> > > > > fine (let's put aside for now remarks about Optional type,
> > >> batching,
> > >> > > > > configs normalization - I agree with all of them).
> > >> > > > > So the second part is Response. I see there are two cases
> here.
> > >> > > > > a) "Mutate" requests - Create/Alter/... ; b) "Get" requests -
> > >> > > > > List/Describe...
> > >> > > > >
> > >> > > > > a) should only hold request result (regardless what we decide
> > >> about
> > >> > > > > blocking/non-blocking commands execution).
> > >> > > > > Usually we provide error code in response but since we will
> use
> > >> this
> > >> > in
> > >> > > > > interactive shell we need some human readable error
> description
> > -
> > >> so
> > >> > I
> > >> > > > > added errorDesription field where you can at least leave
> > >> > > > > exception.getMessage.
> > >> > > > >
> > >> > > > > b) in addition to previous item message should hold command
> > >> specific
> > >> > > > > response data. We can discuss in detail each of them but let's
> > for
> > >> > now
> > >> > > > > agree about the overall pattern.
> > >> > > > >
> > >> > > > > Thanks,
> > >> > > > > Andrii Biletskyi
> > >> > > > >
> > >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <
> jay.kreps@gmail.com
> > >
> > >> > > wrote:
> > >> > > > >
> > >> > > > > > Hey Joe,
> > >> > > > > >
> > >> > > > > > This is great. A few comments on KIP-4
> > >> > > > > >
> > >> > > > > > 1. This is much needed functionality, but there are a lot of
> > >> the so
> > >> > > > let's
> > >> > > > > > really think these protocols through. We really want to end
> up
> > >> > with a
> > >> > > > set
> > >> > > > > > of well thought-out, orthoganol apis. For this reason I
> think
> > >> it is
> > >> > > > > really
> > >> > > > > > important to think through the end state even if that
> includes
> > >> APIs
> > >> > > we
> > >> > > > > > won't implement in the first phase.
> > >> > > > > >
> > >> > > > > > 2. Let's please please please wait until we have switched
> the
> > >> > server
> > >> > > > over
> > >> > > > > > to the new java protocol definitions. If we add upteen more
> ad
> > >> hoc
> > >> > > > scala
> > >> > > > > > objects that is just generating more work for the conversion
> > we
> > >> > know
> > >> > > we
> > >> > > > > > have to do.
> > >> > > > > >
> > >> > > > > > 3. This proposal introduces a new type of optional
> parameter.
> > >> This
> > >> > is
> > >> > > > > > inconsistent with everything else in the protocol where we
> use
> > >> -1
> > >> > or
> > >> > > > some
> > >> > > > > > other marker value. You could argue either way but let's
> stick
> > >> with
> > >> > > > that
> > >> > > > > > for consistency. For clients that implemented the protocol
> in
> > a
> > >> > > better
> > >> > > > > way
> > >> > > > > > than our scala code these basic primitives are hard to
> change.
> > >> > > > > >
> > >> > > > > > 4. ClusterMetadata: This seems to duplicate
> > TopicMetadataRequest
> > >> > > which
> > >> > > > > has
> > >> > > > > > brokers, topics, and partitions. I think we should rename
> that
> > >> > > request
> > >> > > > > > ClusterMetadataRequest (or just MetadataRequest) and include
> > >> the id
> > >> > > of
> > >> > > > > the
> > >> > > > > > controller. Or are there other things we could add here?
> > >> > > > > >
> > >> > > > > > 5. We have a tendency to try to make a lot of requests that
> > can
> > >> > only
> > >> > > go
> > >> > > > > to
> > >> > > > > > particular nodes. This adds a lot of burden for client
> > >> > > implementations
> > >> > > > > (it
> > >> > > > > > sounds easy but each discovery can fail in many parts so it
> > >> ends up
> > >> > > > > being a
> > >> > > > > > full state machine to do right). I think we should consider
> > >> making
> > >> > > > admin
> > >> > > > > > commands and ideally as many of the other apis as possible
> > >> > available
> > >> > > on
> > >> > > > > all
> > >> > > > > > brokers and just redirect to the controller on the broker
> > side.
> > >> > > Perhaps
> > >> > > > > > there would be a general way to encapsulate this re-routing
> > >> > behavior.
> > >> > > > > >
> > >> > > > > > 6. We should probably normalize the key value pairs used for
> > >> > configs
> > >> > > > > rather
> > >> > > > > > than embedding a new formatting. So two strings rather than
> > one
> > >> > with
> > >> > > an
> > >> > > > > > internal equals sign.
> > >> > > > > >
> > >> > > > > > 7. Is the postcondition of these APIs that the command has
> > >> begun or
> > >> > > > that
> > >> > > > > > the command has been completed? It is a lot more usable if
> the
> > >> > > command
> > >> > > > > has
> > >> > > > > > been completed so you know that if you create a topic and
> then
> > >> > > publish
> > >> > > > to
> > >> > > > > > it you won't get an exception about there being no such
> topic.
> > >> > > > > >
> > >> > > > > > 8. Describe topic and list topics duplicate a lot of stuff
> in
> > >> the
> > >> > > > > metadata
> > >> > > > > > request. Is there a reason to give back topics marked for
> > >> > deletion? I
> > >> > > > > feel
> > >> > > > > > like if we just make the post-condition of the delete
> command
> > be
> > >> > that
> > >> > > > the
> > >> > > > > > topic is deleted that will get rid of the need for this
> right?
> > >> And
> > >> > it
> > >> > > > > will
> > >> > > > > > be much more intuitive.
> > >> > > > > >
> > >> > > > > > 9. Should we consider batching these requests? We have
> > generally
> > >> > > tried
> > >> > > > to
> > >> > > > > > allow multiple operations to be batched. My suspicion is
> that
> > >> > without
> > >> > > > > this
> > >> > > > > > we will get a lot of code that does something like
> > >> > > > > >    for(topic: adminClient.listTopics())
> > >> > > > > >       adminClient.describeTopic(topic)
> > >> > > > > > this code will work great when you test on 5 topics but not
> do
> > >> as
> > >> > > well
> > >> > > > if
> > >> > > > > > you have 50k.
> > >> > > > > >
> > >> > > > > > 10. I think we should also discuss how we want to expose a
> > >> > > programmatic
> > >> > > > > JVM
> > >> > > > > > client api for these operations. Currently people rely on
> > >> > AdminUtils
> > >> > > > > which
> > >> > > > > > is totally sketchy. I think we probably need another client
> > >> under
> > >> > > > > clients/
> > >> > > > > > that exposes administrative functionality. We will need this
> > >> just
> > >> > to
> > >> > > > > > properly test the new apis, I suspect. We should figure out
> > that
> > >> > API.
> > >> > > > > >
> > >> > > > > > 11. The other information that would be really useful to get
> > >> would
> > >> > be
> > >> > > > > > information about partitions--how much data is in the
> > partition,
> > >> > what
> > >> > > > are
> > >> > > > > > the segment offsets, what is the log-end offset (i.e. last
> > >> offset),
> > >> > > > what
> > >> > > > > is
> > >> > > > > > the compaction point, etc. I think that done right this
> would
> > be
> > >> > the
> > >> > > > > > successor to the very awkward OffsetRequest we have today.
> > >> > > > > >
> > >> > > > > > -Jay
> > >> > > > > >
> > >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
> > >> joe.stein@stealth.ly>
> > >> > > > > wrote:
> > >> > > > > >
> > >> > > > > > > Hi, created a KIP
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >> > > > > > >
> > >> > > > > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> > >> > > > > > >
> > >> > > > > > > /*******************************************
> > >> > > > > > >  Joe Stein
> > >> > > > > > >  Founder, Principal Consultant
> > >> > > > > > >  Big Data Open Source Security LLC
> > >> > > > > > >  http://www.stealth.ly
> > >> > > > > > >  Twitter: @allthingshadoop <
> > >> > http://www.twitter.com/allthingshadoop
> > >> > > >
> > >> > > > > > > ********************************************/
> > >> > > > > > >
> > >> > > > > >
> > >> > > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> > >>
> > >>
> > >> --
> > >> -- Guozhang
> > >>
> > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
Hey Andrii,

Generally we can do good error handling without needing custom server-side
messages. I.e. generally the client has the context to know that if it got
an error that the topic doesn't exist to say "Topic X doesn't exist" rather
than "error code 14" (or whatever). Maybe there are specific cases where
this is hard? If we want to add server-side error messages we really do
need to do this in a consistent way across the protocol.

I still have a bunch of open questions here from my previous list. I will
be out for the next few days for Strata though. Maybe we could do a Google
Hangout chat on any open issues some time towards the end of next week for
anyone interested in this ticket? I have a feeling that might progress
things a little faster than email--I think we could talk through those
issues I brought up fairly quickly...

-Jay

On Wed, Feb 18, 2015 at 7:27 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Hi all,
>
> I'm trying to address some of the issues which were mentioned earlier about
> Admin RQ/RP format. One of those was about batching operations. What if we
> follow TopicCommand approach and let people specify topic-name by regexp -
> would that cover most of the use cases?
>
> Secondly, is what information should we generally provide in Admin
> responses.
> I realize that Admin commands don't imply they will be used only in CLI
> but,
> it seems to me, CLI is a very important client of this feature. In this
> case,
> seems logical, we would like to provide users with rich experience in terms
> of
> getting results / errors of the executed commands. Usually we supply with
> responses only errorCode, which looks very limiting, in case of CLI we may
> want to print human readable error description.
>
> So, taking into account previous item about batching, what do you think
> about
> having smth like:
>
> ('create' doesn't support regexp)
> CreateTopicRequest => TopicName Partitions Replicas ReplicaAssignment
> [Config]
> CreateTopicResponse => ErrorCode ErrorDescription
>   ErrorCode => int16
>   ErrorDescription => string (empty if successful)
>
> AlterTopicRequest -> TopicNameRegexp Partitions ReplicaAssignment
> [AddedConfig] [DeletedConfig]
> AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
> CommandErrorCode CommandErrorDescription
>   CommandErrorCode => int16
>   CommandErrorDescription => string (nonempty in case of fatal error, e.g.
> we couldn't get topics by regexp)
>
> DescribeTopicRequest -> TopicNameRegexp
> DescribeTopicResponse -> [TopicName TopicDescription ErrorCode
> ErrorDescription] CommandErrorCode CommandErrorDescription
>
> Also, any thoughts about our discussion regarding re-routing facility? In
> my
> understanding, it is like between augmenting TopicMetadataRequest
> (to include at least controllerId) and implementing new generic re-routing
> facility so sending messages to controller will be handled by it.
>
> Thanks,
> Andrii Biletskyi
>
> On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > @Guozhang:
> > Thanks for your comments, I've answered some of those. The main thing is
> > having merged request for create-alter-delete-describe - I have some
> > concerns about this approach.
> >
> > @*Jay*:
> > I see that introduced ClusterMetadaRequest is also one of the concerns.
> We
> > can solve it if we implement re-routing facility. But I agree with
> > Guozhang - it will make clients' internals a little bit easier but this
> > seems to be a complex logic to implement and support then. Especially for
> > Fetch and Produce (even if we add re-routing later for these requests).
> > Also people will tend to avoid this re-routing facility and hold local
> > cluster cache to ensure their high-priority requests (which some of  the
> > admin requests are) not sent to some busy broker where they wait to be
> > routed to the correct one.
> > As pointed out by Jun here (
> >
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530
> )
> > to solve the issue we might introduce a message type to get cluster
> state.
> > But I agree we can just update TopicMetadataResponse to include
> > controllerId (and probably smth else).
> > What are you thougths?
> >
> > Thanks,
> > Andrii
> >
> > On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <wa...@gmail.com>
> wrote:
> >
> >> I think for the topics commands we can actually merge
> >> create/alter/delete/describe as one request type since their formats are
> >> very much similar, and keep list-topics and others like
> >> partition-reassignment / preferred-leader-election as separate request
> >> types, I also left some other comments on the RB (
> >> https://reviews.apache.org/r/29301/).
> >>
> >> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <ja...@gmail.com> wrote:
> >>
> >> > Yeah I totally agree that we don't want to just have one "do admin
> >> stuff"
> >> > command that has the union of all parameters.
> >> >
> >> > What I am saying is that command line tools are one client of the
> >> > administrative apis, but these will be used in a number of scenarios
> so
> >> > they should make logical sense even in the absence of the command line
> >> > tool. Hence comments like trying to clarify the relationship between
> >> > ClusterMetadata and TopicMetadata...these kinds of things really need
> >> to be
> >> > thought through.
> >> >
> >> > Hope that makes sense.
> >> >
> >> > -Jay
> >> >
> >> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> >> > andrii.biletskyi@stealth.ly> wrote:
> >> >
> >> > > Jay,
> >> > >
> >> > > Thanks for answering. You understood correctly, most of my comments
> >> were
> >> > > related to your point 1) - about "well thought-out" apis. Also, yes,
> >> as I
> >> > > understood we would like to introduce a single unified CLI tool with
> >> > > centralized server-side request handling for lots of existing ones
> >> (incl.
> >> > > TopicCommand, CommitOffsetChecker, ReassignPartitions, smth else if
> >> added
> >> > > in future). In our previous discussion (
> >> > > https://issues.apache.org/jira/browse/KAFKA-1694) people said
> they'd
> >> > > rather
> >> > > have a separate message for each command, so, yes, this way I came
> to
> >> 1-1
> >> > > mapping between commands in the tool and protocol additions. But I
> >> might
> >> > be
> >> > > wrong.
> >> > > At the end I just try to start discussion how at least generally
> this
> >> > > protocol should look like.
> >> > >
> >> > > Thanks,
> >> > > Andrii
> >> > >
> >> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <ja...@gmail.com>
> >> wrote:
> >> > >
> >> > > > Hey Andrii,
> >> > > >
> >> > > > To answer your earlier question we just really can't be adding any
> >> more
> >> > > > scala protocol objects. These things are super hard to maintain
> >> because
> >> > > > they hand code the byte parsing and don't have good versioning
> >> support.
> >> > > > Since we are already planning on converting we definitely don't
> >> want to
> >> > > add
> >> > > > a ton more of these--they are total tech debt.
> >> > > >
> >> > > > What does it mean that the changes are isolated from the current
> >> code
> >> > > base?
> >> > > >
> >> > > > I actually didn't understand the remaining comments, which of the
> >> > points
> >> > > > are you responding to?
> >> > > >
> >> > > > Maybe one sticking point here is that it seems like you want to
> make
> >> > some
> >> > > > kind of tool, and you have made a 1-1 mapping between commands you
> >> > > imagine
> >> > > > in the tool and protocol additions. I want to make sure we don't
> do
> >> > that.
> >> > > > The protocol needs to be really really well thought out against
> many
> >> > use
> >> > > > cases so it should make perfect logical sense in the absence of
> >> knowing
> >> > > the
> >> > > > command line tool, right?
> >> > > >
> >> > > > -Jay
> >> > > >
> >> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
> >> > > > andrii.biletskyi@stealth.ly> wrote:
> >> > > >
> >> > > > > Hey Jay,
> >> > > > >
> >> > > > > I would like to continue this discussion as it seem there is no
> >> > > progress
> >> > > > > here.
> >> > > > >
> >> > > > > First of all, could you please explain what did you mean in 2?
> How
> >> > > > exactly
> >> > > > > are we going to migrate to the new java protocol definitions.
> And
> >> why
> >> > > > it's
> >> > > > > a blocker for centralized CLI?
> >> > > > >
> >> > > > > I agree with you, this feature includes lots of stuff, but
> >> thankfully
> >> > > > > almost all changes are isolated from the current code base,
> >> > > > > so the main thing, I think, we need to agree is RQ/RP format.
> >> > > > > So how can we start discussion about the concrete messages
> format?
> >> > > > > Can we take (
> >> > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> >> > > > > )
> >> > > > > as starting point?
> >> > > > >
> >> > > > > We had some doubts earlier whether it worth introducing one
> >> generic
> >> > > Admin
> >> > > > > Request for all commands (
> >> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> >> > > > > )
> >> > > > > but then everybody agreed it would be better to have separate
> >> message
> >> > > for
> >> > > > > each admin command. The Request part is really dictated from the
> >> > > command
> >> > > > > (e.g. TopicCommand) arguments itself, so the proposed version
> >> should
> >> > be
> >> > > > > fine (let's put aside for now remarks about Optional type,
> >> batching,
> >> > > > > configs normalization - I agree with all of them).
> >> > > > > So the second part is Response. I see there are two cases here.
> >> > > > > a) "Mutate" requests - Create/Alter/... ; b) "Get" requests -
> >> > > > > List/Describe...
> >> > > > >
> >> > > > > a) should only hold request result (regardless what we decide
> >> about
> >> > > > > blocking/non-blocking commands execution).
> >> > > > > Usually we provide error code in response but since we will use
> >> this
> >> > in
> >> > > > > interactive shell we need some human readable error description
> -
> >> so
> >> > I
> >> > > > > added errorDesription field where you can at least leave
> >> > > > > exception.getMessage.
> >> > > > >
> >> > > > > b) in addition to previous item message should hold command
> >> specific
> >> > > > > response data. We can discuss in detail each of them but let's
> for
> >> > now
> >> > > > > agree about the overall pattern.
> >> > > > >
> >> > > > > Thanks,
> >> > > > > Andrii Biletskyi
> >> > > > >
> >> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <jay.kreps@gmail.com
> >
> >> > > wrote:
> >> > > > >
> >> > > > > > Hey Joe,
> >> > > > > >
> >> > > > > > This is great. A few comments on KIP-4
> >> > > > > >
> >> > > > > > 1. This is much needed functionality, but there are a lot of
> >> the so
> >> > > > let's
> >> > > > > > really think these protocols through. We really want to end up
> >> > with a
> >> > > > set
> >> > > > > > of well thought-out, orthoganol apis. For this reason I think
> >> it is
> >> > > > > really
> >> > > > > > important to think through the end state even if that includes
> >> APIs
> >> > > we
> >> > > > > > won't implement in the first phase.
> >> > > > > >
> >> > > > > > 2. Let's please please please wait until we have switched the
> >> > server
> >> > > > over
> >> > > > > > to the new java protocol definitions. If we add upteen more ad
> >> hoc
> >> > > > scala
> >> > > > > > objects that is just generating more work for the conversion
> we
> >> > know
> >> > > we
> >> > > > > > have to do.
> >> > > > > >
> >> > > > > > 3. This proposal introduces a new type of optional parameter.
> >> This
> >> > is
> >> > > > > > inconsistent with everything else in the protocol where we use
> >> -1
> >> > or
> >> > > > some
> >> > > > > > other marker value. You could argue either way but let's stick
> >> with
> >> > > > that
> >> > > > > > for consistency. For clients that implemented the protocol in
> a
> >> > > better
> >> > > > > way
> >> > > > > > than our scala code these basic primitives are hard to change.
> >> > > > > >
> >> > > > > > 4. ClusterMetadata: This seems to duplicate
> TopicMetadataRequest
> >> > > which
> >> > > > > has
> >> > > > > > brokers, topics, and partitions. I think we should rename that
> >> > > request
> >> > > > > > ClusterMetadataRequest (or just MetadataRequest) and include
> >> the id
> >> > > of
> >> > > > > the
> >> > > > > > controller. Or are there other things we could add here?
> >> > > > > >
> >> > > > > > 5. We have a tendency to try to make a lot of requests that
> can
> >> > only
> >> > > go
> >> > > > > to
> >> > > > > > particular nodes. This adds a lot of burden for client
> >> > > implementations
> >> > > > > (it
> >> > > > > > sounds easy but each discovery can fail in many parts so it
> >> ends up
> >> > > > > being a
> >> > > > > > full state machine to do right). I think we should consider
> >> making
> >> > > > admin
> >> > > > > > commands and ideally as many of the other apis as possible
> >> > available
> >> > > on
> >> > > > > all
> >> > > > > > brokers and just redirect to the controller on the broker
> side.
> >> > > Perhaps
> >> > > > > > there would be a general way to encapsulate this re-routing
> >> > behavior.
> >> > > > > >
> >> > > > > > 6. We should probably normalize the key value pairs used for
> >> > configs
> >> > > > > rather
> >> > > > > > than embedding a new formatting. So two strings rather than
> one
> >> > with
> >> > > an
> >> > > > > > internal equals sign.
> >> > > > > >
> >> > > > > > 7. Is the postcondition of these APIs that the command has
> >> begun or
> >> > > > that
> >> > > > > > the command has been completed? It is a lot more usable if the
> >> > > command
> >> > > > > has
> >> > > > > > been completed so you know that if you create a topic and then
> >> > > publish
> >> > > > to
> >> > > > > > it you won't get an exception about there being no such topic.
> >> > > > > >
> >> > > > > > 8. Describe topic and list topics duplicate a lot of stuff in
> >> the
> >> > > > > metadata
> >> > > > > > request. Is there a reason to give back topics marked for
> >> > deletion? I
> >> > > > > feel
> >> > > > > > like if we just make the post-condition of the delete command
> be
> >> > that
> >> > > > the
> >> > > > > > topic is deleted that will get rid of the need for this right?
> >> And
> >> > it
> >> > > > > will
> >> > > > > > be much more intuitive.
> >> > > > > >
> >> > > > > > 9. Should we consider batching these requests? We have
> generally
> >> > > tried
> >> > > > to
> >> > > > > > allow multiple operations to be batched. My suspicion is that
> >> > without
> >> > > > > this
> >> > > > > > we will get a lot of code that does something like
> >> > > > > >    for(topic: adminClient.listTopics())
> >> > > > > >       adminClient.describeTopic(topic)
> >> > > > > > this code will work great when you test on 5 topics but not do
> >> as
> >> > > well
> >> > > > if
> >> > > > > > you have 50k.
> >> > > > > >
> >> > > > > > 10. I think we should also discuss how we want to expose a
> >> > > programmatic
> >> > > > > JVM
> >> > > > > > client api for these operations. Currently people rely on
> >> > AdminUtils
> >> > > > > which
> >> > > > > > is totally sketchy. I think we probably need another client
> >> under
> >> > > > > clients/
> >> > > > > > that exposes administrative functionality. We will need this
> >> just
> >> > to
> >> > > > > > properly test the new apis, I suspect. We should figure out
> that
> >> > API.
> >> > > > > >
> >> > > > > > 11. The other information that would be really useful to get
> >> would
> >> > be
> >> > > > > > information about partitions--how much data is in the
> partition,
> >> > what
> >> > > > are
> >> > > > > > the segment offsets, what is the log-end offset (i.e. last
> >> offset),
> >> > > > what
> >> > > > > is
> >> > > > > > the compaction point, etc. I think that done right this would
> be
> >> > the
> >> > > > > > successor to the very awkward OffsetRequest we have today.
> >> > > > > >
> >> > > > > > -Jay
> >> > > > > >
> >> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
> >> joe.stein@stealth.ly>
> >> > > > > wrote:
> >> > > > > >
> >> > > > > > > Hi, created a KIP
> >> > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >> > > > > > >
> >> > > > > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> >> > > > > > >
> >> > > > > > > /*******************************************
> >> > > > > > >  Joe Stein
> >> > > > > > >  Founder, Principal Consultant
> >> > > > > > >  Big Data Open Source Security LLC
> >> > > > > > >  http://www.stealth.ly
> >> > > > > > >  Twitter: @allthingshadoop <
> >> > http://www.twitter.com/allthingshadoop
> >> > > >
> >> > > > > > > ********************************************/
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>
> >>
> >> --
> >> -- Guozhang
> >>
> >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Hi all,

I'm trying to address some of the issues which were mentioned earlier about
Admin RQ/RP format. One of those was about batching operations. What if we
follow TopicCommand approach and let people specify topic-name by regexp -
would that cover most of the use cases?

Secondly, is what information should we generally provide in Admin
responses.
I realize that Admin commands don't imply they will be used only in CLI
but,
it seems to me, CLI is a very important client of this feature. In this
case,
seems logical, we would like to provide users with rich experience in terms
of
getting results / errors of the executed commands. Usually we supply with
responses only errorCode, which looks very limiting, in case of CLI we may
want to print human readable error description.

So, taking into account previous item about batching, what do you think
about
having smth like:

('create' doesn't support regexp)
CreateTopicRequest => TopicName Partitions Replicas ReplicaAssignment
[Config]
CreateTopicResponse => ErrorCode ErrorDescription
  ErrorCode => int16
  ErrorDescription => string (empty if successful)

AlterTopicRequest -> TopicNameRegexp Partitions ReplicaAssignment
[AddedConfig] [DeletedConfig]
AlterTopicResponse -> [TopicName ErrorCode ErrorDescription]
CommandErrorCode CommandErrorDescription
  CommandErrorCode => int16
  CommandErrorDescription => string (nonempty in case of fatal error, e.g.
we couldn't get topics by regexp)

DescribeTopicRequest -> TopicNameRegexp
DescribeTopicResponse -> [TopicName TopicDescription ErrorCode
ErrorDescription] CommandErrorCode CommandErrorDescription

Also, any thoughts about our discussion regarding re-routing facility? In
my
understanding, it is like between augmenting TopicMetadataRequest
(to include at least controllerId) and implementing new generic re-routing
facility so sending messages to controller will be handled by it.

Thanks,
Andrii Biletskyi

On Mon, Feb 16, 2015 at 5:26 PM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> @Guozhang:
> Thanks for your comments, I've answered some of those. The main thing is
> having merged request for create-alter-delete-describe - I have some
> concerns about this approach.
>
> @*Jay*:
> I see that introduced ClusterMetadaRequest is also one of the concerns. We
> can solve it if we implement re-routing facility. But I agree with
> Guozhang - it will make clients' internals a little bit easier but this
> seems to be a complex logic to implement and support then. Especially for
> Fetch and Produce (even if we add re-routing later for these requests).
> Also people will tend to avoid this re-routing facility and hold local
> cluster cache to ensure their high-priority requests (which some of  the
> admin requests are) not sent to some busy broker where they wait to be
> routed to the correct one.
> As pointed out by Jun here (
> https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530)
> to solve the issue we might introduce a message type to get cluster state.
> But I agree we can just update TopicMetadataResponse to include
> controllerId (and probably smth else).
> What are you thougths?
>
> Thanks,
> Andrii
>
> On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <wa...@gmail.com> wrote:
>
>> I think for the topics commands we can actually merge
>> create/alter/delete/describe as one request type since their formats are
>> very much similar, and keep list-topics and others like
>> partition-reassignment / preferred-leader-election as separate request
>> types, I also left some other comments on the RB (
>> https://reviews.apache.org/r/29301/).
>>
>> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <ja...@gmail.com> wrote:
>>
>> > Yeah I totally agree that we don't want to just have one "do admin
>> stuff"
>> > command that has the union of all parameters.
>> >
>> > What I am saying is that command line tools are one client of the
>> > administrative apis, but these will be used in a number of scenarios so
>> > they should make logical sense even in the absence of the command line
>> > tool. Hence comments like trying to clarify the relationship between
>> > ClusterMetadata and TopicMetadata...these kinds of things really need
>> to be
>> > thought through.
>> >
>> > Hope that makes sense.
>> >
>> > -Jay
>> >
>> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
>> > andrii.biletskyi@stealth.ly> wrote:
>> >
>> > > Jay,
>> > >
>> > > Thanks for answering. You understood correctly, most of my comments
>> were
>> > > related to your point 1) - about "well thought-out" apis. Also, yes,
>> as I
>> > > understood we would like to introduce a single unified CLI tool with
>> > > centralized server-side request handling for lots of existing ones
>> (incl.
>> > > TopicCommand, CommitOffsetChecker, ReassignPartitions, smth else if
>> added
>> > > in future). In our previous discussion (
>> > > https://issues.apache.org/jira/browse/KAFKA-1694) people said they'd
>> > > rather
>> > > have a separate message for each command, so, yes, this way I came to
>> 1-1
>> > > mapping between commands in the tool and protocol additions. But I
>> might
>> > be
>> > > wrong.
>> > > At the end I just try to start discussion how at least generally this
>> > > protocol should look like.
>> > >
>> > > Thanks,
>> > > Andrii
>> > >
>> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <ja...@gmail.com>
>> wrote:
>> > >
>> > > > Hey Andrii,
>> > > >
>> > > > To answer your earlier question we just really can't be adding any
>> more
>> > > > scala protocol objects. These things are super hard to maintain
>> because
>> > > > they hand code the byte parsing and don't have good versioning
>> support.
>> > > > Since we are already planning on converting we definitely don't
>> want to
>> > > add
>> > > > a ton more of these--they are total tech debt.
>> > > >
>> > > > What does it mean that the changes are isolated from the current
>> code
>> > > base?
>> > > >
>> > > > I actually didn't understand the remaining comments, which of the
>> > points
>> > > > are you responding to?
>> > > >
>> > > > Maybe one sticking point here is that it seems like you want to make
>> > some
>> > > > kind of tool, and you have made a 1-1 mapping between commands you
>> > > imagine
>> > > > in the tool and protocol additions. I want to make sure we don't do
>> > that.
>> > > > The protocol needs to be really really well thought out against many
>> > use
>> > > > cases so it should make perfect logical sense in the absence of
>> knowing
>> > > the
>> > > > command line tool, right?
>> > > >
>> > > > -Jay
>> > > >
>> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
>> > > > andrii.biletskyi@stealth.ly> wrote:
>> > > >
>> > > > > Hey Jay,
>> > > > >
>> > > > > I would like to continue this discussion as it seem there is no
>> > > progress
>> > > > > here.
>> > > > >
>> > > > > First of all, could you please explain what did you mean in 2? How
>> > > > exactly
>> > > > > are we going to migrate to the new java protocol definitions. And
>> why
>> > > > it's
>> > > > > a blocker for centralized CLI?
>> > > > >
>> > > > > I agree with you, this feature includes lots of stuff, but
>> thankfully
>> > > > > almost all changes are isolated from the current code base,
>> > > > > so the main thing, I think, we need to agree is RQ/RP format.
>> > > > > So how can we start discussion about the concrete messages format?
>> > > > > Can we take (
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
>> > > > > )
>> > > > > as starting point?
>> > > > >
>> > > > > We had some doubts earlier whether it worth introducing one
>> generic
>> > > Admin
>> > > > > Request for all commands (
>> > > > https://issues.apache.org/jira/browse/KAFKA-1694
>> > > > > )
>> > > > > but then everybody agreed it would be better to have separate
>> message
>> > > for
>> > > > > each admin command. The Request part is really dictated from the
>> > > command
>> > > > > (e.g. TopicCommand) arguments itself, so the proposed version
>> should
>> > be
>> > > > > fine (let's put aside for now remarks about Optional type,
>> batching,
>> > > > > configs normalization - I agree with all of them).
>> > > > > So the second part is Response. I see there are two cases here.
>> > > > > a) "Mutate" requests - Create/Alter/... ; b) "Get" requests -
>> > > > > List/Describe...
>> > > > >
>> > > > > a) should only hold request result (regardless what we decide
>> about
>> > > > > blocking/non-blocking commands execution).
>> > > > > Usually we provide error code in response but since we will use
>> this
>> > in
>> > > > > interactive shell we need some human readable error description -
>> so
>> > I
>> > > > > added errorDesription field where you can at least leave
>> > > > > exception.getMessage.
>> > > > >
>> > > > > b) in addition to previous item message should hold command
>> specific
>> > > > > response data. We can discuss in detail each of them but let's for
>> > now
>> > > > > agree about the overall pattern.
>> > > > >
>> > > > > Thanks,
>> > > > > Andrii Biletskyi
>> > > > >
>> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <ja...@gmail.com>
>> > > wrote:
>> > > > >
>> > > > > > Hey Joe,
>> > > > > >
>> > > > > > This is great. A few comments on KIP-4
>> > > > > >
>> > > > > > 1. This is much needed functionality, but there are a lot of
>> the so
>> > > > let's
>> > > > > > really think these protocols through. We really want to end up
>> > with a
>> > > > set
>> > > > > > of well thought-out, orthoganol apis. For this reason I think
>> it is
>> > > > > really
>> > > > > > important to think through the end state even if that includes
>> APIs
>> > > we
>> > > > > > won't implement in the first phase.
>> > > > > >
>> > > > > > 2. Let's please please please wait until we have switched the
>> > server
>> > > > over
>> > > > > > to the new java protocol definitions. If we add upteen more ad
>> hoc
>> > > > scala
>> > > > > > objects that is just generating more work for the conversion we
>> > know
>> > > we
>> > > > > > have to do.
>> > > > > >
>> > > > > > 3. This proposal introduces a new type of optional parameter.
>> This
>> > is
>> > > > > > inconsistent with everything else in the protocol where we use
>> -1
>> > or
>> > > > some
>> > > > > > other marker value. You could argue either way but let's stick
>> with
>> > > > that
>> > > > > > for consistency. For clients that implemented the protocol in a
>> > > better
>> > > > > way
>> > > > > > than our scala code these basic primitives are hard to change.
>> > > > > >
>> > > > > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
>> > > which
>> > > > > has
>> > > > > > brokers, topics, and partitions. I think we should rename that
>> > > request
>> > > > > > ClusterMetadataRequest (or just MetadataRequest) and include
>> the id
>> > > of
>> > > > > the
>> > > > > > controller. Or are there other things we could add here?
>> > > > > >
>> > > > > > 5. We have a tendency to try to make a lot of requests that can
>> > only
>> > > go
>> > > > > to
>> > > > > > particular nodes. This adds a lot of burden for client
>> > > implementations
>> > > > > (it
>> > > > > > sounds easy but each discovery can fail in many parts so it
>> ends up
>> > > > > being a
>> > > > > > full state machine to do right). I think we should consider
>> making
>> > > > admin
>> > > > > > commands and ideally as many of the other apis as possible
>> > available
>> > > on
>> > > > > all
>> > > > > > brokers and just redirect to the controller on the broker side.
>> > > Perhaps
>> > > > > > there would be a general way to encapsulate this re-routing
>> > behavior.
>> > > > > >
>> > > > > > 6. We should probably normalize the key value pairs used for
>> > configs
>> > > > > rather
>> > > > > > than embedding a new formatting. So two strings rather than one
>> > with
>> > > an
>> > > > > > internal equals sign.
>> > > > > >
>> > > > > > 7. Is the postcondition of these APIs that the command has
>> begun or
>> > > > that
>> > > > > > the command has been completed? It is a lot more usable if the
>> > > command
>> > > > > has
>> > > > > > been completed so you know that if you create a topic and then
>> > > publish
>> > > > to
>> > > > > > it you won't get an exception about there being no such topic.
>> > > > > >
>> > > > > > 8. Describe topic and list topics duplicate a lot of stuff in
>> the
>> > > > > metadata
>> > > > > > request. Is there a reason to give back topics marked for
>> > deletion? I
>> > > > > feel
>> > > > > > like if we just make the post-condition of the delete command be
>> > that
>> > > > the
>> > > > > > topic is deleted that will get rid of the need for this right?
>> And
>> > it
>> > > > > will
>> > > > > > be much more intuitive.
>> > > > > >
>> > > > > > 9. Should we consider batching these requests? We have generally
>> > > tried
>> > > > to
>> > > > > > allow multiple operations to be batched. My suspicion is that
>> > without
>> > > > > this
>> > > > > > we will get a lot of code that does something like
>> > > > > >    for(topic: adminClient.listTopics())
>> > > > > >       adminClient.describeTopic(topic)
>> > > > > > this code will work great when you test on 5 topics but not do
>> as
>> > > well
>> > > > if
>> > > > > > you have 50k.
>> > > > > >
>> > > > > > 10. I think we should also discuss how we want to expose a
>> > > programmatic
>> > > > > JVM
>> > > > > > client api for these operations. Currently people rely on
>> > AdminUtils
>> > > > > which
>> > > > > > is totally sketchy. I think we probably need another client
>> under
>> > > > > clients/
>> > > > > > that exposes administrative functionality. We will need this
>> just
>> > to
>> > > > > > properly test the new apis, I suspect. We should figure out that
>> > API.
>> > > > > >
>> > > > > > 11. The other information that would be really useful to get
>> would
>> > be
>> > > > > > information about partitions--how much data is in the partition,
>> > what
>> > > > are
>> > > > > > the segment offsets, what is the log-end offset (i.e. last
>> offset),
>> > > > what
>> > > > > is
>> > > > > > the compaction point, etc. I think that done right this would be
>> > the
>> > > > > > successor to the very awkward OffsetRequest we have today.
>> > > > > >
>> > > > > > -Jay
>> > > > > >
>> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
>> joe.stein@stealth.ly>
>> > > > > wrote:
>> > > > > >
>> > > > > > > Hi, created a KIP
>> > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>> > > > > > >
>> > > > > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
>> > > > > > >
>> > > > > > > /*******************************************
>> > > > > > >  Joe Stein
>> > > > > > >  Founder, Principal Consultant
>> > > > > > >  Big Data Open Source Security LLC
>> > > > > > >  http://www.stealth.ly
>> > > > > > >  Twitter: @allthingshadoop <
>> > http://www.twitter.com/allthingshadoop
>> > > >
>> > > > > > > ********************************************/
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>>
>>
>> --
>> -- Guozhang
>>
>
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
@Guozhang:
Thanks for your comments, I've answered some of those. The main thing is
having merged request for create-alter-delete-describe - I have some
concerns about this approach.

@*Jay*:
I see that introduced ClusterMetadaRequest is also one of the concerns. We
can solve it if we implement re-routing facility. But I agree with Guozhang -
it will make clients' internals a little bit easier but this seems to be a
complex logic to implement and support then. Especially for Fetch and
Produce (even if we add re-routing later for these requests).
Also people will tend to avoid this re-routing facility and hold local
cluster cache to ensure their high-priority requests (which some of  the
admin requests are) not sent to some busy broker where they wait to be
routed to the correct one.
As pointed out by Jun here (
https://issues.apache.org/jira/browse/KAFKA-1772?focusedCommentId=14234530&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14234530)
to solve the issue we might introduce a message type to get cluster state.
But I agree we can just update TopicMetadataResponse to include
controllerId (and probably smth else).
What are you thougths?

Thanks,
Andrii

On Thu, Feb 12, 2015 at 8:31 AM, Guozhang Wang <wa...@gmail.com> wrote:

> I think for the topics commands we can actually merge
> create/alter/delete/describe as one request type since their formats are
> very much similar, and keep list-topics and others like
> partition-reassignment / preferred-leader-election as separate request
> types, I also left some other comments on the RB (
> https://reviews.apache.org/r/29301/).
>
> On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Yeah I totally agree that we don't want to just have one "do admin stuff"
> > command that has the union of all parameters.
> >
> > What I am saying is that command line tools are one client of the
> > administrative apis, but these will be used in a number of scenarios so
> > they should make logical sense even in the absence of the command line
> > tool. Hence comments like trying to clarify the relationship between
> > ClusterMetadata and TopicMetadata...these kinds of things really need to
> be
> > thought through.
> >
> > Hope that makes sense.
> >
> > -Jay
> >
> > On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Jay,
> > >
> > > Thanks for answering. You understood correctly, most of my comments
> were
> > > related to your point 1) - about "well thought-out" apis. Also, yes,
> as I
> > > understood we would like to introduce a single unified CLI tool with
> > > centralized server-side request handling for lots of existing ones
> (incl.
> > > TopicCommand, CommitOffsetChecker, ReassignPartitions, smth else if
> added
> > > in future). In our previous discussion (
> > > https://issues.apache.org/jira/browse/KAFKA-1694) people said they'd
> > > rather
> > > have a separate message for each command, so, yes, this way I came to
> 1-1
> > > mapping between commands in the tool and protocol additions. But I
> might
> > be
> > > wrong.
> > > At the end I just try to start discussion how at least generally this
> > > protocol should look like.
> > >
> > > Thanks,
> > > Andrii
> > >
> > > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hey Andrii,
> > > >
> > > > To answer your earlier question we just really can't be adding any
> more
> > > > scala protocol objects. These things are super hard to maintain
> because
> > > > they hand code the byte parsing and don't have good versioning
> support.
> > > > Since we are already planning on converting we definitely don't want
> to
> > > add
> > > > a ton more of these--they are total tech debt.
> > > >
> > > > What does it mean that the changes are isolated from the current code
> > > base?
> > > >
> > > > I actually didn't understand the remaining comments, which of the
> > points
> > > > are you responding to?
> > > >
> > > > Maybe one sticking point here is that it seems like you want to make
> > some
> > > > kind of tool, and you have made a 1-1 mapping between commands you
> > > imagine
> > > > in the tool and protocol additions. I want to make sure we don't do
> > that.
> > > > The protocol needs to be really really well thought out against many
> > use
> > > > cases so it should make perfect logical sense in the absence of
> knowing
> > > the
> > > > command line tool, right?
> > > >
> > > > -Jay
> > > >
> > > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
> > > > andrii.biletskyi@stealth.ly> wrote:
> > > >
> > > > > Hey Jay,
> > > > >
> > > > > I would like to continue this discussion as it seem there is no
> > > progress
> > > > > here.
> > > > >
> > > > > First of all, could you please explain what did you mean in 2? How
> > > > exactly
> > > > > are we going to migrate to the new java protocol definitions. And
> why
> > > > it's
> > > > > a blocker for centralized CLI?
> > > > >
> > > > > I agree with you, this feature includes lots of stuff, but
> thankfully
> > > > > almost all changes are isolated from the current code base,
> > > > > so the main thing, I think, we need to agree is RQ/RP format.
> > > > > So how can we start discussion about the concrete messages format?
> > > > > Can we take (
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > > > )
> > > > > as starting point?
> > > > >
> > > > > We had some doubts earlier whether it worth introducing one generic
> > > Admin
> > > > > Request for all commands (
> > > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > )
> > > > > but then everybody agreed it would be better to have separate
> message
> > > for
> > > > > each admin command. The Request part is really dictated from the
> > > command
> > > > > (e.g. TopicCommand) arguments itself, so the proposed version
> should
> > be
> > > > > fine (let's put aside for now remarks about Optional type,
> batching,
> > > > > configs normalization - I agree with all of them).
> > > > > So the second part is Response. I see there are two cases here.
> > > > > a) "Mutate" requests - Create/Alter/... ; b) "Get" requests -
> > > > > List/Describe...
> > > > >
> > > > > a) should only hold request result (regardless what we decide about
> > > > > blocking/non-blocking commands execution).
> > > > > Usually we provide error code in response but since we will use
> this
> > in
> > > > > interactive shell we need some human readable error description -
> so
> > I
> > > > > added errorDesription field where you can at least leave
> > > > > exception.getMessage.
> > > > >
> > > > > b) in addition to previous item message should hold command
> specific
> > > > > response data. We can discuss in detail each of them but let's for
> > now
> > > > > agree about the overall pattern.
> > > > >
> > > > > Thanks,
> > > > > Andrii Biletskyi
> > > > >
> > > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <ja...@gmail.com>
> > > wrote:
> > > > >
> > > > > > Hey Joe,
> > > > > >
> > > > > > This is great. A few comments on KIP-4
> > > > > >
> > > > > > 1. This is much needed functionality, but there are a lot of the
> so
> > > > let's
> > > > > > really think these protocols through. We really want to end up
> > with a
> > > > set
> > > > > > of well thought-out, orthoganol apis. For this reason I think it
> is
> > > > > really
> > > > > > important to think through the end state even if that includes
> APIs
> > > we
> > > > > > won't implement in the first phase.
> > > > > >
> > > > > > 2. Let's please please please wait until we have switched the
> > server
> > > > over
> > > > > > to the new java protocol definitions. If we add upteen more ad
> hoc
> > > > scala
> > > > > > objects that is just generating more work for the conversion we
> > know
> > > we
> > > > > > have to do.
> > > > > >
> > > > > > 3. This proposal introduces a new type of optional parameter.
> This
> > is
> > > > > > inconsistent with everything else in the protocol where we use -1
> > or
> > > > some
> > > > > > other marker value. You could argue either way but let's stick
> with
> > > > that
> > > > > > for consistency. For clients that implemented the protocol in a
> > > better
> > > > > way
> > > > > > than our scala code these basic primitives are hard to change.
> > > > > >
> > > > > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
> > > which
> > > > > has
> > > > > > brokers, topics, and partitions. I think we should rename that
> > > request
> > > > > > ClusterMetadataRequest (or just MetadataRequest) and include the
> id
> > > of
> > > > > the
> > > > > > controller. Or are there other things we could add here?
> > > > > >
> > > > > > 5. We have a tendency to try to make a lot of requests that can
> > only
> > > go
> > > > > to
> > > > > > particular nodes. This adds a lot of burden for client
> > > implementations
> > > > > (it
> > > > > > sounds easy but each discovery can fail in many parts so it ends
> up
> > > > > being a
> > > > > > full state machine to do right). I think we should consider
> making
> > > > admin
> > > > > > commands and ideally as many of the other apis as possible
> > available
> > > on
> > > > > all
> > > > > > brokers and just redirect to the controller on the broker side.
> > > Perhaps
> > > > > > there would be a general way to encapsulate this re-routing
> > behavior.
> > > > > >
> > > > > > 6. We should probably normalize the key value pairs used for
> > configs
> > > > > rather
> > > > > > than embedding a new formatting. So two strings rather than one
> > with
> > > an
> > > > > > internal equals sign.
> > > > > >
> > > > > > 7. Is the postcondition of these APIs that the command has begun
> or
> > > > that
> > > > > > the command has been completed? It is a lot more usable if the
> > > command
> > > > > has
> > > > > > been completed so you know that if you create a topic and then
> > > publish
> > > > to
> > > > > > it you won't get an exception about there being no such topic.
> > > > > >
> > > > > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > > > > metadata
> > > > > > request. Is there a reason to give back topics marked for
> > deletion? I
> > > > > feel
> > > > > > like if we just make the post-condition of the delete command be
> > that
> > > > the
> > > > > > topic is deleted that will get rid of the need for this right?
> And
> > it
> > > > > will
> > > > > > be much more intuitive.
> > > > > >
> > > > > > 9. Should we consider batching these requests? We have generally
> > > tried
> > > > to
> > > > > > allow multiple operations to be batched. My suspicion is that
> > without
> > > > > this
> > > > > > we will get a lot of code that does something like
> > > > > >    for(topic: adminClient.listTopics())
> > > > > >       adminClient.describeTopic(topic)
> > > > > > this code will work great when you test on 5 topics but not do as
> > > well
> > > > if
> > > > > > you have 50k.
> > > > > >
> > > > > > 10. I think we should also discuss how we want to expose a
> > > programmatic
> > > > > JVM
> > > > > > client api for these operations. Currently people rely on
> > AdminUtils
> > > > > which
> > > > > > is totally sketchy. I think we probably need another client under
> > > > > clients/
> > > > > > that exposes administrative functionality. We will need this just
> > to
> > > > > > properly test the new apis, I suspect. We should figure out that
> > API.
> > > > > >
> > > > > > 11. The other information that would be really useful to get
> would
> > be
> > > > > > information about partitions--how much data is in the partition,
> > what
> > > > are
> > > > > > the segment offsets, what is the log-end offset (i.e. last
> offset),
> > > > what
> > > > > is
> > > > > > the compaction point, etc. I think that done right this would be
> > the
> > > > > > successor to the very awkward OffsetRequest we have today.
> > > > > >
> > > > > > -Jay
> > > > > >
> > > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <
> joe.stein@stealth.ly>
> > > > > wrote:
> > > > > >
> > > > > > > Hi, created a KIP
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > > >
> > > > > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > > >
> > > > > > > /*******************************************
> > > > > > >  Joe Stein
> > > > > > >  Founder, Principal Consultant
> > > > > > >  Big Data Open Source Security LLC
> > > > > > >  http://www.stealth.ly
> > > > > > >  Twitter: @allthingshadoop <
> > http://www.twitter.com/allthingshadoop
> > > >
> > > > > > > ********************************************/
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Guozhang Wang <wa...@gmail.com>.
I think for the topics commands we can actually merge
create/alter/delete/describe as one request type since their formats are
very much similar, and keep list-topics and others like
partition-reassignment / preferred-leader-election as separate request
types, I also left some other comments on the RB (
https://reviews.apache.org/r/29301/).

On Wed, Feb 11, 2015 at 2:04 PM, Jay Kreps <ja...@gmail.com> wrote:

> Yeah I totally agree that we don't want to just have one "do admin stuff"
> command that has the union of all parameters.
>
> What I am saying is that command line tools are one client of the
> administrative apis, but these will be used in a number of scenarios so
> they should make logical sense even in the absence of the command line
> tool. Hence comments like trying to clarify the relationship between
> ClusterMetadata and TopicMetadata...these kinds of things really need to be
> thought through.
>
> Hope that makes sense.
>
> -Jay
>
> On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Jay,
> >
> > Thanks for answering. You understood correctly, most of my comments were
> > related to your point 1) - about "well thought-out" apis. Also, yes, as I
> > understood we would like to introduce a single unified CLI tool with
> > centralized server-side request handling for lots of existing ones (incl.
> > TopicCommand, CommitOffsetChecker, ReassignPartitions, smth else if added
> > in future). In our previous discussion (
> > https://issues.apache.org/jira/browse/KAFKA-1694) people said they'd
> > rather
> > have a separate message for each command, so, yes, this way I came to 1-1
> > mapping between commands in the tool and protocol additions. But I might
> be
> > wrong.
> > At the end I just try to start discussion how at least generally this
> > protocol should look like.
> >
> > Thanks,
> > Andrii
> >
> > On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Andrii,
> > >
> > > To answer your earlier question we just really can't be adding any more
> > > scala protocol objects. These things are super hard to maintain because
> > > they hand code the byte parsing and don't have good versioning support.
> > > Since we are already planning on converting we definitely don't want to
> > add
> > > a ton more of these--they are total tech debt.
> > >
> > > What does it mean that the changes are isolated from the current code
> > base?
> > >
> > > I actually didn't understand the remaining comments, which of the
> points
> > > are you responding to?
> > >
> > > Maybe one sticking point here is that it seems like you want to make
> some
> > > kind of tool, and you have made a 1-1 mapping between commands you
> > imagine
> > > in the tool and protocol additions. I want to make sure we don't do
> that.
> > > The protocol needs to be really really well thought out against many
> use
> > > cases so it should make perfect logical sense in the absence of knowing
> > the
> > > command line tool, right?
> > >
> > > -Jay
> > >
> > > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
> > > andrii.biletskyi@stealth.ly> wrote:
> > >
> > > > Hey Jay,
> > > >
> > > > I would like to continue this discussion as it seem there is no
> > progress
> > > > here.
> > > >
> > > > First of all, could you please explain what did you mean in 2? How
> > > exactly
> > > > are we going to migrate to the new java protocol definitions. And why
> > > it's
> > > > a blocker for centralized CLI?
> > > >
> > > > I agree with you, this feature includes lots of stuff, but thankfully
> > > > almost all changes are isolated from the current code base,
> > > > so the main thing, I think, we need to agree is RQ/RP format.
> > > > So how can we start discussion about the concrete messages format?
> > > > Can we take (
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > > )
> > > > as starting point?
> > > >
> > > > We had some doubts earlier whether it worth introducing one generic
> > Admin
> > > > Request for all commands (
> > > https://issues.apache.org/jira/browse/KAFKA-1694
> > > > )
> > > > but then everybody agreed it would be better to have separate message
> > for
> > > > each admin command. The Request part is really dictated from the
> > command
> > > > (e.g. TopicCommand) arguments itself, so the proposed version should
> be
> > > > fine (let's put aside for now remarks about Optional type, batching,
> > > > configs normalization - I agree with all of them).
> > > > So the second part is Response. I see there are two cases here.
> > > > a) "Mutate" requests - Create/Alter/... ; b) "Get" requests -
> > > > List/Describe...
> > > >
> > > > a) should only hold request result (regardless what we decide about
> > > > blocking/non-blocking commands execution).
> > > > Usually we provide error code in response but since we will use this
> in
> > > > interactive shell we need some human readable error description - so
> I
> > > > added errorDesription field where you can at least leave
> > > > exception.getMessage.
> > > >
> > > > b) in addition to previous item message should hold command specific
> > > > response data. We can discuss in detail each of them but let's for
> now
> > > > agree about the overall pattern.
> > > >
> > > > Thanks,
> > > > Andrii Biletskyi
> > > >
> > > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <ja...@gmail.com>
> > wrote:
> > > >
> > > > > Hey Joe,
> > > > >
> > > > > This is great. A few comments on KIP-4
> > > > >
> > > > > 1. This is much needed functionality, but there are a lot of the so
> > > let's
> > > > > really think these protocols through. We really want to end up
> with a
> > > set
> > > > > of well thought-out, orthoganol apis. For this reason I think it is
> > > > really
> > > > > important to think through the end state even if that includes APIs
> > we
> > > > > won't implement in the first phase.
> > > > >
> > > > > 2. Let's please please please wait until we have switched the
> server
> > > over
> > > > > to the new java protocol definitions. If we add upteen more ad hoc
> > > scala
> > > > > objects that is just generating more work for the conversion we
> know
> > we
> > > > > have to do.
> > > > >
> > > > > 3. This proposal introduces a new type of optional parameter. This
> is
> > > > > inconsistent with everything else in the protocol where we use -1
> or
> > > some
> > > > > other marker value. You could argue either way but let's stick with
> > > that
> > > > > for consistency. For clients that implemented the protocol in a
> > better
> > > > way
> > > > > than our scala code these basic primitives are hard to change.
> > > > >
> > > > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
> > which
> > > > has
> > > > > brokers, topics, and partitions. I think we should rename that
> > request
> > > > > ClusterMetadataRequest (or just MetadataRequest) and include the id
> > of
> > > > the
> > > > > controller. Or are there other things we could add here?
> > > > >
> > > > > 5. We have a tendency to try to make a lot of requests that can
> only
> > go
> > > > to
> > > > > particular nodes. This adds a lot of burden for client
> > implementations
> > > > (it
> > > > > sounds easy but each discovery can fail in many parts so it ends up
> > > > being a
> > > > > full state machine to do right). I think we should consider making
> > > admin
> > > > > commands and ideally as many of the other apis as possible
> available
> > on
> > > > all
> > > > > brokers and just redirect to the controller on the broker side.
> > Perhaps
> > > > > there would be a general way to encapsulate this re-routing
> behavior.
> > > > >
> > > > > 6. We should probably normalize the key value pairs used for
> configs
> > > > rather
> > > > > than embedding a new formatting. So two strings rather than one
> with
> > an
> > > > > internal equals sign.
> > > > >
> > > > > 7. Is the postcondition of these APIs that the command has begun or
> > > that
> > > > > the command has been completed? It is a lot more usable if the
> > command
> > > > has
> > > > > been completed so you know that if you create a topic and then
> > publish
> > > to
> > > > > it you won't get an exception about there being no such topic.
> > > > >
> > > > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > > > metadata
> > > > > request. Is there a reason to give back topics marked for
> deletion? I
> > > > feel
> > > > > like if we just make the post-condition of the delete command be
> that
> > > the
> > > > > topic is deleted that will get rid of the need for this right? And
> it
> > > > will
> > > > > be much more intuitive.
> > > > >
> > > > > 9. Should we consider batching these requests? We have generally
> > tried
> > > to
> > > > > allow multiple operations to be batched. My suspicion is that
> without
> > > > this
> > > > > we will get a lot of code that does something like
> > > > >    for(topic: adminClient.listTopics())
> > > > >       adminClient.describeTopic(topic)
> > > > > this code will work great when you test on 5 topics but not do as
> > well
> > > if
> > > > > you have 50k.
> > > > >
> > > > > 10. I think we should also discuss how we want to expose a
> > programmatic
> > > > JVM
> > > > > client api for these operations. Currently people rely on
> AdminUtils
> > > > which
> > > > > is totally sketchy. I think we probably need another client under
> > > > clients/
> > > > > that exposes administrative functionality. We will need this just
> to
> > > > > properly test the new apis, I suspect. We should figure out that
> API.
> > > > >
> > > > > 11. The other information that would be really useful to get would
> be
> > > > > information about partitions--how much data is in the partition,
> what
> > > are
> > > > > the segment offsets, what is the log-end offset (i.e. last offset),
> > > what
> > > > is
> > > > > the compaction point, etc. I think that done right this would be
> the
> > > > > successor to the very awkward OffsetRequest we have today.
> > > > >
> > > > > -Jay
> > > > >
> > > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly>
> > > > wrote:
> > > > >
> > > > > > Hi, created a KIP
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > > >
> > > > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> > > > > >
> > > > > > /*******************************************
> > > > > >  Joe Stein
> > > > > >  Founder, Principal Consultant
> > > > > >  Big Data Open Source Security LLC
> > > > > >  http://www.stealth.ly
> > > > > >  Twitter: @allthingshadoop <
> http://www.twitter.com/allthingshadoop
> > >
> > > > > > ********************************************/
> > > > > >
> > > > >
> > > >
> > >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
Yeah I totally agree that we don't want to just have one "do admin stuff"
command that has the union of all parameters.

What I am saying is that command line tools are one client of the
administrative apis, but these will be used in a number of scenarios so
they should make logical sense even in the absence of the command line
tool. Hence comments like trying to clarify the relationship between
ClusterMetadata and TopicMetadata...these kinds of things really need to be
thought through.

Hope that makes sense.

-Jay

On Wed, Feb 11, 2015 at 1:41 PM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Jay,
>
> Thanks for answering. You understood correctly, most of my comments were
> related to your point 1) - about "well thought-out" apis. Also, yes, as I
> understood we would like to introduce a single unified CLI tool with
> centralized server-side request handling for lots of existing ones (incl.
> TopicCommand, CommitOffsetChecker, ReassignPartitions, smth else if added
> in future). In our previous discussion (
> https://issues.apache.org/jira/browse/KAFKA-1694) people said they'd
> rather
> have a separate message for each command, so, yes, this way I came to 1-1
> mapping between commands in the tool and protocol additions. But I might be
> wrong.
> At the end I just try to start discussion how at least generally this
> protocol should look like.
>
> Thanks,
> Andrii
>
> On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Andrii,
> >
> > To answer your earlier question we just really can't be adding any more
> > scala protocol objects. These things are super hard to maintain because
> > they hand code the byte parsing and don't have good versioning support.
> > Since we are already planning on converting we definitely don't want to
> add
> > a ton more of these--they are total tech debt.
> >
> > What does it mean that the changes are isolated from the current code
> base?
> >
> > I actually didn't understand the remaining comments, which of the points
> > are you responding to?
> >
> > Maybe one sticking point here is that it seems like you want to make some
> > kind of tool, and you have made a 1-1 mapping between commands you
> imagine
> > in the tool and protocol additions. I want to make sure we don't do that.
> > The protocol needs to be really really well thought out against many use
> > cases so it should make perfect logical sense in the absence of knowing
> the
> > command line tool, right?
> >
> > -Jay
> >
> > On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
> > andrii.biletskyi@stealth.ly> wrote:
> >
> > > Hey Jay,
> > >
> > > I would like to continue this discussion as it seem there is no
> progress
> > > here.
> > >
> > > First of all, could you please explain what did you mean in 2? How
> > exactly
> > > are we going to migrate to the new java protocol definitions. And why
> > it's
> > > a blocker for centralized CLI?
> > >
> > > I agree with you, this feature includes lots of stuff, but thankfully
> > > almost all changes are isolated from the current code base,
> > > so the main thing, I think, we need to agree is RQ/RP format.
> > > So how can we start discussion about the concrete messages format?
> > > Can we take (
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > > )
> > > as starting point?
> > >
> > > We had some doubts earlier whether it worth introducing one generic
> Admin
> > > Request for all commands (
> > https://issues.apache.org/jira/browse/KAFKA-1694
> > > )
> > > but then everybody agreed it would be better to have separate message
> for
> > > each admin command. The Request part is really dictated from the
> command
> > > (e.g. TopicCommand) arguments itself, so the proposed version should be
> > > fine (let's put aside for now remarks about Optional type, batching,
> > > configs normalization - I agree with all of them).
> > > So the second part is Response. I see there are two cases here.
> > > a) "Mutate" requests - Create/Alter/... ; b) "Get" requests -
> > > List/Describe...
> > >
> > > a) should only hold request result (regardless what we decide about
> > > blocking/non-blocking commands execution).
> > > Usually we provide error code in response but since we will use this in
> > > interactive shell we need some human readable error description - so I
> > > added errorDesription field where you can at least leave
> > > exception.getMessage.
> > >
> > > b) in addition to previous item message should hold command specific
> > > response data. We can discuss in detail each of them but let's for now
> > > agree about the overall pattern.
> > >
> > > Thanks,
> > > Andrii Biletskyi
> > >
> > > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <ja...@gmail.com>
> wrote:
> > >
> > > > Hey Joe,
> > > >
> > > > This is great. A few comments on KIP-4
> > > >
> > > > 1. This is much needed functionality, but there are a lot of the so
> > let's
> > > > really think these protocols through. We really want to end up with a
> > set
> > > > of well thought-out, orthoganol apis. For this reason I think it is
> > > really
> > > > important to think through the end state even if that includes APIs
> we
> > > > won't implement in the first phase.
> > > >
> > > > 2. Let's please please please wait until we have switched the server
> > over
> > > > to the new java protocol definitions. If we add upteen more ad hoc
> > scala
> > > > objects that is just generating more work for the conversion we know
> we
> > > > have to do.
> > > >
> > > > 3. This proposal introduces a new type of optional parameter. This is
> > > > inconsistent with everything else in the protocol where we use -1 or
> > some
> > > > other marker value. You could argue either way but let's stick with
> > that
> > > > for consistency. For clients that implemented the protocol in a
> better
> > > way
> > > > than our scala code these basic primitives are hard to change.
> > > >
> > > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest
> which
> > > has
> > > > brokers, topics, and partitions. I think we should rename that
> request
> > > > ClusterMetadataRequest (or just MetadataRequest) and include the id
> of
> > > the
> > > > controller. Or are there other things we could add here?
> > > >
> > > > 5. We have a tendency to try to make a lot of requests that can only
> go
> > > to
> > > > particular nodes. This adds a lot of burden for client
> implementations
> > > (it
> > > > sounds easy but each discovery can fail in many parts so it ends up
> > > being a
> > > > full state machine to do right). I think we should consider making
> > admin
> > > > commands and ideally as many of the other apis as possible available
> on
> > > all
> > > > brokers and just redirect to the controller on the broker side.
> Perhaps
> > > > there would be a general way to encapsulate this re-routing behavior.
> > > >
> > > > 6. We should probably normalize the key value pairs used for configs
> > > rather
> > > > than embedding a new formatting. So two strings rather than one with
> an
> > > > internal equals sign.
> > > >
> > > > 7. Is the postcondition of these APIs that the command has begun or
> > that
> > > > the command has been completed? It is a lot more usable if the
> command
> > > has
> > > > been completed so you know that if you create a topic and then
> publish
> > to
> > > > it you won't get an exception about there being no such topic.
> > > >
> > > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > > metadata
> > > > request. Is there a reason to give back topics marked for deletion? I
> > > feel
> > > > like if we just make the post-condition of the delete command be that
> > the
> > > > topic is deleted that will get rid of the need for this right? And it
> > > will
> > > > be much more intuitive.
> > > >
> > > > 9. Should we consider batching these requests? We have generally
> tried
> > to
> > > > allow multiple operations to be batched. My suspicion is that without
> > > this
> > > > we will get a lot of code that does something like
> > > >    for(topic: adminClient.listTopics())
> > > >       adminClient.describeTopic(topic)
> > > > this code will work great when you test on 5 topics but not do as
> well
> > if
> > > > you have 50k.
> > > >
> > > > 10. I think we should also discuss how we want to expose a
> programmatic
> > > JVM
> > > > client api for these operations. Currently people rely on AdminUtils
> > > which
> > > > is totally sketchy. I think we probably need another client under
> > > clients/
> > > > that exposes administrative functionality. We will need this just to
> > > > properly test the new apis, I suspect. We should figure out that API.
> > > >
> > > > 11. The other information that would be really useful to get would be
> > > > information about partitions--how much data is in the partition, what
> > are
> > > > the segment offsets, what is the log-end offset (i.e. last offset),
> > what
> > > is
> > > > the compaction point, etc. I think that done right this would be the
> > > > successor to the very awkward OffsetRequest we have today.
> > > >
> > > > -Jay
> > > >
> > > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly>
> > > wrote:
> > > >
> > > > > Hi, created a KIP
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > > >
> > > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> > > > >
> > > > > /*******************************************
> > > > >  Joe Stein
> > > > >  Founder, Principal Consultant
> > > > >  Big Data Open Source Security LLC
> > > > >  http://www.stealth.ly
> > > > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop
> >
> > > > > ********************************************/
> > > > >
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Jay,

Thanks for answering. You understood correctly, most of my comments were
related to your point 1) - about "well thought-out" apis. Also, yes, as I
understood we would like to introduce a single unified CLI tool with
centralized server-side request handling for lots of existing ones (incl.
TopicCommand, CommitOffsetChecker, ReassignPartitions, smth else if added
in future). In our previous discussion (
https://issues.apache.org/jira/browse/KAFKA-1694) people said they'd rather
have a separate message for each command, so, yes, this way I came to 1-1
mapping between commands in the tool and protocol additions. But I might be
wrong.
At the end I just try to start discussion how at least generally this
protocol should look like.

Thanks,
Andrii

On Wed, Feb 11, 2015 at 11:10 PM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Andrii,
>
> To answer your earlier question we just really can't be adding any more
> scala protocol objects. These things are super hard to maintain because
> they hand code the byte parsing and don't have good versioning support.
> Since we are already planning on converting we definitely don't want to add
> a ton more of these--they are total tech debt.
>
> What does it mean that the changes are isolated from the current code base?
>
> I actually didn't understand the remaining comments, which of the points
> are you responding to?
>
> Maybe one sticking point here is that it seems like you want to make some
> kind of tool, and you have made a 1-1 mapping between commands you imagine
> in the tool and protocol additions. I want to make sure we don't do that.
> The protocol needs to be really really well thought out against many use
> cases so it should make perfect logical sense in the absence of knowing the
> command line tool, right?
>
> -Jay
>
> On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
> andrii.biletskyi@stealth.ly> wrote:
>
> > Hey Jay,
> >
> > I would like to continue this discussion as it seem there is no progress
> > here.
> >
> > First of all, could you please explain what did you mean in 2? How
> exactly
> > are we going to migrate to the new java protocol definitions. And why
> it's
> > a blocker for centralized CLI?
> >
> > I agree with you, this feature includes lots of stuff, but thankfully
> > almost all changes are isolated from the current code base,
> > so the main thing, I think, we need to agree is RQ/RP format.
> > So how can we start discussion about the concrete messages format?
> > Can we take (
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> > )
> > as starting point?
> >
> > We had some doubts earlier whether it worth introducing one generic Admin
> > Request for all commands (
> https://issues.apache.org/jira/browse/KAFKA-1694
> > )
> > but then everybody agreed it would be better to have separate message for
> > each admin command. The Request part is really dictated from the command
> > (e.g. TopicCommand) arguments itself, so the proposed version should be
> > fine (let's put aside for now remarks about Optional type, batching,
> > configs normalization - I agree with all of them).
> > So the second part is Response. I see there are two cases here.
> > a) "Mutate" requests - Create/Alter/... ; b) "Get" requests -
> > List/Describe...
> >
> > a) should only hold request result (regardless what we decide about
> > blocking/non-blocking commands execution).
> > Usually we provide error code in response but since we will use this in
> > interactive shell we need some human readable error description - so I
> > added errorDesription field where you can at least leave
> > exception.getMessage.
> >
> > b) in addition to previous item message should hold command specific
> > response data. We can discuss in detail each of them but let's for now
> > agree about the overall pattern.
> >
> > Thanks,
> > Andrii Biletskyi
> >
> > On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <ja...@gmail.com> wrote:
> >
> > > Hey Joe,
> > >
> > > This is great. A few comments on KIP-4
> > >
> > > 1. This is much needed functionality, but there are a lot of the so
> let's
> > > really think these protocols through. We really want to end up with a
> set
> > > of well thought-out, orthoganol apis. For this reason I think it is
> > really
> > > important to think through the end state even if that includes APIs we
> > > won't implement in the first phase.
> > >
> > > 2. Let's please please please wait until we have switched the server
> over
> > > to the new java protocol definitions. If we add upteen more ad hoc
> scala
> > > objects that is just generating more work for the conversion we know we
> > > have to do.
> > >
> > > 3. This proposal introduces a new type of optional parameter. This is
> > > inconsistent with everything else in the protocol where we use -1 or
> some
> > > other marker value. You could argue either way but let's stick with
> that
> > > for consistency. For clients that implemented the protocol in a better
> > way
> > > than our scala code these basic primitives are hard to change.
> > >
> > > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest which
> > has
> > > brokers, topics, and partitions. I think we should rename that request
> > > ClusterMetadataRequest (or just MetadataRequest) and include the id of
> > the
> > > controller. Or are there other things we could add here?
> > >
> > > 5. We have a tendency to try to make a lot of requests that can only go
> > to
> > > particular nodes. This adds a lot of burden for client implementations
> > (it
> > > sounds easy but each discovery can fail in many parts so it ends up
> > being a
> > > full state machine to do right). I think we should consider making
> admin
> > > commands and ideally as many of the other apis as possible available on
> > all
> > > brokers and just redirect to the controller on the broker side. Perhaps
> > > there would be a general way to encapsulate this re-routing behavior.
> > >
> > > 6. We should probably normalize the key value pairs used for configs
> > rather
> > > than embedding a new formatting. So two strings rather than one with an
> > > internal equals sign.
> > >
> > > 7. Is the postcondition of these APIs that the command has begun or
> that
> > > the command has been completed? It is a lot more usable if the command
> > has
> > > been completed so you know that if you create a topic and then publish
> to
> > > it you won't get an exception about there being no such topic.
> > >
> > > 8. Describe topic and list topics duplicate a lot of stuff in the
> > metadata
> > > request. Is there a reason to give back topics marked for deletion? I
> > feel
> > > like if we just make the post-condition of the delete command be that
> the
> > > topic is deleted that will get rid of the need for this right? And it
> > will
> > > be much more intuitive.
> > >
> > > 9. Should we consider batching these requests? We have generally tried
> to
> > > allow multiple operations to be batched. My suspicion is that without
> > this
> > > we will get a lot of code that does something like
> > >    for(topic: adminClient.listTopics())
> > >       adminClient.describeTopic(topic)
> > > this code will work great when you test on 5 topics but not do as well
> if
> > > you have 50k.
> > >
> > > 10. I think we should also discuss how we want to expose a programmatic
> > JVM
> > > client api for these operations. Currently people rely on AdminUtils
> > which
> > > is totally sketchy. I think we probably need another client under
> > clients/
> > > that exposes administrative functionality. We will need this just to
> > > properly test the new apis, I suspect. We should figure out that API.
> > >
> > > 11. The other information that would be really useful to get would be
> > > information about partitions--how much data is in the partition, what
> are
> > > the segment offsets, what is the log-end offset (i.e. last offset),
> what
> > is
> > > the compaction point, etc. I think that done right this would be the
> > > successor to the very awkward OffsetRequest we have today.
> > >
> > > -Jay
> > >
> > > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly>
> > wrote:
> > >
> > > > Hi, created a KIP
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > > >
> > > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> > > >
> > > > /*******************************************
> > > >  Joe Stein
> > > >  Founder, Principal Consultant
> > > >  Big Data Open Source Security LLC
> > > >  http://www.stealth.ly
> > > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > > ********************************************/
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
Hey Andrii,

To answer your earlier question we just really can't be adding any more
scala protocol objects. These things are super hard to maintain because
they hand code the byte parsing and don't have good versioning support.
Since we are already planning on converting we definitely don't want to add
a ton more of these--they are total tech debt.

What does it mean that the changes are isolated from the current code base?

I actually didn't understand the remaining comments, which of the points
are you responding to?

Maybe one sticking point here is that it seems like you want to make some
kind of tool, and you have made a 1-1 mapping between commands you imagine
in the tool and protocol additions. I want to make sure we don't do that.
The protocol needs to be really really well thought out against many use
cases so it should make perfect logical sense in the absence of knowing the
command line tool, right?

-Jay

On Wed, Feb 11, 2015 at 11:57 AM, Andrii Biletskyi <
andrii.biletskyi@stealth.ly> wrote:

> Hey Jay,
>
> I would like to continue this discussion as it seem there is no progress
> here.
>
> First of all, could you please explain what did you mean in 2? How exactly
> are we going to migrate to the new java protocol definitions. And why it's
> a blocker for centralized CLI?
>
> I agree with you, this feature includes lots of stuff, but thankfully
> almost all changes are isolated from the current code base,
> so the main thing, I think, we need to agree is RQ/RP format.
> So how can we start discussion about the concrete messages format?
> Can we take (
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat
> )
> as starting point?
>
> We had some doubts earlier whether it worth introducing one generic Admin
> Request for all commands (https://issues.apache.org/jira/browse/KAFKA-1694
> )
> but then everybody agreed it would be better to have separate message for
> each admin command. The Request part is really dictated from the command
> (e.g. TopicCommand) arguments itself, so the proposed version should be
> fine (let's put aside for now remarks about Optional type, batching,
> configs normalization - I agree with all of them).
> So the second part is Response. I see there are two cases here.
> a) "Mutate" requests - Create/Alter/... ; b) "Get" requests -
> List/Describe...
>
> a) should only hold request result (regardless what we decide about
> blocking/non-blocking commands execution).
> Usually we provide error code in response but since we will use this in
> interactive shell we need some human readable error description - so I
> added errorDesription field where you can at least leave
> exception.getMessage.
>
> b) in addition to previous item message should hold command specific
> response data. We can discuss in detail each of them but let's for now
> agree about the overall pattern.
>
> Thanks,
> Andrii Biletskyi
>
> On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey Joe,
> >
> > This is great. A few comments on KIP-4
> >
> > 1. This is much needed functionality, but there are a lot of the so let's
> > really think these protocols through. We really want to end up with a set
> > of well thought-out, orthoganol apis. For this reason I think it is
> really
> > important to think through the end state even if that includes APIs we
> > won't implement in the first phase.
> >
> > 2. Let's please please please wait until we have switched the server over
> > to the new java protocol definitions. If we add upteen more ad hoc scala
> > objects that is just generating more work for the conversion we know we
> > have to do.
> >
> > 3. This proposal introduces a new type of optional parameter. This is
> > inconsistent with everything else in the protocol where we use -1 or some
> > other marker value. You could argue either way but let's stick with that
> > for consistency. For clients that implemented the protocol in a better
> way
> > than our scala code these basic primitives are hard to change.
> >
> > 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest which
> has
> > brokers, topics, and partitions. I think we should rename that request
> > ClusterMetadataRequest (or just MetadataRequest) and include the id of
> the
> > controller. Or are there other things we could add here?
> >
> > 5. We have a tendency to try to make a lot of requests that can only go
> to
> > particular nodes. This adds a lot of burden for client implementations
> (it
> > sounds easy but each discovery can fail in many parts so it ends up
> being a
> > full state machine to do right). I think we should consider making admin
> > commands and ideally as many of the other apis as possible available on
> all
> > brokers and just redirect to the controller on the broker side. Perhaps
> > there would be a general way to encapsulate this re-routing behavior.
> >
> > 6. We should probably normalize the key value pairs used for configs
> rather
> > than embedding a new formatting. So two strings rather than one with an
> > internal equals sign.
> >
> > 7. Is the postcondition of these APIs that the command has begun or that
> > the command has been completed? It is a lot more usable if the command
> has
> > been completed so you know that if you create a topic and then publish to
> > it you won't get an exception about there being no such topic.
> >
> > 8. Describe topic and list topics duplicate a lot of stuff in the
> metadata
> > request. Is there a reason to give back topics marked for deletion? I
> feel
> > like if we just make the post-condition of the delete command be that the
> > topic is deleted that will get rid of the need for this right? And it
> will
> > be much more intuitive.
> >
> > 9. Should we consider batching these requests? We have generally tried to
> > allow multiple operations to be batched. My suspicion is that without
> this
> > we will get a lot of code that does something like
> >    for(topic: adminClient.listTopics())
> >       adminClient.describeTopic(topic)
> > this code will work great when you test on 5 topics but not do as well if
> > you have 50k.
> >
> > 10. I think we should also discuss how we want to expose a programmatic
> JVM
> > client api for these operations. Currently people rely on AdminUtils
> which
> > is totally sketchy. I think we probably need another client under
> clients/
> > that exposes administrative functionality. We will need this just to
> > properly test the new apis, I suspect. We should figure out that API.
> >
> > 11. The other information that would be really useful to get would be
> > information about partitions--how much data is in the partition, what are
> > the segment offsets, what is the log-end offset (i.e. last offset), what
> is
> > the compaction point, etc. I think that done right this would be the
> > successor to the very awkward OffsetRequest we have today.
> >
> > -Jay
> >
> > On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly>
> wrote:
> >
> > > Hi, created a KIP
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> > >
> > > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> > >
> > > /*******************************************
> > >  Joe Stein
> > >  Founder, Principal Consultant
> > >  Big Data Open Source Security LLC
> > >  http://www.stealth.ly
> > >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > > ********************************************/
> > >
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Andrii Biletskyi <an...@stealth.ly>.
Hey Jay,

I would like to continue this discussion as it seem there is no progress
here.

First of all, could you please explain what did you mean in 2? How exactly
are we going to migrate to the new java protocol definitions. And why it's
a blocker for centralized CLI?

I agree with you, this feature includes lots of stuff, but thankfully
almost all changes are isolated from the current code base,
so the main thing, I think, we need to agree is RQ/RP format.
So how can we start discussion about the concrete messages format?
Can we take (
https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations#KIP-4-Commandlineandcentralizedadministrativeoperations-ProposedRQ/RPFormat)
as starting point?

We had some doubts earlier whether it worth introducing one generic Admin
Request for all commands (https://issues.apache.org/jira/browse/KAFKA-1694)
but then everybody agreed it would be better to have separate message for
each admin command. The Request part is really dictated from the command
(e.g. TopicCommand) arguments itself, so the proposed version should be
fine (let's put aside for now remarks about Optional type, batching,
configs normalization - I agree with all of them).
So the second part is Response. I see there are two cases here.
a) "Mutate" requests - Create/Alter/... ; b) "Get" requests -
List/Describe...

a) should only hold request result (regardless what we decide about
blocking/non-blocking commands execution).
Usually we provide error code in response but since we will use this in
interactive shell we need some human readable error description - so I
added errorDesription field where you can at least leave
exception.getMessage.

b) in addition to previous item message should hold command specific
response data. We can discuss in detail each of them but let's for now
agree about the overall pattern.

Thanks,
Andrii Biletskyi

On Fri, Jan 23, 2015 at 6:59 AM, Jay Kreps <ja...@gmail.com> wrote:

> Hey Joe,
>
> This is great. A few comments on KIP-4
>
> 1. This is much needed functionality, but there are a lot of the so let's
> really think these protocols through. We really want to end up with a set
> of well thought-out, orthoganol apis. For this reason I think it is really
> important to think through the end state even if that includes APIs we
> won't implement in the first phase.
>
> 2. Let's please please please wait until we have switched the server over
> to the new java protocol definitions. If we add upteen more ad hoc scala
> objects that is just generating more work for the conversion we know we
> have to do.
>
> 3. This proposal introduces a new type of optional parameter. This is
> inconsistent with everything else in the protocol where we use -1 or some
> other marker value. You could argue either way but let's stick with that
> for consistency. For clients that implemented the protocol in a better way
> than our scala code these basic primitives are hard to change.
>
> 4. ClusterMetadata: This seems to duplicate TopicMetadataRequest which has
> brokers, topics, and partitions. I think we should rename that request
> ClusterMetadataRequest (or just MetadataRequest) and include the id of the
> controller. Or are there other things we could add here?
>
> 5. We have a tendency to try to make a lot of requests that can only go to
> particular nodes. This adds a lot of burden for client implementations (it
> sounds easy but each discovery can fail in many parts so it ends up being a
> full state machine to do right). I think we should consider making admin
> commands and ideally as many of the other apis as possible available on all
> brokers and just redirect to the controller on the broker side. Perhaps
> there would be a general way to encapsulate this re-routing behavior.
>
> 6. We should probably normalize the key value pairs used for configs rather
> than embedding a new formatting. So two strings rather than one with an
> internal equals sign.
>
> 7. Is the postcondition of these APIs that the command has begun or that
> the command has been completed? It is a lot more usable if the command has
> been completed so you know that if you create a topic and then publish to
> it you won't get an exception about there being no such topic.
>
> 8. Describe topic and list topics duplicate a lot of stuff in the metadata
> request. Is there a reason to give back topics marked for deletion? I feel
> like if we just make the post-condition of the delete command be that the
> topic is deleted that will get rid of the need for this right? And it will
> be much more intuitive.
>
> 9. Should we consider batching these requests? We have generally tried to
> allow multiple operations to be batched. My suspicion is that without this
> we will get a lot of code that does something like
>    for(topic: adminClient.listTopics())
>       adminClient.describeTopic(topic)
> this code will work great when you test on 5 topics but not do as well if
> you have 50k.
>
> 10. I think we should also discuss how we want to expose a programmatic JVM
> client api for these operations. Currently people rely on AdminUtils which
> is totally sketchy. I think we probably need another client under clients/
> that exposes administrative functionality. We will need this just to
> properly test the new apis, I suspect. We should figure out that API.
>
> 11. The other information that would be really useful to get would be
> information about partitions--how much data is in the partition, what are
> the segment offsets, what is the log-end offset (i.e. last offset), what is
> the compaction point, etc. I think that done right this would be the
> successor to the very awkward OffsetRequest we have today.
>
> -Jay
>
> On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly> wrote:
>
> > Hi, created a KIP
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
> >
> > JIRA https://issues.apache.org/jira/browse/KAFKA-1694
> >
> > /*******************************************
> >  Joe Stein
> >  Founder, Principal Consultant
> >  Big Data Open Source Security LLC
> >  http://www.stealth.ly
> >  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> > ********************************************/
> >
>

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations

Posted by Jay Kreps <ja...@gmail.com>.
Hey Joe,

This is great. A few comments on KIP-4

1. This is much needed functionality, but there are a lot of the so let's
really think these protocols through. We really want to end up with a set
of well thought-out, orthoganol apis. For this reason I think it is really
important to think through the end state even if that includes APIs we
won't implement in the first phase.

2. Let's please please please wait until we have switched the server over
to the new java protocol definitions. If we add upteen more ad hoc scala
objects that is just generating more work for the conversion we know we
have to do.

3. This proposal introduces a new type of optional parameter. This is
inconsistent with everything else in the protocol where we use -1 or some
other marker value. You could argue either way but let's stick with that
for consistency. For clients that implemented the protocol in a better way
than our scala code these basic primitives are hard to change.

4. ClusterMetadata: This seems to duplicate TopicMetadataRequest which has
brokers, topics, and partitions. I think we should rename that request
ClusterMetadataRequest (or just MetadataRequest) and include the id of the
controller. Or are there other things we could add here?

5. We have a tendency to try to make a lot of requests that can only go to
particular nodes. This adds a lot of burden for client implementations (it
sounds easy but each discovery can fail in many parts so it ends up being a
full state machine to do right). I think we should consider making admin
commands and ideally as many of the other apis as possible available on all
brokers and just redirect to the controller on the broker side. Perhaps
there would be a general way to encapsulate this re-routing behavior.

6. We should probably normalize the key value pairs used for configs rather
than embedding a new formatting. So two strings rather than one with an
internal equals sign.

7. Is the postcondition of these APIs that the command has begun or that
the command has been completed? It is a lot more usable if the command has
been completed so you know that if you create a topic and then publish to
it you won't get an exception about there being no such topic.

8. Describe topic and list topics duplicate a lot of stuff in the metadata
request. Is there a reason to give back topics marked for deletion? I feel
like if we just make the post-condition of the delete command be that the
topic is deleted that will get rid of the need for this right? And it will
be much more intuitive.

9. Should we consider batching these requests? We have generally tried to
allow multiple operations to be batched. My suspicion is that without this
we will get a lot of code that does something like
   for(topic: adminClient.listTopics())
      adminClient.describeTopic(topic)
this code will work great when you test on 5 topics but not do as well if
you have 50k.

10. I think we should also discuss how we want to expose a programmatic JVM
client api for these operations. Currently people rely on AdminUtils which
is totally sketchy. I think we probably need another client under clients/
that exposes administrative functionality. We will need this just to
properly test the new apis, I suspect. We should figure out that API.

11. The other information that would be really useful to get would be
information about partitions--how much data is in the partition, what are
the segment offsets, what is the log-end offset (i.e. last offset), what is
the compaction point, etc. I think that done right this would be the
successor to the very awkward OffsetRequest we have today.

-Jay

On Wed, Jan 21, 2015 at 10:27 PM, Joe Stein <jo...@stealth.ly> wrote:

> Hi, created a KIP
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations
>
> JIRA https://issues.apache.org/jira/browse/KAFKA-1694
>
> /*******************************************
>  Joe Stein
>  Founder, Principal Consultant
>  Big Data Open Source Security LLC
>  http://www.stealth.ly
>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> ********************************************/
>