[
https://issues.apache.org/jira/browse/FLINK-1725?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14716619#comment-14716619
]
ASF GitHub Bot commented on FLINK-1725:
---------------------------------------
Github user mbalassi commented on the pull request:
https://github.com/apache/flink/pull/1069#issuecomment-135418178
Thanks for the fast response on your side addressing my comments. :)
I generally love the idea described at
[FLINK-1725](https://issues.apache.org/jira/browse/FLINK-1725), I have a couple
of concerns though:
1. Having the code in is nice, but we need a way to surface it in the
`DataStream` API. The current implementation can not do that in a
straight-forward way, as you need the number of output channels in a
constructor parameter. That is an information that we do not have when creating
the partitioning strategy for the data stream yet (E.g.
[here](https://github.com/apache/flink/blob/master/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java#L502-L504)).
2. The comment you have added to `PartialPartitioner` did not give me a
clear picture of the behaviour. I much prefer the one you wrote
[here](https://issues.apache.org/jira/browse/FLINK-1725?focusedCommentId=14371032&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14371032).
Some comment would be nice on the test side too, I only got it after reading
the aforementioned JIRA issue.
3. How sensitive are your algorithms fed by this partitioning to loosing
the state in `targetChannelTasks`? Because if a Flink operator was to go down
this information would not be recovered after failure.
4. I share the wish of @gdfm that it would be really nice if the
implementation did not depend on `toString`.
> New Partitioner for better load balancing for skewed data
> ---------------------------------------------------------
>
> Key: FLINK-1725
> URL: https://issues.apache.org/jira/browse/FLINK-1725
> Project: Flink
> Issue Type: Improvement
> Components: New Components
> Affects Versions: 0.8.1
> Reporter: Anis Nasir
> Assignee: Anis Nasir
> Labels: LoadBalancing, Partitioner
> Original Estimate: 336h
> Remaining Estimate: 336h
>
> Hi,
> We have recently studied the problem of load balancing in Storm [1].
> In particular, we focused on key distribution of the stream for skewed data.
> We developed a new stream partitioning scheme (which we call Partial Key
> Grouping). It achieves better load balancing than key grouping while being
> more scalable than shuffle grouping in terms of memory.
> In the paper we show a number of mining algorithms that are easy to implement
> with partial key grouping, and whose performance can benefit from it. We
> think that it might also be useful for a larger class of algorithms.
> Partial key grouping is very easy to implement: it requires just a few lines
> of code in Java when implemented as a custom grouping in Storm [2].
> For all these reasons, we believe it will be a nice addition to the standard
> Partitioners available in Flink. If the community thinks it's a good idea, we
> will be happy to offer support in the porting.
> References:
> [1].
> https://melmeric.files.wordpress.com/2014/11/the-power-of-both-choices-practical-load-balancing-for-distributed-stream-processing-engines.pdf
> [2]. https://github.com/gdfm/partial-key-grouping
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)