[ 
https://issues.apache.org/jira/browse/KAFKA-2350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14643442#comment-14643442
 ] 

Jason Gustafson commented on KAFKA-2350:
----------------------------------------

[~becket_qin] I think that we're on the same page as far as supporting only 
automatic or manual assignment and not trying to mix them. I think my confusion 
is that subscribe(partition) in your proposal is used both a) to subscribe to a 
partition when manual assignment is used, and b) to unpause a partition when 
automatic assignment is used. This leads to the weird ordering problems that we 
have been talking about. By the way, I added the line about seek() and 
position() since it seems like something that intuitively should be supported 
by pause semantics. I think it's debatable whether it's really needed, but I 
think it would cause a bit a surprise to the user if we didn't allow it.

> Add KafkaConsumer pause capability
> ----------------------------------
>
>                 Key: KAFKA-2350
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2350
>             Project: Kafka
>          Issue Type: Improvement
>            Reporter: Jason Gustafson
>            Assignee: Jason Gustafson
>
> There are some use cases in stream processing where it is helpful to be able 
> to pause consumption of a topic. For example, when joining two topics, you 
> may need to delay processing of one topic while you wait for the consumer of 
> the other topic to catch up. The new consumer currently doesn't provide a 
> nice way to do this. If you skip calls to poll() or if you unsubscribe, then 
> a rebalance will be triggered and your partitions will be reassigned to 
> another consumer. The desired behavior is instead that you keep the partition 
> assigned and simply 
> One way to achieve this would be to add two new methods to KafkaConsumer:
> {code}
> void pause(TopicPartition... partitions);
> void resume(TopicPartition... partitions);
> {code}
> Here is the expected behavior of pause/resume:
> * When a partition is paused, calls to KafkaConsumer.poll will not initiate 
> any new fetches for that partition.
> * After the partition is resumed, fetches will begin again. 
> * While a partition is paused, seek() and position() can still be used to 
> advance or query the current position.
> * Rebalance does not preserve pause/resume state.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to