kafka-jira mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Guozhang Wang (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (KAFKA-6106) Postpone normal processing of tasks within a thread until restoration of all tasks have completed
Date Fri, 27 Oct 2017 23:51:00 GMT

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

Guozhang Wang commented on KAFKA-6106:
--------------------------------------

I think it is better to enforce this behavior and here's my motivations:

1. I cannot actually think of a scenario that people would prefer start partial processing
asap while keeping the whole instance's state in `PARTITION_ASSIGNED`. But I can be convinced
if you have a concrete use case in mind.
2. I'd prefer to expose configs that are "easy to understand" for users and not leaking too
much internal runtime details. This config looks much more "leaky" and hard to understand
to end users, and hence I'm hesitant to introducing it to the user: usually if I find myself
needing more than 5 sentences to explain a config semantic, it is a good sign that probably
I should not add it.

My current PR assumes no config exposed, but we can keep this discussion on the ticket or
on the PR itself while we review it.


> Postpone normal processing of tasks within a thread until restoration of all tasks have
completed
> -------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-6106
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6106
>             Project: Kafka
>          Issue Type: Improvement
>          Components: streams
>    Affects Versions: 0.11.0.1, 1.0.0
>            Reporter: Guozhang Wang
>            Assignee: Guozhang Wang
>
> Let's say a stream thread hosts multiple tasks, A and B. At the very beginning when A
and B are assigned to the thread, the thread state is {{TASKS_ASSIGNED}}, and the thread start
restoring these two tasks during this state using the restore consumer while using normal
consumer for heartbeating.
> If task A's restoration has completed earlier than task B, then the thread will start
processing A immediately even when it is still in the {{TASKS_ASSIGNED}} phase. But processing
task A will slow down restoration of task B since it is single-thread. So the thread's transition
to {{RUNNING}} when all of its assigned tasks have completed restoring and now can be processed
will be delayed.
> Note that the streams instance's state will only transit to {{RUNNING}} when all of its
threads have transit to {{RUNNING}}, so the instance's transition will also be delayed by
this scenario.
> We'd better to not start processing ready tasks immediately, but instead focus on restoration
during the {{TASKS_ASSIGNED}} state to shorten the overall time of the instance's state transition.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message