+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 <j...@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.bilets...@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 <j...@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 <jjkosh...@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.bilets...@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.bilets...@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 <
> > jjkosh...@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 <j...@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.bilets...@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 <
> > > j...@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.bilets...@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 <
> > > > > j...@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.bilets...@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.bilets...@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
> > <
> > > > > > > >> > > > > gshap...@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
> > > <
> > > > > > > >> > > > > > gshap...@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
> > > > <
> > > > > > > >> > > > > > wangg...@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
> > > > <
> > > > > > > >> > > > > > wangg...@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 <
> > > > > > > >> > > > > > jjkosh...@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.st...@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.kr...@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
> > > > > > > >> <
> > > > > > > >> > > > > > > > >> >>>> jholo...@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.kr...@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.st...@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.kr...@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.bilets...@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.bilets...@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.kr...@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.bilets...@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]
> > > > > > >
> > > > >
> > > > >
> > > >
> > >
> >

Reply via email to