The interface looks good overall. Thanks for the much needed work Colin.
Just a few comments:

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

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

Thanks,

Jiangjie (Becket) Qin

On Tue, Mar 14, 2017 at 10:15 AM, Ismael Juma <ism...@juma.me.uk> wrote:

> Hi Gwen,
>
> I agree with you that it's unfortunate that we now have 4 methods per
> operation (including the batch ones). I personally still prefer the option
> of using a single class to represent the parameters of each method (i.e. 1
> batch and 1 non-batch method per operation). I would follow a very simple
> and boring pattern for each operation:
>
> CreateTopicResults createTopic(CreateTopicRequest request);
> CreateTopicsResults createTopics(CreateTopicsRequest request);
>
> This is simple to understand, easy to keep consistent as we add methods and
> easy to evolve. A downside is that it's a bit more verbose than the current
> proposal.
>
> Regarding LogConfig, we can't use it as it's in the core module (and in
> Scala).
>
> I'm not too sure about the Unstable point. A couple of points:
>
> 1. A major reason why we are adding the AdminClient is so that people don't
> have to deal with incompatible changes. So, unless we truly have to, I
> don't think we'll be making incompatible changes after the release.
>
> 2. If we make changes to unstable code, it still has to go through the KIP
> process. If they're minor changes in this release cycle, we can simply send
> an email to the discuss thread, but if it's a major change, then it would
> require a new round of voting (or a new KIP).
>
> Anyway, I think we're close on this one, but it would be good to agree on
> the general pattern so that we can easily add more methods.
>
> Ismael
>
> On Tue, Mar 14, 2017 at 5:29 AM, Gwen Shapira <g...@confluent.io> wrote:
>
> > I'm torn between my desire to get this in already and the fact that parts
> > of the API feel a bit alien to Kafka.
> >
> > I will resolve my difficulties by giving my feedback here and then going
> to
> > vote +1 on the vote thread.
> > Colin can choose whether to address my concerns now or use his "unstable"
> > option to wait and see...
> >
> > My main concern is the RequestOptions objects... It was discussed
> earlier a
> > bit, but I'm not sure my particular concerns were addressed. I see the
> > following issues with it:
> > * They double the number of methods we have
> > * They are pretty similar, so it isn't clear why we need all those
> > different objects
> > * All our other APIs specify timeouts either in method calls directly or
> in
> > the configuration for the entire object.
> >
> > We also typically don't use methods that start with "set", but this is
> > minor.
> >
> > The configs for NewTopic are Map<String, String> - shouldn't we use the
> > LogConfig object that we already have? This will take care of
> documentation
> > and be similar to ProducerConfig and ConsumerConfig?
> >
> > My concerns aside, thank you for working on this much needed API.
> >
> > Gwen
> >
> > On Mon, Mar 13, 2017 at 3:26 PM, Colin McCabe <cmcc...@apache.org>
> wrote:
> >
> > > Hi Radai,
> > >
> > > Thanks for looking at the KIP again.
> > >
> > > On Mon, Mar 13, 2017, at 12:33, radai wrote:
> > > > looking at the KIP as it is now, looks like all *Options objects
> have a
> > > > common timeout property. could it be extracted to a common
> > > > AdminRequestOptions or something?
> > >
> > > Perhaps I'm missing something, but I don't think there is any reason to
> > > extract the timeout property.  It doesn't simplify the implementation
> > > (I've already implemented the interface in a branch, so I know this for
> > > sure.)  It doesn't simplify the API exposed to the users, since they
> > > will still want to provide the specific option type corresponding to
> the
> > > call.  Also, as we discussed previously in the thread (about NewTopic),
> > > having lot of inheritance and base classes makes it difficult to change
> > > classes over time.  It is better to simply use composition.
> > >
> > > I think it would be much better to get the AdminClient interface in,
> and
> > > start iterating on it incrementally as we discover ways it could be
> > > better.  This is similar to how some things in Streams were added as
> > > unstable interfaces and then stabilized over time.
> > >
> > > best,
> > > Colin
> > >
> > >
> > > >
> > > > On Thu, Mar 9, 2017 at 2:14 PM, Colin McCabe <cmcc...@apache.org>
> > wrote:
> > > >
> > > > > Hi all,
> > > > >
> > > > > We've been discussing this for a while (about a month) and I think
> > > > > people have made some great points that improved the proposal.  In
> > > > > particular, adding async and batching was important.  I've also
> been
> > > > > talking with some end-users who would like to make use of this API.
> > > > > Once this is in, we can iterate on it before the release, and it
> will
> > > > > also unblock a lot of other admin proposals.  I think it would be
> > good
> > > > > to start the vote in a little bit, assuming there are no
> objections.
> > > > >
> > > > > best,
> > > > > Colin
> > > > >
> > > > >
> > > > > On Wed, Mar 8, 2017, at 17:09, Colin McCabe wrote:
> > > > > > On Mon, Mar 6, 2017, at 11:55, Ismael Juma wrote:
> > > > > > > Thanks Colin.
> > > > > > >
> > > > > > > I am familiar with the protocol semantics, but we need to
> > document
> > > the
> > > > > > > API
> > > > > > > for users who don't know the protocol. I still think it would
> be
> > > > > valuable
> > > > > > > to have some examples of how the API would be used for common
> use
> > > > > cases.
> > > > > >
> > > > > > Getting the version of all nodes in the cluster:
> > > > > >   Map<Node, NodeApiVersions> nodesToVersions =
> > > > > >     adminClient.listNodes().nodes().thenApply(
> > > > > >       nodes -> adminClient.apiVersions(nodes)).all().get();
> > > > > >
> > > > > > Creating a topic:
> > > > > >   adminClient.createTopic(new NewTopic("myNewTopic", 3, (short)
> > > > > >   3)).all().get();
> > > > > >
> > > > > > Validating that a topic can be created (but not creating it):
> > > > > >   adminClient.createTopic(new NewTopic("myNewTopic", 3, (short)
> 3),
> > > > > >     new CreateTopicOptions().setValidateOnly(true)).all().get();
> > > > > >
> > > > > > > For example, say someone creates a topic and then produces to
> it.
> > > What
> > > > > > > would be the recommended way to do that?
> > > > > >
> > > > > > Once the future returned by createTopics has successfully
> > completed,
> > > it
> > > > > > should be possible to produce to the topic.
> > > > > >
> > > > > > There are a few warts that are definitely worth calling out.
> These
> > > are
> > > > > > things that need to be fixed at the protocol layer, so they're
> > > outside
> > > > > > the scope of this KIP.  But you made a good point that we need to
> > > > > > document this well.  Here's my list (I wonder if anyone has
> more?):
> > > > > >
> > > > > > * If auto.create.topics.enable is true on the brokers,
> > > > > > AdminClient#describeTopic(topicName) may create a topic named
> > > topicName.
> > > > > >  There are two workarounds: either use AdminClient#listTopics and
> > > ensure
> > > > > > that the topic is present before describing, or disable
> > > > > > auto.create.topics.enable.
> > > > > >
> > > > > > * If delete.topic.enable is false on the brokers,
> > > > > > AdminClient#deleteTopic(topicName) will mark topicName for
> > > deletion, but
> > > > > > not actually delete it.  deleteTopic will return success in this
> > > case.
> > > > > >
> > > > > > * It may take several seconds after AdminClient#deleteTopic
> returns
> > > > > > success for all the brokers to become aware that the topic is
> gone.
> > > > > > During this time, AdminClient#listTopics and
> > > AdminClient#describeTopic
> > > > > > may continue to return information about the deleted topic.
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > > >
> > > > > > >
> > > > > > > Ismael
> > > > > > >
> > > > > > > On Mon, Mar 6, 2017 at 7:43 PM, Colin McCabe <
> cmcc...@apache.org
> > >
> > > > > wrote:
> > > > > > >
> > > > > > > > On Mon, Mar 6, 2017, at 05:50, Ismael Juma wrote:
> > > > > > > > > Thanks Colin. It seems like you replied to me accidentally
> > > instead
> > > > > of the
> > > > > > > > > list, so leaving your reply below for the benefit of
> others.
> > > > > > > >
> > > > > > > > Thanks, Ismael.  I actually realized my mistake right after I
> > > sent to
> > > > > > > > you, and re-posted it to the mailing list instead of sending
> > > > > directly.
> > > > > > > > Sigh...
> > > > > > > >
> > > > > > > > >
> > > > > > > > > Regarding the disadvantage of having to hunt through the
> > > request
> > > > > class,
> > > > > > > > > don't people have to do that anyway with the Options
> classes?
> > > > > > > >
> > > > > > > > A lot of people will simply choose the default options, until
> > > they
> > > > > have
> > > > > > > > a reason to do otherwise (for example, they want a longer or
> > > shorter
> > > > > > > > timeout, etc.)
> > > > > > > >
> > > > > > > > >
> > > > > > > > > Aside from that, it would be great if the KIP included more
> > > > > detailed
> > > > > > > > > javadoc for each method including information about
> potential
> > > > > exceptions.
> > > > > > > >
> > > > > > > > That's a good question.  Because this is an asynchronous API,
> > > methods
> > > > > > > > never throw exceptions.  Instead, if you call get() /
> > > whenComplete()
> > > > > /
> > > > > > > > isCompletedExceptionally() / etc. on one of the
> > CompletableFuture
> > > > > > > > objects, you will get the exception.  This is to allow
> > > Node.js-style
> > > > > > > > completion chaining.  I will add this explanation to the KIP.
> > > > > > > >
> > > > > > > > > I'm particularly interested in what a user can expect if a
> > > create
> > > > > topics
> > > > > > > > > succeeds versus what they can expect if a timeout exception
> > is
> > > > > thrown. It
> > > > > > > > > would be good to consider partial failures as well.
> > > > > > > >
> > > > > > > > This is spelled out by KIP-4.
> > > > > > > >
> > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+
> > > > > > > > Command+line+and+centralized+administrative+operations
> > > > > > > >
> > > > > > > > Specifically,
> > > > > > > >
> > > > > > > > > If a timeout error occurs [in CreateTopic], the topic could
> > > still
> > > > > be
> > > > > > > > > created successfully at a later time. Its up to the client
> to
> > > query
> > > > > > > > > for the state at that point.
> > > > > > > >
> > > > > > > > Since we're specifically not changing the server as part of
> > this
> > > KIP,
> > > > > > > > those semantics will still be in force.  Of course, there are
> > > plenty
> > > > > of
> > > > > > > > other exceptions that you can get from CreateTopics that are
> > more
> > > > > > > > meaningful, such as permission-related or network-related
> ones.
> > > But
> > > > > if
> > > > > > > > you get a timeout, the operation may or may not have
> succeeded.
> > > > > > > >
> > > > > > > > Could we fix the timeout problem?  Sort of.  We could
> implement
> > > > > > > > something like a retry cache.  The brokers would have to
> > > maintain a
> > > > > > > > cache of operations (and their results) which had succeeded
> or
> > > > > failed.
> > > > > > > > Then, if an RPC got interrupted after the server had
> performed
> > > it,
> > > > > but
> > > > > > > > before the client had received the response message, the
> client
> > > could
> > > > > > > > simply reconnect on another TCP session and ask the broker
> for
> > > the
> > > > > > > > result of the previous operation.  The broker could look up
> the
> > > > > result
> > > > > > > > in the cache and re-send it.
> > > > > > > >
> > > > > > > > This fix works, but it is very complex.  The cache requires
> > > space in
> > > > > > > > memory (and to do it perfectly, you also want to persist the
> > > cache to
> > > > > > > > disk in case the broker restarts and the client re-appears).
> > > The fix
> > > > > > > > also requires the client to wait for an indefinite amount of
> > > time for
> > > > > > > > the server to come back.  If the client ever "gives up" and
> > just
> > > > > throws
> > > > > > > > a timeout exception, we are back to not knowing what happened
> > on
> > > the
> > > > > > > > server.
> > > > > > > >
> > > > > > > > In any case, I think we should discuss RPC change in a
> separate
> > > > > KIP...
> > > > > > > > the scope is already big enough here.  Also, in practice,
> users
> > > have
> > > > > > > > workarounds for cases where there are timeouts or failures to
> > > > > > > > communicate.
> > > > > > > >
> > > > > > > > best,
> > > > > > > > Colin
> > > > > > > >
> > > > > > > > >
> > > > > > > > > Ismael
> > > > > > > > >
> > > > > > > > > On Fri, Mar 3, 2017 at 9:37 PM, Colin McCabe <
> > > cmcc...@apache.org>
> > > > > wrote:
> > > > > > > > >
> > > > > > > > > > On Fri, Mar 3, 2017, at 06:41, Ismael Juma wrote:
> > > > > > > > > > > Hi Colin,
> > > > > > > > > > >
> > > > > > > > > > > I still need to do a detailed review, but I have a
> couple
> > > of
> > > > > > > > > > > comments/questions:
> > > > > > > > > > >
> > > > > > > > > > > 1. I am not sure about having the options/response
> > classes
> > > as
> > > > > inner
> > > > > > > > > > > classes
> > > > > > > > > > > of the interface. It means that file containing the
> > > interface
> > > > > will be
> > > > > > > > > > > huge
> > > > > > > > > > > eventually. And the classes are not necessarily related
> > > > > either. Why
> > > > > > > > not
> > > > > > > > > > > use
> > > > > > > > > > > a separate package for them?
> > > > > > > > > >
> > > > > > > > > > Yeah, I think it's reasonable to make these top-level
> > classes
> > > > > and put
> > > > > > > > > > them in separate files.  We can put them all in
> > > > > > > > > > org.apache.kafka.clients.admin.
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > 2. Can you elaborate on how one decides one goes in the
> > > > > Options class
> > > > > > > > > > > versus the first parameter?
> > > > > > > > > >
> > > > > > > > > > I guess I think of options as things that you don't have
> to
> > > > > set.  For
> > > > > > > > > > example, when deleting a topic, you must supply the topic
> > > name,
> > > > > but
> > > > > > > > > > supplying a non-default timeout is optional.
> > > > > > > > > >
> > > > > > > > > > > I wonder if it would be simpler to just have a
> > > > > > > > > > > single parameter. In that case it should probably be
> > > called a
> > > > > > > > Request as
> > > > > > > > > > > Radai suggested, but that's a separate point and we can
> > > > > discuss it
> > > > > > > > > > > separately.
> > > > > > > > > >
> > > > > > > > > > Hmm.  I don't think it would be simpler for users.  It
> > would
> > > > > force
> > > > > > > > > > people who just want to do something simple like delete a
> > > topic
> > > > > or get
> > > > > > > > > > the api version of a single node to go hunting through
> the
> > > > > request
> > > > > > > > > > class.
> > > > > > > > > >
> > > > > > > > > > best,
> > > > > > > > > > Colin
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > > > Ismael
> > > > > > > > > > >
> > > > > > > > > > > On Thu, Mar 2, 2017 at 1:58 AM, Colin McCabe <
> > > > > cmcc...@apache.org>
> > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > On Wed, Mar 1, 2017, at 15:52, radai wrote:
> > > > > > > > > > > > > quick comment on the request objects:
> > > > > > > > > > > > >
> > > > > > > > > > > > > i see "abstract class NewTopic" and "class
> > > > > > > > NewTopicWithReplication"
> > > > > > > > > > and "
> > > > > > > > > > > > > NewTopicWithReplicaAssignments"
> > > > > > > > > > > > >
> > > > > > > > > > > > > 1. since the result object is called
> > CreateTopicResults
> > > > > should
> > > > > > > > these
> > > > > > > > > > be
> > > > > > > > > > > > > called *Request?
> > > > > > > > > > > >
> > > > > > > > > > > > Hi radai,
> > > > > > > > > > > >
> > > > > > > > > > > > Thanks for taking a look.
> > > > > > > > > > > >
> > > > > > > > > > > > I think using the name "request" would be very
> > confusing
> > > > > here,
> > > > > > > > because
> > > > > > > > > > > > we have a whole family of internal Request classes
> such
> > > as
> > > > > > > > > > > > CreateTopicsRequest, TopicMetataRequest, etc. which
> are
> > > used
> > > > > for
> > > > > > > > RPCs.
> > > > > > > > > > > >
> > > > > > > > > > > > > 2. this seems like a suboptimal approach to me.
> > > imagine we
> > > > > add a
> > > > > > > > > > > > > NewTopicWithSecurity, and then we would need
> > > > > > > > > > > > > NewTopicWithReplicationAndSecurity? (or any
> > composable
> > > > > > > > "traits").
> > > > > > > > > > > > > this wont really scale. Wouldnt it be better to
> have
> > a
> > > > > single
> > > > > > > > (rather
> > > > > > > > > > > > complicated)
> > > > > > > > > > > > > CreateTopicRequest, and use a builder pattern to
> deal
> > > with
> > > > > the
> > > > > > > > > > compexity
> > > > > > > > > > > > > and options? like so:
> > > > > > > > > > > > >
> > > > > > > > > > > > > CreateTopicRequest req =
> > > > > > > > > > > > > AdminRequests.newTopic("bob").
> replicationFactor(2).
> > > > > > > > > > > > withPartitionAssignment(1,
> > > > > > > > > > > > > "boker7", "broker10").withOption(...).build();
> > > > > > > > > > > > >
> > > > > > > > > > > > > the builder would validate any potentially
> > conflicting
> > > > > options
> > > > > > > > and
> > > > > > > > > > would
> > > > > > > > > > > > > allow piling on the complexity in a more manageable
> > way
> > > > > (note -
> > > > > > > > my
> > > > > > > > > > code
> > > > > > > > > > > > > above intends to demonstrate both a general
> > replication
> > > > > factor
> > > > > > > > and a
> > > > > > > > > > > > > specific assignment for a partiocular partition of
> > that
> > > > > topic,
> > > > > > > > which
> > > > > > > > > > may
> > > > > > > > > > > > > be
> > > > > > > > > > > > > too much freedom).
> > > > > > > > > > > >
> > > > > > > > > > > > We don't need to express every optional bell and
> > whistle
> > > by
> > > > > > > > creating a
> > > > > > > > > > > > subclass.  In fact, the proposal already had
> > > setConfigs() in
> > > > > the
> > > > > > > > base
> > > > > > > > > > > > class, since it applies to every new topic creation.
> > > > > > > > > > > >
> > > > > > > > > > > > Thinking about it a little more, though, the
> subclasses
> > > don't
> > > > > > > > really
> > > > > > > > > > add
> > > > > > > > > > > > that much value, so we should probably just have
> > NewTopic
> > > > > and no
> > > > > > > > > > > > subclasses.  I removed the subclasses.
> > > > > > > > > > > >
> > > > > > > > > > > > best,
> > > > > > > > > > > > Colin
> > > > > > > > > > > >
> > > > > > > > > > > > >
> > > > > > > > > > > > > On Wed, Mar 1, 2017 at 1:58 PM, Colin McCabe <
> > > > > cmcc...@apache.org
> > > > > > > > >
> > > > > > > > > > wrote:
> > > > > > > > > > > > >
> > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > Thanks for commenting, everyone.  Does anyone
> have
> > > more
> > > > > > > > questions
> > > > > > > > > > or
> > > > > > > > > > > > > > comments, or should we vote?  The latest proposal
> > is
> > > up
> > > > > at
> > > > > > > > > > > > > > https://cwiki.apache.org/
> > > confluence/display/KAFKA/KIP-
> > > > > > > > > > > > 117%3A+Add+a+public+
> > > > > > > > > > > > > > AdminClient+API+for+Kafka+admin+operations
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > > > > On Thu, Feb 16, 2017, at 15:00, Colin McCabe
> wrote:
> > > > > > > > > > > > > > > On Thu, Feb 16, 2017, at 14:11, Dong Lin wrote:
> > > > > > > > > > > > > > > > Hey Colin,
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Thanks for the update. I have two comments:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - I actually think it is simpler and good
> > enough
> > > to
> > > > > have
> > > > > > > > > > per-topic
> > > > > > > > > > > > API
> > > > > > > > > > > > > > > > instead of batch-of-topic API. This is
> > different
> > > > > from the
> > > > > > > > > > argument
> > > > > > > > > > > > for
> > > > > > > > > > > > > > > > batch-of-partition API because, unlike
> > operation
> > > on
> > > > > topic,
> > > > > > > > > > people
> > > > > > > > > > > > > > usually
> > > > > > > > > > > > > > > > operate on multiple partitions (e.g. seek(),
> > > > > purge()) at a
> > > > > > > > > > time. Is
> > > > > > > > > > > > > > there
> > > > > > > > > > > > > > > > performance concern with per-topic API? I am
> > > > > wondering if
> > > > > > > > we
> > > > > > > > > > > > should do
> > > > > > > > > > > > > > > > per-topic API until there is use-case or
> > > performance
> > > > > > > > benefits
> > > > > > > > > > of
> > > > > > > > > > > > > > > > batch-of-topic API.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > Yes, there is a performance concern with only
> > > > > supporting
> > > > > > > > > > operations
> > > > > > > > > > > > on
> > > > > > > > > > > > > > > one topic at a time.  Jay expressed this in
> some
> > > of his
> > > > > > > > earlier
> > > > > > > > > > > > emails
> > > > > > > > > > > > > > > and some other people did as well.  We have
> cases
> > > in
> > > > > mind for
> > > > > > > > > > > > management
> > > > > > > > > > > > > > > software where many topics are created at once.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > - Currently we have interface "Consumer" and
> > > > > "Producer".
> > > > > > > > And we
> > > > > > > > > > > > also
> > > > > > > > > > > > > > have
> > > > > > > > > > > > > > > > implementations of these two interfaces as
> > > > > "KafkaConsumer"
> > > > > > > > and
> > > > > > > > > > > > > > > > "KafkaProducer". If we follow the same naming
> > > > > pattern,
> > > > > > > > should
> > > > > > > > > > we
> > > > > > > > > > > > have
> > > > > > > > > > > > > > > > interface "AdminClient" and the
> implementation
> > > > > > > > > > "KafkaAdminClient",
> > > > > > > > > > > > > > > > instead
> > > > > > > > > > > > > > > > of the other way around?
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > That's a good point.  We should do that for
> > > > > consistency.
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > Dong
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > On Thu, Feb 16, 2017 at 10:19 AM, Colin
> McCabe
> > <
> > > > > > > > > > cmcc...@apache.org
> > > > > > > > > > > > >
> > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > So I think people have made some very good
> > > points
> > > > > so far.
> > > > > > > > > > There
> > > > > > > > > > > > > > seems
> > > > > > > > > > > > > > > > > to be agreement that we need to have
> explicit
> > > > > batch APIs
> > > > > > > > for
> > > > > > > > > > the
> > > > > > > > > > > > > > sake of
> > > > > > > > > > > > > > > > > efficiency, so I added that back.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Contexts seem a little more complex than we
> > > > > thought, so I
> > > > > > > > > > removed
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > from the proposal.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I removed the Impl class.  Instead, we now
> > > have a
> > > > > > > > > > > > KafkaAdminClient
> > > > > > > > > > > > > > > > > interface and an AdminClient
> > implementation.  I
> > > > > think
> > > > > > > > this
> > > > > > > > > > > > matches
> > > > > > > > > > > > > > our
> > > > > > > > > > > > > > > > > other code better, as Jay commented.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Each call now has an "Options" object that
> is
> > > > > passed in.
> > > > > > > > > > This
> > > > > > > > > > > > will
> > > > > > > > > > > > > > > > > allow us to easily add new parameters to
> the
> > > calls
> > > > > > > > without
> > > > > > > > > > having
> > > > > > > > > > > > > > tons
> > > > > > > > > > > > > > > > > of function overloads.  Similarly, each
> call
> > > now
> > > > > has a
> > > > > > > > > > Results
> > > > > > > > > > > > > > object,
> > > > > > > > > > > > > > > > > which will let us easily extend the results
> > we
> > > are
> > > > > > > > returning
> > > > > > > > > > if
> > > > > > > > > > > > > > needed.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > Many people made the point that Java 7
> > Futures
> > > are
> > > > > not
> > > > > > > > that
> > > > > > > > > > > > useful,
> > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > Java 8 CompletableFutures are.  With
> > > > > CompletableFutures,
> > > > > > > > you
> > > > > > > > > > can
> > > > > > > > > > > > > > chain
> > > > > > > > > > > > > > > > > calls, adapt them, join them-- basically
> all
> > > the
> > > > > stuff
> > > > > > > > > > people are
> > > > > > > > > > > > > > doing
> > > > > > > > > > > > > > > > > in Node.js and Twisted Python.  Java 7
> > Futures
> > > > > don't
> > > > > > > > really
> > > > > > > > > > let
> > > > > > > > > > > > you
> > > > > > > > > > > > > > do
> > > > > > > > > > > > > > > > > anything but poll for a value or block.
> So I
> > > felt
> > > > > that
> > > > > > > > it
> > > > > > > > > > was
> > > > > > > > > > > > > > better to
> > > > > > > > > > > > > > > > > just go with a CompletableFuture-based API.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > People also made the point that they would
> > > like an
> > > > > easy
> > > > > > > > API
> > > > > > > > > > for
> > > > > > > > > > > > > > waiting
> > > > > > > > > > > > > > > > > on complete success of a batch call.  For
> > > example,
> > > > > an API
> > > > > > > > > > that
> > > > > > > > > > > > would
> > > > > > > > > > > > > > > > > fail if even one topic wasn't created in
> > > > > createTopics.
> > > > > > > > So I
> > > > > > > > > > > > came up
> > > > > > > > > > > > > > > > > with Result objects that provide multiple
> > > futures
> > > > > that
> > > > > > > > you
> > > > > > > > > > can
> > > > > > > > > > > > wait
> > > > > > > > > > > > > > on.
> > > > > > > > > > > > > > > > > You can wait on a future that fires when
> > > > > everything is
> > > > > > > > > > complete,
> > > > > > > > > > > > or
> > > > > > > > > > > > > > you
> > > > > > > > > > > > > > > > > can wait on futures for individual topics
> > being
> > > > > created.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > I updated the wiki, so please take a look.
> > > Note
> > > > > that
> > > > > > > > this
> > > > > > > > > > new
> > > > > > > > > > > > API
> > > > > > > > > > > > > > > > > requires JDK8.  It seems like JDK8 is
> coming
> > > soon,
> > > > > > > > though,
> > > > > > > > > > and
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > disadvantages of sticking to Java 7 are
> > pretty
> > > big
> > > > > here,
> > > > > > > > I
> > > > > > > > > > think.
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > On Mon, Feb 13, 2017, at 11:51, Colin
> McCabe
> > > wrote:
> > > > > > > > > > > > > > > > > > On Sun, Feb 12, 2017, at 09:21, Jay Kreps
> > > wrote:
> > > > > > > > > > > > > > > > > > > Hey Colin,
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Thanks for the hard work on this. I
> know
> > > going
> > > > > back
> > > > > > > > and
> > > > > > > > > > > > forth on
> > > > > > > > > > > > > > APIs
> > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > kind of frustrating but we're at the
> > point
> > > > > where
> > > > > > > > these
> > > > > > > > > > things
> > > > > > > > > > > > > > live long
> > > > > > > > > > > > > > > > > > > enough and are used by enough people
> that
> > > it is
> > > > > > > > worth the
> > > > > > > > > > > > pain.
> > > > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > > > sure
> > > > > > > > > > > > > > > > > > > it'll come down in the right place
> > > eventually.
> > > > > A
> > > > > > > > couple
> > > > > > > > > > > > things
> > > > > > > > > > > > > > I've
> > > > > > > > > > > > > > > > > found
> > > > > > > > > > > > > > > > > > > helped in the past:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >    1. The burden of evidence needs to
> > fall
> > > on
> > > > > the
> > > > > > > > > > > > complicator.
> > > > > > > > > > > > > > i.e. if
> > > > > > > > > > > > > > > > > > >    person X thinks the api should be
> > async
> > > > > they need
> > > > > > > > to
> > > > > > > > > > > > produce
> > > > > > > > > > > > > > a set
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > >    common use cases that require this.
> > > > > Otherwise you
> > > > > > > > are
> > > > > > > > > > > > > > perpetually
> > > > > > > > > > > > > > > > > > >    having to
> > > > > > > > > > > > > > > > > > >    think "we might need x". I think it
> is
> > > good
> > > > > to
> > > > > > > > have a
> > > > > > > > > > > > rule of
> > > > > > > > > > > > > > > > > "simple
> > > > > > > > > > > > > > > > > > >    until
> > > > > > > > > > > > > > > > > > >    proven insufficient".
> > > > > > > > > > > > > > > > > > >    2. Make sure we frame things for the
> > > > > intended
> > > > > > > > > > audience. At
> > > > > > > > > > > > > > this
> > > > > > > > > > > > > > > > > point
> > > > > > > > > > > > > > > > > > >    our apis get used by a very broad
> set
> > of
> > > > > Java
> > > > > > > > > > engineers.
> > > > > > > > > > > > This
> > > > > > > > > > > > > > is a
> > > > > > > > > > > > > > > > > > >    very
> > > > > > > > > > > > > > > > > > >    different audience from our
> developer
> > > > > mailing
> > > > > > > > list.
> > > > > > > > > > These
> > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > code
> > > > > > > > > > > > > > > > > > >    for a
> > > > > > > > > > > > > > > > > > >    living not necessarily as a passion,
> > and
> > > > > may not
> > > > > > > > > > > > understand
> > > > > > > > > > > > > > details
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > >    the
> > > > > > > > > > > > > > > > > > >    internals of our system or even
> basic
> > > > > things like
> > > > > > > > > > > > > > multi-threaded
> > > > > > > > > > > > > > > > > > >    programming. I don't think this
> means
> > we
> > > > > want to
> > > > > > > > dumb
> > > > > > > > > > > > things
> > > > > > > > > > > > > > down,
> > > > > > > > > > > > > > > > > but
> > > > > > > > > > > > > > > > > > >    rather try really hard to make
> things
> > > truly
> > > > > simple
> > > > > > > > > > when
> > > > > > > > > > > > > > possible.
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > Okay here were a couple of comments:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >    1. Conceptually what is a
> > TopicContext?
> > > I
> > > > > think it
> > > > > > > > > > means
> > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > >    like
> > > > > > > > > > > > > > > > > > >    TopicAdmin? It is not literally
> > context
> > > > > about
> > > > > > > > Topics
> > > > > > > > > > > > right?
> > > > > > > > > > > > > > What is
> > > > > > > > > > > > > > > > > > >    the
> > > > > > > > > > > > > > > > > > >    relationship of Contexts to clients?
> > Is
> > > > > there a
> > > > > > > > > > > > threadsafety
> > > > > > > > > > > > > > > > > > >    difference?
> > > > > > > > > > > > > > > > > > >    Would be nice to not have to think
> > about
> > > > > this,
> > > > > > > > this is
> > > > > > > > > > > > what I
> > > > > > > > > > > > > > mean
> > > > > > > > > > > > > > > > > by
> > > > > > > > > > > > > > > > > > >    "conceptual weight". We introduce a
> > new
> > > > > concept
> > > > > > > > that
> > > > > > > > > > is a
> > > > > > > > > > > > bit
> > > > > > > > > > > > > > > > > nebulous
> > > > > > > > > > > > > > > > > > >    that
> > > > > > > > > > > > > > > > > > >    I have to figure out to use what
> could
> > > be a
> > > > > simple
> > > > > > > > > > api.
> > > > > > > > > > > > I'm
> > > > > > > > > > > > > > sure
> > > > > > > > > > > > > > > > > > >    you've
> > > > > > > > > > > > > > > > > > >    been through this experience before
> > > where
> > > > > you have
> > > > > > > > > > these
> > > > > > > > > > > > > > various
> > > > > > > > > > > > > > > > > > >    objects
> > > > > > > > > > > > > > > > > > >    and you're trying to figure out what
> > > they
> > > > > > > > represent
> > > > > > > > > > (the
> > > > > > > > > > > > > > connection
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >    the
> > > > > > > > > > > > > > > > > > >    server? the information to create a
> > > > > connection? a
> > > > > > > > > > request
> > > > > > > > > > > > > > session?).
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > The intention was to provide some
> grouping
> > of
> > > > > methods,
> > > > > > > > and
> > > > > > > > > > > > also a
> > > > > > > > > > > > > > place
> > > > > > > > > > > > > > > > > > to put request parameters which were
> often
> > > set to
> > > > > > > > defaults
> > > > > > > > > > > > rather
> > > > > > > > > > > > > > than
> > > > > > > > > > > > > > > > > > being explicitly set.  If it seems
> complex,
> > > we
> > > > > can
> > > > > > > > > > certainly
> > > > > > > > > > > > get
> > > > > > > > > > > > > > rid of
> > > > > > > > > > > > > > > > > > it.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >    2. We've tried to avoid the Impl
> > naming
> > > > > > > > convention. In
> > > > > > > > > > > > > > general the
> > > > > > > > > > > > > > > > > > >    rule
> > > > > > > > > > > > > > > > > > >    has been if there is only going to
> be
> > > one
> > > > > > > > > > implementation
> > > > > > > > > > > > you
> > > > > > > > > > > > > > don't
> > > > > > > > > > > > > > > > > > >    need an
> > > > > > > > > > > > > > > > > > >    interface. If there will be
> multiple,
> > > > > distinguish
> > > > > > > > it
> > > > > > > > > > from
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > others.
> > > > > > > > > > > > > > > > > > >    The
> > > > > > > > > > > > > > > > > > >    other clients follow this pattern:
> > > Producer,
> > > > > > > > > > > > KafkaProducer,
> > > > > > > > > > > > > > > > > > >    MockProducer;
> > > > > > > > > > > > > > > > > > >    Consumer, KafkaConsumer,
> MockConsumer.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Good point.  Let's change the interface
> to
> > > > > > > > > > KafkaAdminClient,
> > > > > > > > > > > > and
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > implementation to AdminClient.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >    3. We generally don't use setters or
> > > > > getters as a
> > > > > > > > > > naming
> > > > > > > > > > > > > > > > > convention. I
> > > > > > > > > > > > > > > > > > >    personally think mutating the
> setting
> > in
> > > > > place
> > > > > > > > seems
> > > > > > > > > > kind
> > > > > > > > > > > > of
> > > > > > > > > > > > > > like
> > > > > > > > > > > > > > > > > late
> > > > > > > > > > > > > > > > > > >    90s
> > > > > > > > > > > > > > > > > > >    Java style. I think it likely has
> > > > > thread-safety
> > > > > > > > > > issues.
> > > > > > > > > > > > i.e.
> > > > > > > > > > > > > > even if
> > > > > > > > > > > > > > > > > > >    it is
> > > > > > > > > > > > > > > > > > >    volatile you may not get the value
> you
> > > just
> > > > > set if
> > > > > > > > > > there
> > > > > > > > > > > > is
> > > > > > > > > > > > > > another
> > > > > > > > > > > > > > > > > > >    thread... I actually really liked
> what
> > > you
> > > > > > > > described
> > > > > > > > > > as
> > > > > > > > > > > > your
> > > > > > > > > > > > > > > > > original
> > > > > > > > > > > > > > > > > > >    idea
> > > > > > > > > > > > > > > > > > >    of having a single parameter object
> > like
> > > > > > > > > > > > CreateTopicRequest
> > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > holds
> > > > > > > > > > > > > > > > > > >    all
> > > > > > > > > > > > > > > > > > >    these parameters and defaults. This
> > > lets you
> > > > > > > > evolve
> > > > > > > > > > the
> > > > > > > > > > > > api
> > > > > > > > > > > > > > with all
> > > > > > > > > > > > > > > > > > >    the
> > > > > > > > > > > > > > > > > > >    various combinations of arguments
> > > without
> > > > > > > > overloading
> > > > > > > > > > > > > > insanity.
> > > > > > > > > > > > > > > > > After
> > > > > > > > > > > > > > > > > > >    doing
> > > > > > > > > > > > > > > > > > >    literally tens of thousands of
> remote
> > > APIs
> > > > > at
> > > > > > > > > > LinkedIn we
> > > > > > > > > > > > > > eventually
> > > > > > > > > > > > > > > > > > >    converged on a rule, which is
> > ultimately
> > > > > every
> > > > > > > > remote
> > > > > > > > > > api
> > > > > > > > > > > > > > needs a
> > > > > > > > > > > > > > > > > > >    single
> > > > > > > > > > > > > > > > > > >    argument object you can add to over
> > time
> > > > > and it
> > > > > > > > must
> > > > > > > > > > be
> > > > > > > > > > > > > > batched.
> > > > > > > > > > > > > > > > > Which
> > > > > > > > > > > > > > > > > > >    brings me to my next point...
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > Just to clarify, volatiles were never a
> > part
> > > of
> > > > > the
> > > > > > > > > > proposal.
> > > > > > > > > > > > I
> > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > that context objects or request objects
> > > should
> > > > > be used
> > > > > > > > by a
> > > > > > > > > > > > single
> > > > > > > > > > > > > > > > > > thread at a time.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > I'm not opposed to request objects, but I
> > > think
> > > > > they
> > > > > > > > raise
> > > > > > > > > > all
> > > > > > > > > > > > the
> > > > > > > > > > > > > > same
> > > > > > > > > > > > > > > > > > questions as context objects.  Basically,
> > the
> > > > > > > > thread-safety
> > > > > > > > > > > > issues
> > > > > > > > > > > > > > need
> > > > > > > > > > > > > > > > > > to be spelled out and understood by the
> > user,
> > > > > and the
> > > > > > > > user
> > > > > > > > > > > > needs
> > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > lines of code to make a request.  And
> there
> > > will
> > > > > be
> > > > > > > > people
> > > > > > > > > > > > trying
> > > > > > > > > > > > > > to do
> > > > > > > > > > > > > > > > > > things like re-use request objects when
> > they
> > > > > should
> > > > > > > > not,
> > > > > > > > > > and so
> > > > > > > > > > > > > > forth.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >    4. I agree batch apis are annoying
> > but I
> > > > > suspect
> > > > > > > > we'll
> > > > > > > > > > > > end up
> > > > > > > > > > > > > > adding
> > > > > > > > > > > > > > > > > > >    one. Doing 1000 requests for 1000
> > > > > operations if
> > > > > > > > > > creating
> > > > > > > > > > > > or
> > > > > > > > > > > > > > deleting
> > > > > > > > > > > > > > > > > > >    will
> > > > > > > > > > > > > > > > > > >    be bad, right? This won't be the
> > common
> > > > > case, but
> > > > > > > > when
> > > > > > > > > > > > you do
> > > > > > > > > > > > > > it it
> > > > > > > > > > > > > > > > > > >    will be
> > > > > > > > > > > > > > > > > > >    a deal-breaker problem. I don't
> think
> > we
> > > > > should
> > > > > > > > try
> > > > > > > > > > to fix
> > > > > > > > > > > > > > this one
> > > > > > > > > > > > > > > > > > >    behind
> > > > > > > > > > > > > > > > > > >    the scenes.
> > > > > > > > > > > > > > > > > > >    5. Are we going to do
> > CompletableFuture
> > > > > (which
> > > > > > > > > > requires
> > > > > > > > > > > > java
> > > > > > > > > > > > > > 8) or
> > > > > > > > > > > > > > > > > > >    normal Future? Normal Future is
> > utterly
> > > > > useless
> > > > > > > > for
> > > > > > > > > > most
> > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > >    than
> > > > > > > > > > > > > > > > > > >    just calling wait. If we can evolve
> in
> > > > > place from
> > > > > > > > > > Future
> > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > >    CompletableFuture that is fantastic
> > (we
> > > > > could do
> > > > > > > > it
> > > > > > > > > > for
> > > > > > > > > > > > the
> > > > > > > > > > > > > > producer
> > > > > > > > > > > > > > > > > > >    too!).
> > > > > > > > > > > > > > > > > > >    My belief was that this was binary
> > > > > incompatible
> > > > > > > > but I
> > > > > > > > > > > > > > actually don't
> > > > > > > > > > > > > > > > > > >    know
> > > > > > > > > > > > > > > > > > >    (obviously it's source compatible).
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > In my testing, replacing a return type
> > with a
> > > > > subclass
> > > > > > > > of
> > > > > > > > > > that
> > > > > > > > > > > > > > return
> > > > > > > > > > > > > > > > > > type did not break binary
> compatibility.  I
> > > > > haven't
> > > > > > > > been
> > > > > > > > > > able
> > > > > > > > > > > > to
> > > > > > > > > > > > > > find
> > > > > > > > > > > > > > > > > > chapter and verse on this from the Java
> > > > > implementers,
> > > > > > > > > > though.
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > On Wed, Feb 8, 2017 at 5:00 PM, Colin
> > > McCabe <
> > > > > > > > > > > > cmcc...@apache.org
> > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Hi all,
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I made some major revisions to the
> > > proposal
> > > > > on the
> > > > > > > > > > wiki, so
> > > > > > > > > > > > > > please
> > > > > > > > > > > > > > > > > check
> > > > > > > > > > > > > > > > > > > > it out.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > The new API is based on Ismael's
> > > suggestion
> > > > > of
> > > > > > > > grouping
> > > > > > > > > > > > > > related APIs.
> > > > > > > > > > > > > > > > > > > > There is only one layer of
> grouping.  I
> > > > > think that
> > > > > > > > it's
> > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > > pretty
> > > > > > > > > > > > > > > > > > > > intuitive.  It's also based on the
> idea
> > > of
> > > > > using
> > > > > > > > > > Futures,
> > > > > > > > > > > > which
> > > > > > > > > > > > > > > > > several
> > > > > > > > > > > > > > > > > > > > people commented that they'd like to
> > see.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Here's a simple example:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >  > AdminClient client = new
> > > > > > > > AdminClientImpl(myConfig);
> > > > > > > > > > > > > > > > > > > >  > try {
> > > > > > > > > > > > > > > > > > > >  >   client.topics().create("foo", 3,
> > > > > (short) 2,
> > > > > > > > > > > > false).get();
> > > > > > > > > > > > > > > > > > > >  >   Collection<String> topicNames =
> > > > > > > > > > > > > > client.topics().list(false).
> > > > > > > > > > > > > > > > > get();
> > > > > > > > > > > > > > > > > > > >  >   log.info("Found topics: {}",
> > > > > > > > > > > > Utils.mkString(topicNames,
> > > > > > > > > > > > > > ", "));
> > > > > > > > > > > > > > > > > > > >  >   Collection<Node> nodes =
> > > > > > > > > > client.nodes().list().get();
> > > > > > > > > > > > > > > > > > > >  >   log.info("Found cluster nodes:
> > {}",
> > > > > > > > > > > > > > Utils.mkString(nodes, ",
> > > > > > > > > > > > > > > > > "));
> > > > > > > > > > > > > > > > > > > >  > } finally {
> > > > > > > > > > > > > > > > > > > >  >   client.close();
> > > > > > > > > > > > > > > > > > > >  > }
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > The good thing is, there is no Try,
> no
> > > 'get'
> > > > > > > > prefixes,
> > > > > > > > > > no
> > > > > > > > > > > > > > messing
> > > > > > > > > > > > > > > > > with
> > > > > > > > > > > > > > > > > > > > batch APIs.  If there is an error,
> then
> > > > > > > > Future#get()
> > > > > > > > > > > > throws an
> > > > > > > > > > > > > > > > > > > > ExecutionException which wraps the
> > > relevant
> > > > > > > > exception
> > > > > > > > > > in
> > > > > > > > > > > > the
> > > > > > > > > > > > > > standard
> > > > > > > > > > > > > > > > > > > > Java way.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > Here's a slightly less simple
> example:
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > AdminClient client = new
> > > > > > > > AdminClientImpl(myConfig);
> > > > > > > > > > > > > > > > > > > > > try {
> > > > > > > > > > > > > > > > > > > > >   List<Future<Void>> futures = new
> > > > > > > > LinkedList<>();
> > > > > > > > > > > > > > > > > > > > >   for (String topicName:
> > > myNewTopicNames) {
> > > > > > > > > > > > > > > > > > > > >     creations.add(client.topics().
> > > > > > > > > > > > > > > > > > > > >
>  setClientTimeout(30000).setCr
> > > > > > > > eationConfig(
> > > > > > > > > > > > > > myTopicConfig).
> > > > > > > > > > > > > > > > > > > > >           create(topicName, 3,
> > (short)
> > > 2,
> > > > > > > > false));
> > > > > > > > > > > > > > > > > > > > >   }
> > > > > > > > > > > > > > > > > > > > >   Futures.waitForAll(futures);
> > > > > > > > > > > > > > > > > > > > > } finally {
> > > > > > > > > > > > > > > > > > > > >   client.close();
> > > > > > > > > > > > > > > > > > > > > }
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I went with Futures because I feel
> like
> > > > > ought to
> > > > > > > > have
> > > > > > > > > > some
> > > > > > > > > > > > > > option for
> > > > > > > > > > > > > > > > > > > > doing async.  It's a style of
> > programming
> > > > > that has
> > > > > > > > > > become a
> > > > > > > > > > > > > > lot more
> > > > > > > > > > > > > > > > > > > > popular with the rise of Node.js,
> > Twisted
> > > > > python,
> > > > > > > > etc.
> > > > > > > > > > etc.
> > > > > > > > > > > > > > Also, as
> > > > > > > > > > > > > > > > > > > > Ismael commented, Java 8
> > > CompletableFuture is
> > > > > > > > going to
> > > > > > > > > > make
> > > > > > > > > > > > > > Java's
> > > > > > > > > > > > > > > > > > > > support for fluent async programming
> a
> > > lot
> > > > > > > > stronger by
> > > > > > > > > > > > > > allowing call
> > > > > > > > > > > > > > > > > > > > chaining and much more.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > If we are going to support async, the
> > > > > simplest
> > > > > > > > thing is
> > > > > > > > > > > > just
> > > > > > > > > > > > > > to make
> > > > > > > > > > > > > > > > > > > > everything return a future and let
> > people
> > > > > call
> > > > > > > > get() if
> > > > > > > > > > > > they
> > > > > > > > > > > > > > want to
> > > > > > > > > > > > > > > > > run
> > > > > > > > > > > > > > > > > > > > synchronously.  Having a mix of async
> > and
> > > > > sync
> > > > > > > > APIs is
> > > > > > > > > > just
> > > > > > > > > > > > > > going to
> > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > confusing and redundant.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I think we should try to avoid
> creating
> > > > > single
> > > > > > > > > > functions
> > > > > > > > > > > > that
> > > > > > > > > > > > > > start
> > > > > > > > > > > > > > > > > > > > multiple requests if we can.  It
> makes
> > > > > things much
> > > > > > > > > > > > uglier.  It
> > > > > > > > > > > > > > means
> > > > > > > > > > > > > > > > > > > > that you have to have some kind of
> > > request
> > > > > class
> > > > > > > > that
> > > > > > > > > > > > wraps up
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > request the user is trying to create,
> > so
> > > > > that you
> > > > > > > > can
> > > > > > > > > > > > handle an
> > > > > > > > > > > > > > > > > array of
> > > > > > > > > > > > > > > > > > > > those requests.  The return value has
> > to
> > > be
> > > > > > > > something
> > > > > > > > > > like
> > > > > > > > > > > > > > Map<Node,
> > > > > > > > > > > > > > > > > > > > Try<Value>> to represent which nodes
> > > failed
> > > > > and
> > > > > > > > > > succeeded.
> > > > > > > > > > > > > > This is
> > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > kind of stuff that, in my opinion,
> > makes
> > > > > people
> > > > > > > > scratch
> > > > > > > > > > > > their
> > > > > > > > > > > > > > heads.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > If we need to, we can still get some
> of
> > > the
> > > > > > > > efficiency
> > > > > > > > > > > > > > benefits of
> > > > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > > > APIs by waiting for a millisecond or
> > two
> > > > > before
> > > > > > > > sending
> > > > > > > > > > > > out a
> > > > > > > > > > > > > > topic
> > > > > > > > > > > > > > > > > > > > create() request to see if other
> > create()
> > > > > requests
> > > > > > > > > > > > arrive.  If
> > > > > > > > > > > > > > so, we
> > > > > > > > > > > > > > > > > > > > can coalesce them.  It might be
> better
> > to
> > > > > figure
> > > > > > > > out if
> > > > > > > > > > > > this
> > > > > > > > > > > > > > is an
> > > > > > > > > > > > > > > > > > > > actual performance issue before
> > > implementing
> > > > > it,
> > > > > > > > > > though.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > I think it would be good to get
> > > something out
> > > > > > > > there,
> > > > > > > > > > > > annotate
> > > > > > > > > > > > > > it as
> > > > > > > > > > > > > > > > > > > > @Unstable, and get feedback from
> people
> > > > > building
> > > > > > > > > > against
> > > > > > > > > > > > trunk
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > using
> > > > > > > > > > > > > > > > > > > > it.  We have removed or changed
> > @Unstable
> > > > > APIs in
> > > > > > > > > > streams
> > > > > > > > > > > > > > before, so
> > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > don't think we should worry that it
> > will
> > > get
> > > > > set in
> > > > > > > > > > stone
> > > > > > > > > > > > > > > > > prematurely.
> > > > > > > > > > > > > > > > > > > > The AdminClient API should get much
> > less
> > > > > developer
> > > > > > > > use
> > > > > > > > > > than
> > > > > > > > > > > > > > anything
> > > > > > > > > > > > > > > > > in
> > > > > > > > > > > > > > > > > > > > streams, so changing an unstable API
> > > should
> > > > > be much
> > > > > > > > > > easier.
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > best,
> > > > > > > > > > > > > > > > > > > > Colin
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > On Wed, Feb 8, 2017, at 07:49, Ismael
> > > Juma
> > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > Thanks for elaborating Jay. I
> totally
> > > > > agree that
> > > > > > > > we
> > > > > > > > > > have
> > > > > > > > > > > > to
> > > > > > > > > > > > > > be very
> > > > > > > > > > > > > > > > > > > > > careful
> > > > > > > > > > > > > > > > > > > > > in how we use our complexity
> budget.
> > > > > Easier said
> > > > > > > > than
> > > > > > > > > > > > done
> > > > > > > > > > > > > > when
> > > > > > > > > > > > > > > > > people
> > > > > > > > > > > > > > > > > > > > > don't agree on what is complex and
> > > what is
> > > > > > > > simple.
> > > > > > > > > > :) For
> > > > > > > > > > > > > > example,
> > > > > > > > > > > > > > > > > I
> > > > > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > > batch APIs are a significant source
> > of
> > > > > > > > complexity as
> > > > > > > > > > you
> > > > > > > > > > > > > > have to
> > > > > > > > > > > > > > > > > do a
> > > > > > > > > > > > > > > > > > > > > bunch
> > > > > > > > > > > > > > > > > > > > > of ceremony to group things before
> > > sending
> > > > > the
> > > > > > > > > > request
> > > > > > > > > > > > and
> > > > > > > > > > > > > > error
> > > > > > > > > > > > > > > > > handling
> > > > > > > > > > > > > > > > > > > > > becomes more complex due to partial
> > > > > failures
> > > > > > > > (things
> > > > > > > > > > like
> > > > > > > > > > > > > > `Try` or
> > > > > > > > > > > > > > > > > other
> > > > > > > > > > > > > > > > > > > > > mechanisms that serve a similar
> role
> > > are
> > > > > then
> > > > > > > > > > needed).
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Maybe a way forward is to write API
> > > usage
> > > > > > > > examples to
> > > > > > > > > > > > help
> > > > > > > > > > > > > > > > > validate that
> > > > > > > > > > > > > > > > > > > > > the suggested API is indeed easy to
> > > use.
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > Ismael
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > On Wed, Feb 8, 2017 at 4:40 AM, Jay
> > > Kreps <
> > > > > > > > > > > > j...@confluent.io>
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Totally agree on
> CompletableFuture.
> > > Also
> > > > > agree
> > > > > > > > with
> > > > > > > > > > > > some
> > > > > > > > > > > > > > of the
> > > > > > > > > > > > > > > > > rough
> > > > > > > > > > > > > > > > > > > > edges
> > > > > > > > > > > > > > > > > > > > > > on the Consumer.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > I don't have much of a leg to
> stand
> > > on
> > > > > with the
> > > > > > > > > > > > splitting
> > > > > > > > > > > > > > vs not
> > > > > > > > > > > > > > > > > > > > splitting
> > > > > > > > > > > > > > > > > > > > > > thing, really hard to argue one
> or
> > > the
> > > > > other is
> > > > > > > > > > > > better. I
> > > > > > > > > > > > > > guess
> > > > > > > > > > > > > > > > > the one
> > > > > > > > > > > > > > > > > > > > > > observation in watching us try to
> > > make
> > > > > good
> > > > > > > > public
> > > > > > > > > > apis
> > > > > > > > > > > > > > over the
> > > > > > > > > > > > > > > > > years
> > > > > > > > > > > > > > > > > > > > is I
> > > > > > > > > > > > > > > > > > > > > > am kind of in favor of a
> particular
> > > kind
> > > > > of
> > > > > > > > > > simple. In
> > > > > > > > > > > > > > > > > particular I
> > > > > > > > > > > > > > > > > > > > think
> > > > > > > > > > > > > > > > > > > > > > since the bar is sooo high in
> > support
> > > > > and docs
> > > > > > > > and
> > > > > > > > > > the
> > > > > > > > > > > > > > community
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > users
> > > > > > > > > > > > > > > > > > > > > > so broad in the range of their
> > > > > capabilities, it
> > > > > > > > > > makes
> > > > > > > > > > > > it so
> > > > > > > > > > > > > > > > > there is a
> > > > > > > > > > > > > > > > > > > > lot
> > > > > > > > > > > > > > > > > > > > > > of value in dead simple
> interfaces
> > > that
> > > > > don't
> > > > > > > > have
> > > > > > > > > > a
> > > > > > > > > > > > lot of
> > > > > > > > > > > > > > > > > conceptual
> > > > > > > > > > > > > > > > > > > > > > weight, don't introduce a lot of
> > new
> > > > > classes or
> > > > > > > > > > > > concepts or
> > > > > > > > > > > > > > > > > general
> > > > > > > > > > > > > > > > > > > > > > patterns that must be understood
> to
> > > use
> > > > > them
> > > > > > > > > > > > correctly. So
> > > > > > > > > > > > > > > > > things like
> > > > > > > > > > > > > > > > > > > > > > nesting, or the Try class, or
> async
> > > > > apis, or
> > > > > > > > even
> > > > > > > > > > just
> > > > > > > > > > > > a
> > > > > > > > > > > > > > complex
> > > > > > > > > > > > > > > > > set of
> > > > > > > > > > > > > > > > > > > > > > classes representing arguments or
> > > return
> > > > > values
> > > > > > > > > > kind of
> > > > > > > > > > > > > > all stack
> > > > > > > > > > > > > > > > > > > > > > conceptual burdens on the user to
> > > figure
> > > > > out
> > > > > > > > > > correct
> > > > > > > > > > > > > > usage. So
> > > > > > > > > > > > > > > > > like,
> > > > > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > > > example, the Try class is very
> > > elegant
> > > > > and
> > > > > > > > > > represents a
> > > > > > > > > > > > > > whole
> > > > > > > > > > > > > > > > > > > > generalized
> > > > > > > > > > > > > > > > > > > > > > class of possibly completed
> > actions,
> > > but
> > > > > the
> > > > > > > > flip
> > > > > > > > > > side
> > > > > > > > > > > > is
> > > > > > > > > > > > > > maybe
> > > > > > > > > > > > > > > > > I'm
> > > > > > > > > > > > > > > > > > > > just a
> > > > > > > > > > > > > > > > > > > > > > working guy who needs to list his
> > > kafka
> > > > > topics
> > > > > > > > but
> > > > > > > > > > > > doesn't
> > > > > > > > > > > > > > know
> > > > > > > > > > > > > > > > > Rust,
> > > > > > > > > > > > > > > > > > > > take
> > > > > > > > > > > > > > > > > > > > > > pity on me! :-)
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > Nit picking aside, super excited
> to
> > > see
> > > > > us
> > > > > > > > > > progress on
> > > > > > > > > > > > > > this.
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > -Jay
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > On Tue, Feb 7, 2017 at 3:46 PM
> > Ismael
> > > > > Juma <
> > > > > > > > > > > > > > ism...@juma.me.uk>
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Hi Jay,
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Thanks for the feedback.
> Comments
> > > > > inline.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > On Tue, Feb 7, 2017 at 8:18 PM,
> > Jay
> > > > > Kreps <
> > > > > > > > > > > > > > j...@confluent.io>
> > > > > > > > > > > > > > > > > wrote:
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >    - I think it would be good
> > to
> > > not
> > > > > use
> > > > > > > > "get"
> > > > > > > > > > as
> > > > > > > > > > > > the
> > > > > > > > > > > > > > prefix
> > > > > > > > > > > > > > > > > for
> > > > > > > > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > > > > > > > >    making remote calls. We've
> > > tried
> > > > > to
> > > > > > > > avoid
> > > > > > > > > > the
> > > > > > > > > > > > java
> > > > > > > > > > > > > > getter
> > > > > > > > > > > > > > > > > > > > convention
> > > > > > > > > > > > > > > > > > > > > > > >    entirely (see code style
> > > guide),
> > > > > but for
> > > > > > > > > > remote
> > > > > > > > > > > > > > calls in
> > > > > > > > > > > > > > > > > > > > particular
> > > > > > > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > > > > > kind
> > > > > > > > > > > > > > > > > > > > > > > >    of blurs the line between
> > > field
> > > > > access
> > > > > > > > and
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > RPC in
> > > > > > > > > > > > > > > > > a way
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > > leads
> > > > > > > > > > > > > > > > > > > > > > > >    people to trouble. What
> > about,
> > > > > e.g.,
> > > > > > > > > > > > > > fetchAllGroups() vs
> > > > > > > > > > > > > > > > > > > > > > > getAllGroups().
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Agreed that we should avoid the
> > > `get`
> > > > > prefix
> > > > > > > > for
> > > > > > > > > > > > remote
> > > > > > > > > > > > > > calls.
> > > > > > > > > > > > > > > > > There
> > > > > > > > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > few possible prefixes for the
> > read
> > > > > > > > operations:
> > > > > > > > > > list,
> > > > > > > > > > > > > > fetch,
> > > > > > > > > > > > > > > > > describe.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >    - I think futures and
> > > callbacks
> > > > > are a
> > > > > > > > bit
> > > > > > > > > > of a
> > > > > > > > > > > > pain
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > use. I'd
> > > > > > > > > > > > > > > > > > > > > > second
> > > > > > > > > > > > > > > > > > > > > > > >    Becket's comment: let's
> > ensure
> > > > > there a
> > > > > > > > > > common
> > > > > > > > > > > > use
> > > > > > > > > > > > > > case
> > > > > > > > > > > > > > > > > > > > motivating
> > > > > > > > > > > > > > > > > > > > > > > these
> > > > > > > > > > > > > > > > > > > > > > > >    that wouldn't be just as
> > > easily
> > > > > > > > satisfied
> > > > > > > > > > with
> > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > operations
> > > > > > > > > > > > > > > > > > > > > > (which
> > > > > > > > > > > > > > > > > > > > > > > > we
> > > > > > > > > > > > > > > > > > > > > > > >    seem to have at least for
> > some
> > > > > things).
> > > > > > > > In
> > > > > > > > > > > > terms of
> > > > > > > > > > > > > > > > > flexibility
> > > > > > > > > > > > > > > > > > > > > > > > Callbacks >
> > > > > > > > > > > > > > > > > > > > > > > >    Futures > Batch Ops but I
> > > think in
> > > > > > > > terms of
> > > > > > > > > > > > > > usability it
> > > > > > > > > > > > > > > > > is the
> > > > > > > > > > > > > > > > > > > > > > exact
> > > > > > > > > > > > > > > > > > > > > > > >    opposite so let's make
> sure
> > we
> > > > > have
> > > > > > > > worked
> > > > > > > > > > out
> > > > > > > > > > > > how
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > API will
> > > > > > > > > > > > > > > > > > > > be
> > > > > > > > > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > > > > > > >    before deciding. In
> > > particular I
> > > > > think
> > > > > > > > java
> > > > > > > > > > > > Futures
> > > > > > > > > > > > > > are
> > > > > > > > > > > > > > > > > often an
> > > > > > > > > > > > > > > > > > > > > > > >    uncomfortable half-way
> point
> > > since
> > > > > > > > calling
> > > > > > > > > > > > get() and
> > > > > > > > > > > > > > > > > blocking
> > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > > thread is
> > > > > > > > > > > > > > > > > > > > > > > >    often not what you want
> for
> > > > > chaining
> > > > > > > > > > sequences
> > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > operations in
> > > > > > > > > > > > > > > > > > > > a
> > > > > > > > > > > > > > > > > > > > > > > truly
> > > > > > > > > > > > > > > > > > > > > > > >    async way, so 99% of
> people
> > > just
> > > > > use the
> > > > > > > > > > future
> > > > > > > > > > > > as
> > > > > > > > > > > > > > a way
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > batch
> > > > > > > > > > > > > > > > > > > > > > > calls.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > We should definitely figure out
> > how
> > > > > the APIs
> > > > > > > > are
> > > > > > > > > > > > going
> > > > > > > > > > > > > > to be
> > > > > > > > > > > > > > > > > used
> > > > > > > > > > > > > > > > > > > > before
> > > > > > > > > > > > > > > > > > > > > > > deciding. I agree that
> callbacks
> > > are
> > > > > > > > definitely
> > > > > > > > > > > > painful
> > > > > > > > > > > > > > and
> > > > > > > > > > > > > > > > > there's
> > > > > > > > > > > > > > > > > > > > > > little
> > > > > > > > > > > > > > > > > > > > > > > reason to expose them in a
> modern
> > > API
> > > > > unless
> > > > > > > > it's
> > > > > > > > > > > > meant
> > > > > > > > > > > > > > to be
> > > > > > > > > > > > > > > > > very
> > > > > > > > > > > > > > > > > > > > low
> > > > > > > > > > > > > > > > > > > > > > > level. When it comes to
> Futures,
> > I
> > > > > think it's
> > > > > > > > > > > > important
> > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > distinguish
> > > > > > > > > > > > > > > > > > > > > > what
> > > > > > > > > > > > > > > > > > > > > > > is available in Java 7 and
> below
> > > > > versus what
> > > > > > > > is
> > > > > > > > > > > > > > available from
> > > > > > > > > > > > > > > > > Java 8
> > > > > > > > > > > > > > > > > > > > > > > onwards. CompletableFuture
> makes
> > it
> > > > > much
> > > > > > > > easier
> > > > > > > > > > to
> > > > > > > > > > > > > > > > > compose/chain
> > > > > > > > > > > > > > > > > > > > > > operations
> > > > > > > > > > > > > > > > > > > > > > > (in a similar vein to
> > > > > java.util.Stream, our
> > > > > > > > own
> > > > > > > > > > > > Streams
> > > > > > > > > > > > > > API,
> > > > > > > > > > > > > > > > > etc.)
> > > > > > > > > > > > > > > > > > > > and it
> > > > > > > > > > > > > > > > > > > > > > > gives you the ability to
> register
> > > > > callbacks
> > > > > > > > if
> > > > > > > > > > you
> > > > > > > > > > > > > > really want
> > > > > > > > > > > > > > > > > to
> > > > > > > > > > > > > > > > > > > > > > (avoiding
> > > > > > > > > > > > > > > > > > > > > > > the somewhat odd situation in
> the
> > > > > Producer
> > > > > > > > where
> > > > > > > > > > we
> > > > > > > > > > > > > > return a
> > > > > > > > > > > > > > > > > Future
> > > > > > > > > > > > > > > > > > > > _and_
> > > > > > > > > > > > > > > > > > > > > > > allow you to pass a callback).
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > >    - Personally I don't think
> > > > > splitting the
> > > > > > > > > > admin
> > > > > > > > > > > > > > methods up
> > > > > > > > > > > > > > > > > > > > actually
> > > > > > > > > > > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > > > > > > > >    things more usable. It
> just
> > > makes
> > > > > you
> > > > > > > > have
> > > > > > > > > > to
> > > > > > > > > > > > dig
> > > > > > > > > > > > > > through
> > > > > > > > > > > > > > > > > our
> > > > > > > > > > > > > > > > > > > > > > > > hierarchy. I
> > > > > > > > > > > > > > > > > > > > > > > >    think a flat class with a
> > > bunch of
> > > > > > > > > > operations
> > > > > > > > > > > > (like
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > consumer
> > > > > > > > > > > > > > > > > > > > > > api)
> > > > > > > > > > > > > > > > > > > > > > > is
> > > > > > > > > > > > > > > > > > > > > > > >    probably the easiest for
> > > people
> > > > > to grok
> > > > > > > > and
> > > > > > > > > > find
> > > > > > > > > > > > > > things
> > > > > > > > > > > > > > > > > on. I am
> > > > > > > > > > > > > > > > > > > > > > kind
> > > > > > > > > > > > > > > > > > > > > > > > of a
> > > > > > > > > > > > > > > > > > > > > > > >    dumb PHP programmer at
> > heart,
> > > > > though.
> > > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > I am not sure it's fair to
> > compare
> > > the
> > > > > > > > > > AdminClient
> > > > > > > > > > > > with
> > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > Consumer. The
> > > > > > > > > > > > > > > > > > > > > > > former has APIs for a bunch of
> > > > > unrelated APIs
> > > > > > > > > > > > (topics,
> > > > > > > > > > > > > > ACLs,
> > > > > > > > > > > > > > > > > configs,
> > > > > > > > > > > > > > > > > > > > > > > consumer groups, delegation
> > tokens,
> > > > > preferred
> > > > > > > > > > leader
> > > > > > > > > > > > > > election,
> > > > > > > > > > > > > > > > > > > > partition
> > > > > > > > > > > > > > > > > > > > > > > reassignment, etc.) where the
> > > latter is
> > > > > > > > pretty
> > > > > > > > > > > > > > specialised.
> > > > > > > > > > > > > > > > > For each
> > > > > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > > > > > > > resources, you may have 3-4
> > > > > operations, it
> > > > > > > > will
> > > > > > > > > > get
> > > > > > > > > > > > > > confusing
> > > > > > > > > > > > > > > > > fast.
> > > > > > > > > > > > > > > > > > > > Also,
> > > > > > > > > > > > > > > > > > > > > > > do you really think an API that
> > > has one
> > > > > > > > level of
> > > > > > > > > > > > > > grouping will
> > > > > > > > > > > > > > > > > mean
> > > > > > > > > > > > > > > > > > > > that
> > > > > > > > > > > > > > > > > > > > > > > users have to "dig through our
> > > > > hierarchy"? Or
> > > > > > > > > > are you
> > > > > > > > > > > > > > > > > concerned that
> > > > > > > > > > > > > > > > > > > > once
> > > > > > > > > > > > > > > > > > > > > > > we go in that direction, there
> > is a
> > > > > danger of
> > > > > > > > > > making
> > > > > > > > > > > > the
> > > > > > > > > > > > > > > > > hierarchy
> > > > > > > > > > > > > > > > > > > > more
> > > > > > > > > > > > > > > > > > > > > > > complicated?
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Finally, I am not sure I would
> > use
> > > the
> > > > > > > > consumer
> > > > > > > > > > as an
> > > > > > > > > > > > > > example
> > > > > > > > > > > > > > > > > of
> > > > > > > > > > > > > > > > > > > > > > something
> > > > > > > > > > > > > > > > > > > > > > > that is easy to grok. :) The
> fact
> > > that
> > > > > > > > methods
> > > > > > > > > > behave
> > > > > > > > > > > > > > pretty
> > > > > > > > > > > > > > > > > > > > differently
> > > > > > > > > > > > > > > > > > > > > > > (some are blocking while others
> > > only
> > > > > have an
> > > > > > > > > > effect
> > > > > > > > > > > > after
> > > > > > > > > > > > > > > > > poll) with
> > > > > > > > > > > > > > > > > > > > no
> > > > > > > > > > > > > > > > > > > > > > > indication from the type
> > signature
> > > or
> > > > > naming
> > > > > > > > > > > > convention
> > > > > > > > > > > > > > makes
> > > > > > > > > > > > > > > > > it
> > > > > > > > > > > > > > > > > > > > harder,
> > > > > > > > > > > > > > > > > > > > > > > not easier, to understand.
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > > > Ismael
> > > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > > > > >
> > > > > > > > > > > > > >
> > > > > > > > > > > >
> > > > > > > > > >
> > > > > > > >
> > > > >
> > >
> >
> >
> >
> > --
> > *Gwen Shapira*
> > Product Manager | Confluent
> > 650.450.2760 | @gwenshap
> > Follow us: Twitter <https://twitter.com/ConfluentInc> | blog
> > <http://www.confluent.io/blog>
> >
>

Reply via email to