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

Matthias J. Sax commented on KAFKA-15520:
-----------------------------------------

{quote}However, I'm seeing that when pods restart - it triggers rebalances and 
causes processing to be paused on all pods till the rebalance and state restore 
is in progress.a
{quote}
and
{quote}I have increased session timeout to 480 seconds
{quote}
How long does a pod restart take? It is quicker than 480 seconds? – If yes, no 
rebalance should be triggered. (`max.poll.interval.ms` config would not make a 
difference for this case)
{quote}My understanding is that even if there is a rebalance - only the 
partitions that should be moved around will be restored in a cooperative way 
and not pause all the processing.
{quote}
That's right, however, in KS if a thread gets a new task assigned for which 
state must be restored, KS expliclity pauses processing for all other task to 
put all work into restoring to reduce restore latency (in some version of KS we 
tried to interleave processing of active task plus restoring, but there was 
complaints that it slows down restoring too much – with the new "state updated 
thread" we are adding, we aim to allow processing and restoring to happen in 
parallel again in future versions – maybe 3.7 if we can get it over the finish 
line).
{quote}Also, it should failover to standby replica in this case and avoid state 
restoring on other pods.
{quote}
Yes. Not sure why this does not happen, but you are using a somewhat older 
version of Kafka Streams – we put a lot of work into fixing bugs to this end, 
so it would be best to upgrade to 3.5 to see if the issues are already fiexed.

> Kafka Streams Stateful Aggregation Rebalancing causing processing to pause on 
> all partitions
> --------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-15520
>                 URL: https://issues.apache.org/jira/browse/KAFKA-15520
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 2.6.2
>            Reporter: Rohit Bobade
>            Priority: Major
>
> Kafka broker version: 2.8.0 Kafka Streams client version: 2.6.2
> I am running kafka streams stateful aggregations on K8s statefulset with 
> persistent volume attached to each pod. I have also specified
> props.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, podName);
> which makes sure it gets the sticky partition assignment.
> Enabled standby replica - 
> props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1);
> and set props.put(StreamsConfig.ACCEPTABLE_RECOVERY_LAG_CONFIG, "0");
> However, I'm seeing that when pods restart - it triggers rebalances and 
> causes processing to be paused on all pods till the rebalance and state 
> restore is in progress.
> My understanding is that even if there is a rebalance - only the partitions 
> that should be moved around will be restored in a cooperative way and not 
> pause all the processing. Also, it should failover to standby replica in this 
> case and avoid state restoring on other pods.
> I have increased session timeout to 480 seconds and max poll interval to 15 
> mins to minimize rebalances.
> Also added
> props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, 
> CooperativeStickyAssignor.class.getName());
> to enable CooperativeStickyAssignor
> could someone please help if I'm missing something?
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to