A few items:
1. ConsumerRebalanceCallback
   a. onPartitionsRevoked would be a better name.
   b. We should discuss the possibility of splitting this into two
interfaces. The motivation would be that in Java 8 single method interfaces
can directly take methods which might be more intuitive.
   c. If we stick with a single interface I would prefer the name
RebalanceCallback as its more concise
2. Should subscribe(String topic, int partition) should be subscribe(String
topic, int...partition)?
3. Is lastCommittedOffset call just a local access? If so it would be more
convenient not to batch it.
4. How are we going to handle the earliest/latest starting position
functionality we currently have. Does that remain a config?
5. Do we need to expose the general ability to get known positions from the
log? E.g. the functionality in the OffsetRequest...? That would make the
ability to change position a little easier.
6. Should poll(java.lang.Long timeout) be poll(long timeout, TimeUnit
unit)? Is it Long because it allows null? If so should we just add a poll()
that polls indefinitely?
7. I recommend we remove the boolean parameter from commit as it is really
hard to read code that has boolean parameters without named arguments. Can
we make it something like commit(...) and commitAsync(...)?
8. What about the common case where you just want to commit the current
position for all partitions?
9. How do you unsubscribe?
10. You say in a few places that positions() only impacts the starting
position, but surely that isn't the case, right? Surely it controls the
fetch position for that partition and can be called at any time? Otherwise
it is a pretty weird api, right?
11. How do I get my current position? Not the committed position but the
offset of the next message that will be given to me?

One thing that I really found helpful for the API design was writing out
actual code for different scenarios against the API. I think it might be
good to do that for this too--i.e. enumerate the various use cases and code
that use case up to see how it looks. I'm not sure if it would be useful to
collect these kinds of scenarios from people. I know they have sporadically
popped up on the mailing list.

-Jay


On Mon, Feb 10, 2014 at 10:54 AM, Neha Narkhede <neha.narkh...@gmail.com>wrote:

> As mentioned in previous emails, we are also working on a re-implementation
> of the consumer. I would like to use this email thread to discuss the
> details of the public API. I would also like us to be picky about this
> public api now so it is as good as possible and we don't need to break it
> in the future.
>
> The best way to get a feel for the API is actually to take a look at the
> javadoc<
> http://people.apache.org/~nehanarkhede/kafka-0.9-consumer-javadoc/doc/kafka/clients/consumer/KafkaConsumer.html
> >,
> the hope is to get the api docs good enough so that it is self-explanatory.
> You can also take a look at the configs
> here<
> http://people.apache.org/~nehanarkhede/kafka-0.9-consumer-javadoc/doc/kafka/clients/consumer/ConsumerConfig.html
> >
>
> Some background info on implementation:
>
> At a high level the primary difference in this consumer is that it removes
> the distinction between the "high-level" and "low-level" consumer. The new
> consumer API is non blocking and instead of returning a blocking iterator,
> the consumer provides a poll() API that returns a list of records. We think
> this is better compared to the blocking iterators since it effectively
> decouples the threading strategy used for processing messages from the
> consumer. It is worth noting that the consumer is entirely single threaded
> and runs in the user thread. The advantage is that it can be easily
> rewritten in less multi-threading-friendly languages. The consumer batches
> data and multiplexes I/O over TCP connections to each of the brokers it
> communicates with, for high throughput. The consumer also allows long poll
> to reduce the end-to-end message latency for low throughput data.
>
> The consumer provides a group management facility that supports the concept
> of a group with multiple consumer instances (just like the current
> consumer). This is done through a custom heartbeat and group management
> protocol transparent to the user. At the same time, it allows users the
> option to subscribe to a fixed set of partitions and not use group
> management at all. The offset management strategy defaults to Kafka based
> offset management and the API provides a way for the user to use a
> customized offset store to manage the consumer's offsets.
>
> A key difference in this consumer also is the fact that it does not depend
> on zookeeper at all.
>
> More details about the new consumer design are
> here<
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+0.9+Consumer+Rewrite+Design
> >
>
> Please take a look at the new
> API<
> http://people.apache.org/~nehanarkhede/kafka-0.9-consumer-javadoc/doc/kafka/clients/consumer/KafkaConsumer.html
> >and
> give us any thoughts you may have.
>
> Thanks,
> Neha
>

Reply via email to