[
https://issues.apache.org/jira/browse/SAMZA-123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13981307#comment-13981307
]
Sriram Subramanian commented on SAMZA-123:
------------------------------------------
Posting on JIRA
1. Most of the explanation about retaining the checkpoint and state by task id
applies only for the first two grouping strategies. The group by partitions and
group by SSP maintain the task / partition mapping on adding more partitions. I
agree that changing the grouping strategy should be undefined behavior and we
can try to warn. However, there are cases where the group strategy could be the
same and yet the partitions can be mapped to different tasks. For example, in
the case of GroupIntoNSets, you could have the same strategy but changing the
number of TIs (in order to scale for changing profile over time) would map the
partitions to a different task instance. Now are the checkpoint and state
information for that task valid? We can try to warn by storing the information
of the task id and the partitions it was mapped to previously but it gets to be
confusing. I would like to understand how would we explain the grouping
strategy, state and checkpoint to work consistently across different grouping
strategies. If we cannot do that, we should not make this feature extensible
and rather provide fixed strategies.
2. My vote for the naming would be taskName if it is a string and taskId if we
can map all use cases to ids. Also the naming for the getStorePartitionDir need
to be something like
def getStoreTaskDir(storeBaseDir: File, storeName: String, taskName: String) = {
new File(storeBaseDir, storeName + File.separator + taskName)
3. I am confused about how this grouping strategy can be extended by the user
of the framework. Ideally, one should be able to implement the grouping
strategy and provide that as a value in the framework config
"grouping.strategy=org.apache.mypackage.MyGroupingStrategy. However, with your
third strategy, it requires an additional config to be added to the framework.
Is the expectation that anyone who extends the grouping strategy need to add
their own configs to the framework and wire them in?
4. It would also be useful to talk about how does these grouping strategies map
to a SQL like way of expressing a Samza topology. Are there any other grouping
strategies that might require a lot more change in the framework that just
implementing this API?
5. Finally, unless we are very sure about this working out well, we should not
make this a public API. The framework can provide three strategies and those
are the only available strategies for now. Opening it to be extensible might
cause us to break a lot of client logic if we find our assumptions were wrong.
> 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)