kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Joel Koshy <jjkosh...@gmail.com>
Subject Re: Consumer and offset management support in 0.8.2 and 0.9
Date Tue, 18 Nov 2014 19:55:15 GMT
Inline..

On Tue, Nov 18, 2014 at 04:26:04AM -0500, Marius Bogoevici wrote:
> Hello everyone,
> 
> I have a few questions about the current status and future of the Kafka
> consumers.
> 
> We have been working to adding Kafka support in Spring XD [1], currently
> using the high level consumer via Spring Integration Kafka [2]. We are
> working on adding features such as:
> - the ability to control offsets/replay topics;
> - the ability to control partition allocation across multiple consumers;
> 
> We are currently at version 0.8.1.1, so using the simple consumer is a
> pretty straightforward choice right now. However, in the light of the
> upcoming consumer changes for 0.8.2 and 0.9, I have a few questions:
> 
> 1) With respect to the consumer redesign for 0.9, what is the future of the
> Simple Consumer and High Level Consumer? To my best understanding, the
> existing high level consumer API will be deprecated in favour of the new
> consumer API. What is the future of the Simple Consumer, in this case? it
> will continue to exist as a low-level API implementing the Kafka protocol
> [3] and providing the building blocks for the new consumer, or will it be
> deprecated as well?

The new consumer will subsume both use-cases (simple and high-level).
You can still use the old SimpleConsumer if you wish - i.e., the wire
protocol for fetch and other requests will still be supported.

> 
> 2) Regarding the new consumer: the v0.8.2 codebase contains an early
> implementation of it, but since this a feature scheduled only for 0.9, what
> is its status as well? Is it included only as a future reference and for
> stabilizing the API?

It is a WIP so you cannot really use it.

> 3) Obviously, offset management is a concern if using the simple consumer,
> so - wondering about the Offset Management API as well. The Kafka protocol
> document specifically indicates that it will be fully functional in 0.8.2
> [4] - however, a functional implementation is already available in 0.8.1.1
> (accessible via the SimpleConsumer API but not documented in [5]). Again,
> trying to understand the extent of what 0.8.1.1 already supports
> (ostensibly, the offset manager support seems to have been added only in
> 0.8.2 - please correct me if I am wrong), and whether if it is recommended
> for use in production in any form (with the caveats that accompany the use
> of ZooKeeper).

In 0.8.2 the OffsetCommitRequest and OffsetFetchRequest will use Kafka
as the offsets storage mechanism (not zookeeper). High-level Java
consumers can choose to store offsets in ZooKeeper instead by setting
offsets.storage=zookeeper

However, if you are using the simple consumer and wish to store
offsets in ZooKeeper you will need to commit to ZooKeeper directly.
You can use ZkUtils in the kafka.utils package for this.

If you wish to move to Kafka-based offsets we will be adding a new
OffsetsClient that can be used to commit/fetch offsets to/from Kafka.
This is currently not listed as a blocker for 0.8.2 but I think we
should include it. I will update that ticket.

> 4) Trying to interpret the existing examples in [6] and the comments on [7]
> - the version of the Offset Management API that exists in 0.8.1.1 is using
> ZooKeeper - whereas ZooKeeper will be optional in 0.8.2 - to be replaced by
> Kafka, and phased out if possible. To my understanding, the switch between
> the two will be controlled by the broker configuration (along with other
> parameters that control the performance of offset queues. Is that correct?

The switch is a client-side configuration. That wiki is not
up-to-date. The most current documentation is available as a patch in
https://issues.apache.org/jira/browse/KAFKA-1729

> 5) Also, wondering about the timeline of 0.8.2 - according to the roadmaps
> it should be released relatively shortly. Is that correct?

Yes - once the blockers are ironed out.

> 
> Thanks,
> Marius
> 
> [1] http://projects.spring.io/spring-xd/
> [2] https://github.com/spring-projects/spring-integration-kafka
> [3]
> https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol
> [4]
> https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI
> [5] http://kafka.apache.org/082/documentation.html#simpleconsumerapi
> [6]
> https://cwiki.apache.org/confluence/display/KAFKA/Committing+and+fetching+consumer+offsets+in+Kafka
> [7] https://issues.apache.org/jira/browse/KAFKA-1729


Mime
View raw message