You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by "Jay Kreps (JIRA)" <ji...@apache.org> on 2014/04/25 02:35:15 UTC

[jira] [Commented] (SAMZA-123) Move topic partition grouping to the AM and generalize

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

Jay Kreps commented on SAMZA-123:
---------------------------------

Two ideas:
1. The first is to use the terminology task id rather than cohort (in the code, not the write-up). This is a minor issue but task id is more intuitive to me.
2. Something about the C2P log rubs me the wrong way. A cheaper way to store the mapping would just be in zk, although that would add a zk dependency from the AM. 

An alternative to the C2P topic would be to use the checkpoint topic to store the task definitions:

Tasks have an id 0, 1, ..., N and we store their definition (i.e. the id and its inputs). E.g. if we had inputs A and B and used the GroupBySSP strategy we would have:
 0: {A/0, B/0}, 1: {A/1, B/1}, ... N: {A/N, B/N}
The mapping to any change logs is just by the task id.

The grouping API is then
  interface SSPGrouper {
    public Map<Integer, Set<SystemStreamPartition> group(Map<Integer, Set<SystemStreamPartition> current, Set<SystemStreamPartition> ssps);
  }

Is this better or worse? Not sure. This version of the SSPGrouper api is less convenient I think but makes implementing validation easier.

I think the question is really what the fate of the checkpoint topic is. As you mention KAFKA-1000 is technically superior to what we are doing. However there are several additional things it would be good to discuss and think through:

The first is the additional metadata we would need to retain for fault tolerance semantics (the producer id and sequence number). These would be per task and need to be transactionally written with the offsets.

The other use of the checkpoint log was to be able to run an active-passive pair of clusters.

I *think* both of these things should work equally well with KAFKA-1000. but it would be good for us to just do the thinking since we have an intern coming in a month anyway to work on this...


> Move topic partition grouping to the AM and generalize
> ------------------------------------------------------
>
>                 Key: SAMZA-123
>                 URL: https://issues.apache.org/jira/browse/SAMZA-123
>             Project: Samza
>          Issue Type: Sub-task
>          Components: container
>    Affects Versions: 0.6.0
>            Reporter: Jakob Homan
>            Assignee: Jakob Homan
>         Attachments: SAMZA-123-design-doc.md, SAMZA-123-design-doc.pdf
>
>
> Currently the AM sends a set of all the topics and partitions to the container, which then groups them by partition and assigns each set to a task instance. By moving the grouping to the AM, we can assign arbitrary groups to task instances, which will allow more partitioning strategies, as discussed in SAMZA-71.



--
This message was sent by Atlassian JIRA
(v6.2#6252)